/
SimpleStrategy.java
175 lines (152 loc) · 7.29 KB
/
SimpleStrategy.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
/*
* 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.util.Collection;
import java.util.Collections;
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.DatabaseDescriptor;
import org.apache.cassandra.db.guardrails.Guardrails;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.schema.SchemaConstants;
import org.apache.cassandra.service.ClientState;
import org.apache.cassandra.service.ClientWarn;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.tcm.ClusterMetadata;
import org.apache.cassandra.tcm.Epoch;
import org.apache.cassandra.tcm.compatibility.TokenRingUtils;
import org.apache.cassandra.tcm.membership.Directory;
import org.apache.cassandra.tcm.membership.NodeId;
import org.apache.cassandra.tcm.ownership.DataPlacement;
import org.apache.cassandra.tcm.ownership.ReplicaGroups;
import org.apache.cassandra.tcm.ownership.TokenMap;
import org.apache.cassandra.tcm.ownership.VersionedEndpoints;
/**
* 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
{
public static final String REPLICATION_FACTOR = "replication_factor";
private static final Logger logger = LoggerFactory.getLogger(SimpleStrategy.class);
private final ReplicationFactor rf;
public SimpleStrategy(String keyspaceName, Map<String, String> configOptions)
{
super(keyspaceName, configOptions);
validateOptionsInternal(configOptions);
this.rf = ReplicationFactor.fromString(this.configOptions.get(REPLICATION_FACTOR));
}
@Override
public DataPlacement calculateDataPlacement(Epoch epoch, List<Range<Token>> ranges, ClusterMetadata metadata)
{
ReplicaGroups.Builder builder = ReplicaGroups.builder();
for (Range<Token> range : ranges)
builder.withReplicaGroup(VersionedEndpoints.forRange(epoch,
calculateNaturalReplicas(range.right, metadata.tokenMap.tokens(), range, metadata.directory, metadata.tokenMap)));
ReplicaGroups built = builder.build();
return new DataPlacement(built, built);
}
@Override
public EndpointsForRange calculateNaturalReplicas(Token token, ClusterMetadata metadata)
{
List<Token> ring = metadata.tokenMap.tokens();
if (ring.isEmpty())
return EndpointsForRange.empty(new Range<>(metadata.tokenMap.partitioner().getMinimumToken(), metadata.tokenMap.partitioner().getMinimumToken()));
Range<Token> replicaRange = TokenRingUtils.getRange(ring, token);
return calculateNaturalReplicas(token, ring, replicaRange, metadata.directory, metadata.tokenMap);
}
private EndpointsForRange calculateNaturalReplicas(Token token,
List<Token> ring,
Range<Token> replicaRange,
Directory endpoints,
TokenMap tokens)
{
if (ring.isEmpty())
return EndpointsForRange.empty(new Range<>(tokens.partitioner().getMinimumToken(), token.getPartitioner().getMinimumToken()));
Iterator<Token> iter = TokenRingUtils.ringIterator(ring, token, false);
EndpointsForRange.Builder replicas = new EndpointsForRange.Builder(replicaRange, rf.allReplicas);
// Add the token at the index by default
while (replicas.size() < rf.allReplicas && iter.hasNext())
{
Token tk = iter.next();
NodeId owner = tokens.owner(tk);
InetAddressAndPort ep = endpoints.endpoint(owner);
if (!replicas.endpoints().contains(ep))
replicas.add(new Replica(ep, replicaRange, replicas.size() < rf.fullReplicas));
}
return replicas.build();
}
@Override
public ReplicationFactor getReplicationFactor()
{
return rf;
}
private static void validateOptionsInternal(Map<String, String> configOptions) throws ConfigurationException
{
if (configOptions.get(REPLICATION_FACTOR) == null)
throw new ConfigurationException("SimpleStrategy requires a replication_factor strategy option.");
}
@Override
public void validateOptions() throws ConfigurationException
{
validateOptionsInternal(configOptions);
validateReplicationFactor(configOptions.get(REPLICATION_FACTOR));
}
@Override
public void maybeWarnOnOptions(ClientState state)
{
if (!SchemaConstants.isSystemKeyspace(keyspaceName))
{
int nodeCount = StorageService.instance.getHostIdToEndpoint().size();
// nodeCount==0 on many tests
Guardrails.minimumReplicationFactor.guard(rf.fullReplicas, keyspaceName, false, state);
Guardrails.maximumReplicationFactor.guard(rf.fullReplicas, keyspaceName, false, state);
if (rf.fullReplicas > nodeCount && nodeCount != 0)
{
String msg = "Your replication factor " + rf.fullReplicas
+ " for keyspace "
+ keyspaceName
+ " is higher than the number of nodes "
+ nodeCount;
ClientWarn.instance.warn(msg);
logger.warn(msg);
}
}
}
@Override
public Collection<String> recognizedOptions(ClusterMetadata metadata)
{
return Collections.singleton(REPLICATION_FACTOR);
}
@SuppressWarnings("unused") // used via reflection
protected static void prepareOptions(Map<String, String> options, Map<String, String> previousOptions)
{
// When altering from NTS to SS, previousOptions could have multiple different RFs for different data centers - so we
// will instead default to DefaultRF configuration if RF is not mentioned with the alter statement
String rf = previousOptions.containsKey(REPLICATION_FACTOR) ? previousOptions.get(REPLICATION_FACTOR)
: Integer.toString(DatabaseDescriptor.getDefaultKeyspaceRF());
options.putIfAbsent(REPLICATION_FACTOR, rf);
}
}