80
80
import org .apache .cassandra .utils .progress .jmx .JMXProgressSupport ;
81
81
import org .apache .cassandra .utils .progress .jmx .LegacyJMXProgressSupport ;
82
82
83
+ import java .util .Scanner ;
84
+ import java .util .regex .MatchResult ;
85
+ import java .util .regex .Pattern ;
86
+
83
87
import static java .util .concurrent .TimeUnit .MINUTES ;
84
88
85
89
/**
@@ -1111,12 +1115,26 @@ public boolean isJoined()
1111
1115
public void rebuild (String sourceDc )
1112
1116
{
1113
1117
// check on going rebuild
1118
+ rebuild (sourceDc , null , null , null );
1119
+ }
1120
+
1121
+ public void rebuild (String sourceDc , String keyspace , String tokens , String specificSources )
1122
+ {
1123
+ // check ongoing rebuild
1114
1124
if (!isRebuilding .compareAndSet (false , true ))
1115
1125
{
1116
1126
throw new IllegalStateException ("Node is still rebuilding. Check nodetool netstats." );
1117
1127
}
1118
1128
1119
- logger .info ("rebuild from dc: {}" , sourceDc == null ? "(any dc)" : sourceDc );
1129
+ // check the arguments
1130
+ if (keyspace == null && tokens != null )
1131
+ {
1132
+ throw new IllegalArgumentException ("Cannot specify tokens without keyspace." );
1133
+ }
1134
+
1135
+ logger .info ("rebuild from dc: {}, {}, {}" , sourceDc == null ? "(any dc)" : sourceDc ,
1136
+ keyspace == null ? "(All keyspaces)" : keyspace ,
1137
+ tokens == null ? "(All tokens)" : tokens );
1120
1138
1121
1139
try
1122
1140
{
@@ -1131,8 +1149,78 @@ public void rebuild(String sourceDc)
1131
1149
if (sourceDc != null )
1132
1150
streamer .addSourceFilter (new RangeStreamer .SingleDatacenterFilter (DatabaseDescriptor .getEndpointSnitch (), sourceDc ));
1133
1151
1134
- for (String keyspaceName : Schema .instance .getNonSystemKeyspaces ())
1135
- streamer .addRanges (keyspaceName , getLocalRanges (keyspaceName ));
1152
+ if (keyspace == null )
1153
+ {
1154
+ for (String keyspaceName : Schema .instance .getNonSystemKeyspaces ())
1155
+ streamer .addRanges (keyspaceName , getLocalRanges (keyspaceName ));
1156
+ }
1157
+ else if (tokens == null )
1158
+ {
1159
+ streamer .addRanges (keyspace , getLocalRanges (keyspace ));
1160
+ }
1161
+ else
1162
+ {
1163
+ Token .TokenFactory factory = getPartitioner ().getTokenFactory ();
1164
+ List <Range <Token >> ranges = new ArrayList <>();
1165
+ Pattern rangePattern = Pattern .compile ("\\ (\\ s*(-?\\ w+)\\ s*,\\ s*(-?\\ w+)\\ s*\\ ]" );
1166
+ try (Scanner tokenScanner = new Scanner (tokens ))
1167
+ {
1168
+ while (tokenScanner .findInLine (rangePattern ) != null )
1169
+ {
1170
+ MatchResult range = tokenScanner .match ();
1171
+ Token startToken = factory .fromString (range .group (1 ));
1172
+ Token endToken = factory .fromString (range .group (2 ));
1173
+ logger .info ("adding range: ({},{}]" , startToken , endToken );
1174
+ ranges .add (new Range <>(startToken , endToken ));
1175
+ }
1176
+ if (tokenScanner .hasNext ())
1177
+ throw new IllegalArgumentException ("Unexpected string: " + tokenScanner .next ());
1178
+ }
1179
+
1180
+ // Ensure all specified ranges are actually ranges owned by this host
1181
+ Collection <Range <Token >> localRanges = getLocalRanges (keyspace );
1182
+ for (Range <Token > specifiedRange : ranges )
1183
+ {
1184
+ boolean foundParentRange = false ;
1185
+ for (Range <Token > localRange : localRanges )
1186
+ {
1187
+ if (localRange .contains (specifiedRange ))
1188
+ {
1189
+ foundParentRange = true ;
1190
+ break ;
1191
+ }
1192
+ }
1193
+ if (!foundParentRange )
1194
+ {
1195
+ throw new IllegalArgumentException (String .format ("The specified range %s is not a range that is owned by this node. Please ensure that all token ranges specified to be rebuilt belong to this node." , specifiedRange .toString ()));
1196
+ }
1197
+ }
1198
+
1199
+ if (specificSources != null )
1200
+ {
1201
+ String [] stringHosts = specificSources .split ("," );
1202
+ Set <InetAddress > sources = new HashSet <>(stringHosts .length );
1203
+ for (String stringHost : stringHosts )
1204
+ {
1205
+ try
1206
+ {
1207
+ InetAddress endpoint = InetAddress .getByName (stringHost );
1208
+ if (FBUtilities .getBroadcastAddress ().equals (endpoint ))
1209
+ {
1210
+ throw new IllegalArgumentException ("This host was specified as a source for rebuilding. Sources for a rebuild can only be other nodes in the cluster." );
1211
+ }
1212
+ sources .add (endpoint );
1213
+ }
1214
+ catch (UnknownHostException ex )
1215
+ {
1216
+ throw new IllegalArgumentException ("Unknown host specified " + stringHost , ex );
1217
+ }
1218
+ }
1219
+ streamer .addSourceFilter (new RangeStreamer .WhitelistedSourcesFilter (sources ));
1220
+ }
1221
+
1222
+ streamer .addRanges (keyspace , ranges );
1223
+ }
1136
1224
1137
1225
StreamResultFuture resultFuture = streamer .fetchAsync ();
1138
1226
// wait for result
0 commit comments