| /* |
| * Licensed to the Apache Software Foundation (ASF) under one |
| * or more contributor license agreements. See the NOTICE file |
| * distributed with this work for additional information |
| * regarding copyright ownership. The ASF licenses this file |
| * to you under the Apache License, Version 2.0 (the |
| * "License"); you may not use this file except in compliance |
| * with the License. You may obtain a copy of the License at |
| * |
| * http://www.apache.org/licenses/LICENSE-2.0 |
| * |
| * Unless required by applicable law or agreed to in writing, software |
| * distributed under the License is distributed on an "AS IS" BASIS, |
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| * See the License for the specific language governing permissions and |
| * limitations under the License. |
| */ |
| package org.apache.cassandra.locator; |
| |
| import java.net.InetAddress; |
| import java.util.ArrayList; |
| import java.util.Collections; |
| import java.util.Collection; |
| import java.util.Iterator; |
| import java.util.List; |
| import java.util.Map; |
| |
| import org.slf4j.Logger; |
| import org.slf4j.LoggerFactory; |
| |
| import org.apache.cassandra.config.SchemaConstants; |
| import org.apache.cassandra.exceptions.ConfigurationException; |
| import org.apache.cassandra.dht.Token; |
| import org.apache.cassandra.service.ClientWarn; |
| import org.apache.cassandra.service.StorageService; |
| |
| |
| /** |
| * This class returns the nodes responsible for a given |
| * key but does not respect rack awareness. Basically |
| * returns the RF nodes that lie right next to each other |
| * on the ring. |
| */ |
| public class SimpleStrategy extends AbstractReplicationStrategy |
| { |
| private static final Logger logger = LoggerFactory.getLogger(SimpleStrategy.class); |
| |
| public SimpleStrategy(String keyspaceName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions) |
| { |
| super(keyspaceName, tokenMetadata, snitch, configOptions); |
| } |
| |
| public List<InetAddress> calculateNaturalEndpoints(Token token, TokenMetadata metadata) |
| { |
| int replicas = getReplicationFactor(); |
| ArrayList<Token> tokens = metadata.sortedTokens(); |
| List<InetAddress> endpoints = new ArrayList<InetAddress>(replicas); |
| |
| if (tokens.isEmpty()) |
| return endpoints; |
| |
| // Add the token at the index by default |
| Iterator<Token> iter = TokenMetadata.ringIterator(tokens, token, false); |
| while (endpoints.size() < replicas && iter.hasNext()) |
| { |
| InetAddress ep = metadata.getEndpoint(iter.next()); |
| if (!endpoints.contains(ep)) |
| endpoints.add(ep); |
| } |
| return endpoints; |
| } |
| |
| public int getReplicationFactor() |
| { |
| return Integer.parseInt(this.configOptions.get("replication_factor")); |
| } |
| |
| public void validateOptions() throws ConfigurationException |
| { |
| String rf = configOptions.get("replication_factor"); |
| if (rf == null) |
| throw new ConfigurationException("SimpleStrategy requires a replication_factor strategy option."); |
| validateReplicationFactor(rf); |
| } |
| |
| public void maybeWarnOnOptions() |
| { |
| if (!SchemaConstants.isLocalSystemKeyspace(keyspaceName) && !SchemaConstants.isReplicatedSystemKeyspace(keyspaceName)) |
| { |
| int nodeCount = StorageService.instance.getHostIdToEndpoint().size(); |
| // nodeCount==0 on many tests |
| int rf = getReplicationFactor(); |
| if (rf > nodeCount && nodeCount != 0) |
| { |
| String msg = "Your replication factor " + rf |
| + " for keyspace " |
| + keyspaceName |
| + " is higher than the number of nodes " |
| + nodeCount; |
| if (ClientWarn.instance.getWarnings() == null || !ClientWarn.instance.getWarnings().contains(msg)) |
| { |
| ClientWarn.instance.warn(msg); |
| logger.warn(msg); |
| } |
| } |
| } |
| } |
| |
| public Collection<String> recognizedOptions() |
| { |
| return Collections.<String>singleton("replication_factor"); |
| } |
| } |