forked from voldemort/voldemort
/
RebalanceController.java
808 lines (708 loc) · 42.2 KB
/
RebalanceController.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
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
/*
* Copyright 2008-2013 LinkedIn, Inc
*
* Licensed 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 voldemort.client.rebalance;
import java.io.File;
import java.io.StringReader;
import java.text.DecimalFormat;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map.Entry;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Semaphore;
import org.apache.log4j.Logger;
import voldemort.VoldemortException;
import voldemort.client.ClientConfig;
import voldemort.client.protocol.RequestFormatType;
import voldemort.client.protocol.admin.AdminClient;
import voldemort.client.rebalance.task.DonorBasedRebalanceTask;
import voldemort.client.rebalance.task.RebalanceTask;
import voldemort.client.rebalance.task.StealerBasedRebalanceTask;
import voldemort.cluster.Cluster;
import voldemort.cluster.Node;
import voldemort.server.rebalance.VoldemortRebalancingException;
import voldemort.store.StoreDefinition;
import voldemort.utils.NodeUtils;
import voldemort.utils.RebalanceUtils;
import voldemort.utils.StoreDefinitionUtils;
import voldemort.utils.Time;
import voldemort.versioning.Versioned;
import voldemort.xml.ClusterMapper;
import com.google.common.collect.Lists;
import com.google.common.collect.TreeMultimap;
public class RebalanceController {
private static final Logger logger = Logger.getLogger(RebalanceController.class);
private static final DecimalFormat decimalFormatter = new DecimalFormat("#.##");
private final AdminClient adminClient;
private final RebalanceClientConfig rebalanceConfig;
public RebalanceController(String bootstrapUrl, RebalanceClientConfig rebalanceConfig) {
this.adminClient = new AdminClient(bootstrapUrl,
rebalanceConfig,
new ClientConfig().setRequestFormatType(RequestFormatType.PROTOCOL_BUFFERS));
this.rebalanceConfig = rebalanceConfig;
}
public RebalanceController(Cluster cluster, RebalanceClientConfig config) {
this.adminClient = new AdminClient(cluster, config, new ClientConfig());
this.rebalanceConfig = config;
}
/**
* Grabs the latest cluster definition
* {@link #rebalance(voldemort.cluster.Cluster, voldemort.cluster.Cluster)}
*
* @param targetCluster Target cluster metadata
*/
public void rebalance(final Cluster targetCluster) {
// Retrieve the latest cluster metadata from the existing nodes
Versioned<Cluster> currentVersionedCluster = RebalanceUtils.getLatestCluster(NodeUtils.getNodeIds(Lists.newArrayList(adminClient.getAdminClientCluster()
.getNodes())),
adminClient);
Cluster currentCluster = currentVersionedCluster.getValue();
// Start the rebalance with the current cluster + target cluster
rebalance(currentCluster, targetCluster);
}
/**
* Grabs the store definition and calls
* {@link #rebalance(voldemort.cluster.Cluster, voldemort.cluster.Cluster, java.util.List)}
*
* @param currentCluster Current cluster metadata
* @param targetCluster Target cluster metadata
*/
public void rebalance(Cluster currentCluster, final Cluster targetCluster) {
// Make admin client point to this updated current cluster
adminClient.setAdminClientCluster(targetCluster);
// Retrieve list of stores + check for that all are consistent
List<StoreDefinition> storeDefs = RebalanceUtils.getStoreDefinition(targetCluster,
adminClient);
rebalance(currentCluster, targetCluster, storeDefs);
}
/**
* Does basic verification of the metadata + server state. Finally starts
* the rebalancing.
*
* @param currentCluster The cluster currently used on each node
* @param targetCluster The desired cluster after rebalance
* @param storeDefs Stores to rebalance
*/
public void rebalance(Cluster currentCluster,
final Cluster targetCluster,
List<StoreDefinition> storeDefs) {
logger.info("Current cluster : " + currentCluster);
logger.info("Final target cluster : " + targetCluster);
logger.info("Show plan : " + rebalanceConfig.isShowPlanEnabled());
logger.info("Delete post rebalancing : "
+ rebalanceConfig.isDeleteAfterRebalancingEnabled());
logger.info("Stealer based rebalancing : " + rebalanceConfig.isStealerBasedRebalancing());
logger.info("Primary partition batch size : "
+ rebalanceConfig.getPrimaryPartitionBatchSize());
// Filter the store definitions to a set rebalancing can support
storeDefs = RebalanceUtils.validateRebalanceStore(storeDefs);
// Add all new nodes to a 'new current cluster'
Cluster newCurrentCluster = RebalanceUtils.getClusterWithNewNodes(currentCluster,
targetCluster);
// Make admin client point to this updated current cluster
adminClient.setAdminClientCluster(newCurrentCluster);
// Do some verification
if(!rebalanceConfig.isShowPlanEnabled()) {
// Now validate that all the nodes ( new + old ) are in normal state
RebalanceUtils.validateProdClusterStateIsNormal(newCurrentCluster, adminClient);
// Verify all old RO stores exist at version 2
RebalanceUtils.validateReadOnlyStores(newCurrentCluster, storeDefs, adminClient);
// Propagate the updated cluster metadata to everyone
logger.info("Propagating new cluster " + newCurrentCluster + " to all nodes");
RebalanceUtils.propagateCluster(adminClient, newCurrentCluster);
}
rebalancePerClusterTransition(newCurrentCluster, targetCluster, storeDefs);
}
/**
* Rebalance on a step-by-step transitions from cluster.xml to
* target-cluster.xml
*
* <br>
*
* Each transition represents the migration of one primary partition (
* {@link #rebalancePerPartitionTransition(int, OrderedClusterTransition)} )
* along with all its side effect ( i.e. migration of replicas + deletions
* ).
*
*
* @param currentCluster The normalized cluster. This cluster contains new
* nodes with empty partitions as well
* @param targetCluster The desired cluster after rebalance
* @param storeDefs Stores to rebalance
*/
private void rebalancePerClusterTransition(Cluster currentCluster,
final Cluster targetCluster,
final List<StoreDefinition> storeDefs) {
// Mapping of stealer node to list of primary partitions being moved
final TreeMultimap<Integer, Integer> stealerToStolenPrimaryPartitions = TreeMultimap.create();
// Creates the same mapping as above for dry run
final TreeMultimap<Integer, Integer> stealerToStolenPrimaryPartitionsClone = TreeMultimap.create();
// Various counts for progress bar
int numTasks = 0;
int numCrossZoneMoves = 0;
int numPrimaryPartitionMoves = 0;
// Used for creating clones
ClusterMapper mapper = new ClusterMapper();
// Output initial and final cluster
if(rebalanceConfig.hasOutputDirectory())
RebalanceUtils.dumpCluster(currentCluster,
targetCluster,
new File(rebalanceConfig.getOutputDirectory()));
// Start first dry run to compute the stolen partitions
for(Node stealerNode: targetCluster.getNodes()) {
List<Integer> stolenPrimaryPartitions = RebalanceUtils.getStolenPrimaryPartitions(currentCluster,
targetCluster,
stealerNode.getId());
if(stolenPrimaryPartitions.size() > 0) {
numPrimaryPartitionMoves += stolenPrimaryPartitions.size();
stealerToStolenPrimaryPartitions.putAll(stealerNode.getId(),
stolenPrimaryPartitions);
stealerToStolenPrimaryPartitionsClone.putAll(stealerNode.getId(),
stolenPrimaryPartitions);
}
}
// Create a clone for second dry run
Cluster currentClusterClone = mapper.readCluster(new StringReader(mapper.writeCluster(currentCluster)));
// Start second dry run to pre-compute other total statistics
while(!stealerToStolenPrimaryPartitionsClone.isEmpty()) {
// Generate a snapshot of current initial state
Cluster startCluster = mapper.readCluster(new StringReader(mapper.writeCluster(currentClusterClone)));
// Generate batches of partitions and move then over
int batchCompleted = 0;
List<Entry<Integer, Integer>> partitionsMoved = Lists.newArrayList();
for(Entry<Integer, Integer> stealerToPartition: stealerToStolenPrimaryPartitionsClone.entries()) {
partitionsMoved.add(stealerToPartition);
currentClusterClone = RebalanceUtils.createUpdatedCluster(currentClusterClone,
stealerToPartition.getKey(),
Lists.newArrayList(stealerToPartition.getValue()));
batchCompleted++;
if(batchCompleted == rebalanceConfig.getPrimaryPartitionBatchSize())
break;
}
// Remove the partitions moved
for(Iterator<Entry<Integer, Integer>> partitionMoved = partitionsMoved.iterator(); partitionMoved.hasNext();) {
Entry<Integer, Integer> entry = partitionMoved.next();
stealerToStolenPrimaryPartitionsClone.remove(entry.getKey(), entry.getValue());
}
// Generate a plan to compute the tasks
final RebalanceClusterPlan rebalanceClusterPlan = new RebalanceClusterPlan(startCluster,
currentClusterClone,
storeDefs,
rebalanceConfig.isDeleteAfterRebalancingEnabled(),
rebalanceConfig.isStealerBasedRebalancing());
numCrossZoneMoves += RebalanceUtils.getCrossZoneMoves(currentClusterClone,
rebalanceClusterPlan);
numTasks += RebalanceUtils.getTotalMoves(rebalanceClusterPlan);
}
logger.info("Total number of primary partition moves : " + numPrimaryPartitionMoves);
logger.info("Total number of cross zone moves : " + numCrossZoneMoves);
logger.info("Total number of tasks : " + numTasks);
int tasksCompleted = 0;
int batchCounter = 0;
int primaryPartitionId = 0;
double totalTimeMs = 0.0;
// Starting the real run
while(!stealerToStolenPrimaryPartitions.isEmpty()) {
Cluster transitionCluster = mapper.readCluster(new StringReader(mapper.writeCluster(currentCluster)));
// Generate batches of partitions and move then over
int primaryPartitionBatchSize = 0;
List<Entry<Integer, Integer>> partitionsMoved = Lists.newArrayList();
for(Entry<Integer, Integer> stealerToPartition: stealerToStolenPrimaryPartitions.entries()) {
partitionsMoved.add(stealerToPartition);
transitionCluster = RebalanceUtils.createUpdatedCluster(transitionCluster,
stealerToPartition.getKey(),
Lists.newArrayList(stealerToPartition.getValue()));
primaryPartitionBatchSize++;
if(primaryPartitionBatchSize == rebalanceConfig.getPrimaryPartitionBatchSize())
break;
}
batchCounter++;
// Remove the partitions moved + Prepare message to print
StringBuffer buffer = new StringBuffer();
buffer.append("Partitions being moved : ");
for(Iterator<Entry<Integer, Integer>> partitionMoved = partitionsMoved.iterator(); partitionMoved.hasNext();) {
Entry<Integer, Integer> entry = partitionMoved.next();
buffer.append("[ partition " + entry.getValue() + " to stealer node "
+ entry.getKey() + " ], ");
stealerToStolenPrimaryPartitions.remove(entry.getKey(), entry.getValue());
}
final RebalanceClusterPlan rebalanceClusterPlan = new RebalanceClusterPlan(currentCluster,
transitionCluster,
storeDefs,
rebalanceConfig.isDeleteAfterRebalancingEnabled(),
rebalanceConfig.isStealerBasedRebalancing());
final OrderedClusterTransition orderedClusterTransition = new OrderedClusterTransition(currentCluster,
transitionCluster,
storeDefs,
rebalanceClusterPlan);
// Print message about what is being moved
logger.info("----------------");
RebalanceUtils.printLog(orderedClusterTransition.getId(), logger, buffer.toString());
// Print the transition plan
RebalanceUtils.printLog(orderedClusterTransition.getId(),
logger,
orderedClusterTransition.toString());
// Output the transition plan to the output directory
if(rebalanceConfig.hasOutputDirectory())
RebalanceUtils.dumpCluster(currentCluster,
transitionCluster,
new File(rebalanceConfig.getOutputDirectory()),
"batch-" + Integer.toString(batchCounter) + ".");
long startTimeMs = System.currentTimeMillis();
rebalancePerPartitionTransition(orderedClusterTransition);
totalTimeMs += (System.currentTimeMillis() - startTimeMs);
// Update the current cluster
currentCluster = transitionCluster;
// Bump up the statistics
tasksCompleted += RebalanceUtils.getTotalMoves(rebalanceClusterPlan);
primaryPartitionId += primaryPartitionBatchSize;
// Calculate the estimated end time
double estimatedTimeMs = (totalTimeMs / tasksCompleted) * (numTasks - tasksCompleted);
RebalanceUtils.printLog(orderedClusterTransition.getId(),
logger,
"Completed tasks - "
+ tasksCompleted
+ ". Percent done - "
+ decimalFormatter.format(tasksCompleted * 100.0
/ numTasks));
RebalanceUtils.printLog(orderedClusterTransition.getId(),
logger,
"Primary partitions left to move - "
+ (numPrimaryPartitionMoves - primaryPartitionId));
RebalanceUtils.printLog(orderedClusterTransition.getId(),
logger,
"Estimated time left for completion - " + estimatedTimeMs
+ " ms ( " + estimatedTimeMs / Time.MS_PER_HOUR
+ " hours )");
}
}
/**
* Rebalance per partition transition - This does the actual rebalancing
* work required for a single primary partition move.
*
* <br>
*
* Each operation is split into individual tasks depending on read-only or
* read-write migration. Read-only store migration is done first to avoid
* the overhead of redirecting stores
*
* @param orderedClusterTransition The ordered cluster transition we are
* going to run
*/
private void rebalancePerPartitionTransition(final OrderedClusterTransition orderedClusterTransition) {
try {
final List<RebalancePartitionsInfo> rebalancePartitionsInfoList = orderedClusterTransition.getOrderedRebalancePartitionsInfoList();
if(rebalancePartitionsInfoList.isEmpty()) {
RebalanceUtils.printLog(orderedClusterTransition.getId(),
logger,
"Skipping rebalance task id "
+ orderedClusterTransition.getId()
+ " since it is empty.");
// Even though there is no rebalancing work to do, cluster
// metadata must be updated so that the server is aware of the
// new cluster xml.
adminClient.rebalanceOps.rebalanceStateChange(orderedClusterTransition.getCurrentCluster(),
orderedClusterTransition.getTargetCluster(),
rebalancePartitionsInfoList,
false,
true,
false,
false,
true);
return;
}
RebalanceUtils.printLog(orderedClusterTransition.getId(),
logger,
"Starting rebalance task id "
+ orderedClusterTransition.getId());
// Flatten the node plans to partition plans
List<RebalancePartitionsInfo> rebalancePartitionPlanList = rebalancePartitionsInfoList;
List<StoreDefinition> allStoreDefs = orderedClusterTransition.getStoreDefs();
// Split the store definitions
List<StoreDefinition> readOnlyStoreDefs = StoreDefinitionUtils.filterStores(orderedClusterTransition.getStoreDefs(),
true);
List<StoreDefinition> readWriteStoreDefs = StoreDefinitionUtils.filterStores(orderedClusterTransition.getStoreDefs(),
false);
boolean hasReadOnlyStores = readOnlyStoreDefs != null && readOnlyStoreDefs.size() > 0;
boolean hasReadWriteStores = readWriteStoreDefs != null
&& readWriteStoreDefs.size() > 0;
// STEP 1 - Cluster state change
boolean finishedReadOnlyPhase = false;
List<RebalancePartitionsInfo> filteredRebalancePartitionPlanList = RebalanceUtils.filterPartitionPlanWithStores(rebalancePartitionPlanList,
readOnlyStoreDefs);
// TODO this method right nowtakes just the source stores definition
// the 2nd argument needs to be fixed
// ATTENTION JAY
rebalanceStateChange(orderedClusterTransition.getId(),
orderedClusterTransition.getCurrentCluster(),
orderedClusterTransition.getTargetCluster(),
allStoreDefs,
allStoreDefs,
filteredRebalancePartitionPlanList,
hasReadOnlyStores,
hasReadWriteStores,
finishedReadOnlyPhase);
// STEP 2 - Move RO data
if(hasReadOnlyStores) {
rebalancePerTaskTransition(orderedClusterTransition.getId(),
orderedClusterTransition.getCurrentCluster(),
filteredRebalancePartitionPlanList,
hasReadOnlyStores,
hasReadWriteStores,
finishedReadOnlyPhase);
}
// STEP 3 - Cluster change state
finishedReadOnlyPhase = true;
filteredRebalancePartitionPlanList = RebalanceUtils.filterPartitionPlanWithStores(rebalancePartitionPlanList,
readWriteStoreDefs);
// TODO this method right nowtakes just the source stores definition
// the 2nd argument needs to be fixed
// ATTENTION JAY
rebalanceStateChange(orderedClusterTransition.getId(),
orderedClusterTransition.getCurrentCluster(),
orderedClusterTransition.getTargetCluster(),
allStoreDefs,
allStoreDefs,
filteredRebalancePartitionPlanList,
hasReadOnlyStores,
hasReadWriteStores,
finishedReadOnlyPhase);
// STEP 4 - Move RW data
if(hasReadWriteStores) {
rebalancePerTaskTransition(orderedClusterTransition.getId(),
orderedClusterTransition.getCurrentCluster(),
filteredRebalancePartitionPlanList,
hasReadOnlyStores,
hasReadWriteStores,
finishedReadOnlyPhase);
}
RebalanceUtils.printLog(orderedClusterTransition.getId(),
logger,
"Successfully terminated rebalance task id "
+ orderedClusterTransition.getId());
} catch(Exception e) {
RebalanceUtils.printErrorLog(orderedClusterTransition.getId(),
logger,
"Error in rebalance task id "
+ orderedClusterTransition.getId() + " - "
+ e.getMessage(),
e);
throw new VoldemortException("Rebalance failed on rebalance task id "
+ orderedClusterTransition.getId(), e);
}
}
/**
* TODO JAY -- This interface expects the source stores definition and
* target stores def
*
* Perform a group of state change actions. Also any errors + rollback
* procedures are performed at this level itself.
*
* <pre>
* | Case | hasRO | hasRW | finishedRO | Action |
* | 0 | t | t | t | 2nd one ( cluster change + swap + rebalance state change ) |
* | 1 | t | t | f | 1st one ( rebalance state change ) |
* | 2 | t | f | t | 2nd one ( cluster change + swap ) |
* | 3 | t | f | f | 1st one ( rebalance state change ) |
* | 4 | f | t | t | 2nd one ( cluster change + rebalance state change ) |
* | 5 | f | t | f | ignore |
* | 6 | f | f | t | no stores, exception |
* | 7 | f | f | f | no stores, exception |
* </pre>
*
* Truth table, FTW!
*
* @param taskId Rebalancing task id
* @param currentCluster Current cluster
* @param transitionCluster Transition cluster to propagate
* @param rebalancePartitionPlanList List of partition plan list
* @param hasReadOnlyStores Boolean indicating if read-only stores exist
* @param hasReadWriteStores Boolean indicating if read-write stores exist
* @param finishedReadOnlyStores Boolean indicating if we have finished RO
* store migration
*/
private void rebalanceStateChange(final int taskId,
Cluster currentCluster,
Cluster transitionCluster,
List<StoreDefinition> existingStoreDefs,
List<StoreDefinition> targetStoreDefs,
List<RebalancePartitionsInfo> rebalancePartitionPlanList,
boolean hasReadOnlyStores,
boolean hasReadWriteStores,
boolean finishedReadOnlyStores) {
try {
if(!hasReadOnlyStores && !hasReadWriteStores) {
// Case 6 / 7 - no stores, exception
throw new VoldemortException("Cannot get this state since it means there are no stores");
} else if(!hasReadOnlyStores && hasReadWriteStores && !finishedReadOnlyStores) {
// Case 5 - ignore
RebalanceUtils.printLog(taskId,
logger,
"Ignoring state change since there are no read-only stores");
} else if(!hasReadOnlyStores && hasReadWriteStores && finishedReadOnlyStores) {
// Case 4 - cluster change + rebalance state change
RebalanceUtils.printLog(taskId,
logger,
"Cluster metadata change + rebalance state change");
if(!rebalanceConfig.isShowPlanEnabled())
adminClient.rebalanceOps.rebalanceStateChange(currentCluster,
transitionCluster,
existingStoreDefs,
targetStoreDefs,
rebalancePartitionPlanList,
false,
true,
true,
true,
true);
} else if(hasReadOnlyStores && !finishedReadOnlyStores) {
// Case 1 / 3 - rebalance state change
RebalanceUtils.printLog(taskId, logger, "Rebalance state change");
if(!rebalanceConfig.isShowPlanEnabled())
adminClient.rebalanceOps.rebalanceStateChange(currentCluster,
transitionCluster,
existingStoreDefs,
targetStoreDefs,
rebalancePartitionPlanList,
false,
false,
true,
true,
true);
} else if(hasReadOnlyStores && !hasReadWriteStores && finishedReadOnlyStores) {
// Case 2 - swap + cluster change
RebalanceUtils.printLog(taskId, logger, "Swap + Cluster metadata change");
if(!rebalanceConfig.isShowPlanEnabled())
adminClient.rebalanceOps.rebalanceStateChange(currentCluster,
transitionCluster,
existingStoreDefs,
targetStoreDefs,
rebalancePartitionPlanList,
true,
true,
false,
true,
true);
} else {
// Case 0 - swap + cluster change + rebalance state change
RebalanceUtils.printLog(taskId,
logger,
"Swap + Cluster metadata change + rebalance state change");
if(!rebalanceConfig.isShowPlanEnabled())
adminClient.rebalanceOps.rebalanceStateChange(currentCluster,
transitionCluster,
existingStoreDefs,
targetStoreDefs,
rebalancePartitionPlanList,
true,
true,
true,
true,
true);
}
} catch(VoldemortRebalancingException e) {
RebalanceUtils.printErrorLog(taskId,
logger,
"Failure while changing rebalancing state",
e);
throw e;
}
}
/**
* The smallest granularity of rebalancing where-in we move partitions for a
* sub-set of stores. Finally at the end of the movement, the node is
* removed out of rebalance state
*
* <br>
*
* Also any errors + rollback procedures are performed at this level itself.
*
* <pre>
* | Case | hasRO | hasRW | finishedRO | Action |
* | 0 | t | t | t | rollback cluster change + swap |
* | 1 | t | t | f | nothing to do since "rebalance state change" should have removed everything |
* | 2 | t | f | t | won't be triggered since hasRW is false |
* | 3 | t | f | f | nothing to do since "rebalance state change" should have removed everything |
* | 4 | f | t | t | rollback cluster change |
* | 5 | f | t | f | won't be triggered |
* | 6 | f | f | t | won't be triggered |
* | 7 | f | f | f | won't be triggered |
* </pre>
*
* @param taskId Rebalance task id
* @param currentCluster Cluster to rollback to if we have a problem
* @param rebalancePartitionPlanList The list of rebalance partition plans
* @param hasReadOnlyStores Are we rebalancing any read-only stores?
* @param hasReadWriteStores Are we rebalancing any read-write stores?
* @param finishedReadOnlyStores Have we finished rebalancing of read-only
* stores?
*/
private void rebalancePerTaskTransition(final int taskId,
final Cluster currentCluster,
final List<RebalancePartitionsInfo> rebalancePartitionPlanList,
boolean hasReadOnlyStores,
boolean hasReadWriteStores,
boolean finishedReadOnlyStores) {
RebalanceUtils.printLog(taskId, logger, "Submitting rebalance tasks ");
// If only show plan, done!
if(rebalanceConfig.isShowPlanEnabled()) {
return;
}
// Get an ExecutorService in place used for submitting our tasks
ExecutorService service = RebalanceUtils.createExecutors(rebalanceConfig.getMaxParallelRebalancing());
// Sub-list of the above list
final List<RebalanceTask> failedTasks = Lists.newArrayList();
final List<RebalanceTask> incompleteTasks = Lists.newArrayList();
// Semaphores for donor nodes - To avoid multiple disk sweeps
Semaphore[] donorPermits = new Semaphore[currentCluster.getNumberOfNodes()];
for(Node node: currentCluster.getNodes()) {
donorPermits[node.getId()] = new Semaphore(1);
}
try {
// List of tasks which will run asynchronously
List<RebalanceTask> allTasks = executeTasks(taskId,
service,
rebalancePartitionPlanList,
donorPermits);
// All tasks submitted.
RebalanceUtils.printLog(taskId,
logger,
"All rebalance tasks were submitted ( shutting down in "
+ rebalanceConfig.getRebalancingClientTimeoutSeconds()
+ " sec )");
// Wait and shutdown after timeout
RebalanceUtils.executorShutDown(service,
rebalanceConfig.getRebalancingClientTimeoutSeconds());
RebalanceUtils.printLog(taskId, logger, "Finished waiting for executors");
// Collects all failures + incomplete tasks from the rebalance
// tasks.
List<Exception> failures = Lists.newArrayList();
for(RebalanceTask task: allTasks) {
if(task.hasException()) {
failedTasks.add(task);
failures.add(task.getError());
} else if(!task.isComplete()) {
incompleteTasks.add(task);
}
}
if(failedTasks.size() > 0) {
throw new VoldemortRebalancingException("Rebalance task terminated unsuccessfully on tasks "
+ failedTasks,
failures);
}
// If there were no failures, then we could have had a genuine
// timeout ( Rebalancing took longer than the operator expected ).
// We should throw a VoldemortException and not a
// VoldemortRebalancingException ( which will start reverting
// metadata ). The operator may want to manually then resume the
// process.
if(incompleteTasks.size() > 0) {
throw new VoldemortException("Rebalance tasks are still incomplete / running "
+ incompleteTasks);
}
} catch(VoldemortRebalancingException e) {
logger.error("Failure while migrating partitions for rebalance task " + taskId);
if(hasReadOnlyStores && hasReadWriteStores && finishedReadOnlyStores) {
// Case 0
// TODO this method right nowtakes just the source stores
// definition
// the 2nd argument needs to be fixed
// ATTENTION JAY
adminClient.rebalanceOps.rebalanceStateChange(null, currentCluster, null, null, // pass
// current
// store
// def
null,
true,
true,
false,
false,
false);
} else if(hasReadWriteStores && finishedReadOnlyStores) {
// Case 4
// TODO this method right nowtakes just the source stores
// definition
// the 2nd argument needs to be fixed
// ATTENTION JAY
adminClient.rebalanceOps.rebalanceStateChange(null, currentCluster, null, null, // pass
// current
// store
// def
null,
false,
true,
false,
false,
false);
}
throw e;
} finally {
if(!service.isShutdown()) {
RebalanceUtils.printErrorLog(taskId,
logger,
"Could not shutdown service cleanly for rebalance task "
+ taskId,
null);
service.shutdownNow();
}
}
}
private List<RebalanceTask> executeTasks(final int taskId,
final ExecutorService service,
List<RebalancePartitionsInfo> rebalancePartitionPlanList,
Semaphore[] donorPermits) {
List<RebalanceTask> taskList = Lists.newArrayList();
if(rebalanceConfig.isStealerBasedRebalancing()) {
for(RebalancePartitionsInfo partitionsInfo: rebalancePartitionPlanList) {
StealerBasedRebalanceTask rebalanceTask = new StealerBasedRebalanceTask(taskId,
partitionsInfo,
rebalanceConfig,
donorPermits[partitionsInfo.getDonorId()],
adminClient);
taskList.add(rebalanceTask);
service.execute(rebalanceTask);
}
} else {
// Group by donor nodes
HashMap<Integer, List<RebalancePartitionsInfo>> donorNodeBasedPartitionsInfo = RebalanceUtils.groupPartitionsInfoByNode(rebalancePartitionPlanList,
false);
for(Entry<Integer, List<RebalancePartitionsInfo>> entries: donorNodeBasedPartitionsInfo.entrySet()) {
// TODO: Can this sleep be removed?
/*-
try {
Thread.sleep(10000);
} catch(InterruptedException e) {}
*/
DonorBasedRebalanceTask rebalanceTask = new DonorBasedRebalanceTask(taskId,
entries.getValue(),
rebalanceConfig,
donorPermits[entries.getValue()
.get(0)
.getDonorId()],
adminClient);
taskList.add(rebalanceTask);
service.execute(rebalanceTask);
}
}
return taskList;
}
public AdminClient getAdminClient() {
return adminClient;
}
public void stop() {
adminClient.close();
}
}