forked from voldemort/voldemort
/
AdminServiceRequestHandler.java
1570 lines (1364 loc) · 86 KB
/
AdminServiceRequestHandler.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
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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.server.protocol.admin;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.StringReader;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.log4j.Logger;
import voldemort.VoldemortException;
import voldemort.client.protocol.VoldemortFilter;
import voldemort.client.protocol.admin.AdminClient;
import voldemort.client.protocol.admin.filter.DefaultVoldemortFilter;
import voldemort.client.protocol.pb.ProtoUtils;
import voldemort.client.protocol.pb.VAdminProto;
import voldemort.client.protocol.pb.VAdminProto.RebalancePartitionInfoMap;
import voldemort.client.protocol.pb.VAdminProto.VoldemortAdminRequest;
import voldemort.client.rebalance.RebalancePartitionsInfo;
import voldemort.cluster.Cluster;
import voldemort.common.nio.ByteBufferBackedInputStream;
import voldemort.routing.StoreRoutingPlan;
import voldemort.server.StoreRepository;
import voldemort.server.VoldemortConfig;
import voldemort.server.protocol.RequestHandler;
import voldemort.server.protocol.StreamRequestHandler;
import voldemort.server.rebalance.Rebalancer;
import voldemort.server.storage.RepairJob;
import voldemort.server.storage.StorageService;
import voldemort.store.ErrorCodeMapper;
import voldemort.store.StorageEngine;
import voldemort.store.StoreDefinition;
import voldemort.store.StoreDefinitionBuilder;
import voldemort.store.StoreOperationFailureException;
import voldemort.store.backup.NativeBackupable;
import voldemort.store.metadata.MetadataStore;
import voldemort.store.mysql.MysqlStorageEngine;
import voldemort.store.readonly.FileFetcher;
import voldemort.store.readonly.ReadOnlyStorageConfiguration;
import voldemort.store.readonly.ReadOnlyStorageEngine;
import voldemort.store.readonly.ReadOnlyUtils;
import voldemort.store.slop.SlopStorageEngine;
import voldemort.utils.ByteArray;
import voldemort.utils.ByteUtils;
import voldemort.utils.ClosableIterator;
import voldemort.utils.EventThrottler;
import voldemort.utils.NetworkClassLoader;
import voldemort.utils.Pair;
import voldemort.utils.RebalanceUtils;
import voldemort.utils.ReflectUtils;
import voldemort.utils.Utils;
import voldemort.versioning.ObsoleteVersionException;
import voldemort.versioning.VectorClock;
import voldemort.versioning.Versioned;
import voldemort.xml.ClusterMapper;
import voldemort.xml.StoreDefinitionsMapper;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
/**
* Protocol buffers implementation of a {@link RequestHandler}
*
*/
public class AdminServiceRequestHandler implements RequestHandler {
private final static Logger logger = Logger.getLogger(AdminServiceRequestHandler.class);
private final static Object lock = new Object();
private final ErrorCodeMapper errorCodeMapper;
private final MetadataStore metadataStore;
private final StorageService storageService;
private final StoreRepository storeRepository;
private final NetworkClassLoader networkClassLoader;
private final VoldemortConfig voldemortConfig;
private final AsyncOperationService asyncService;
private final Rebalancer rebalancer;
private FileFetcher fileFetcher;
public AdminServiceRequestHandler(ErrorCodeMapper errorCodeMapper,
StorageService storageService,
StoreRepository storeRepository,
MetadataStore metadataStore,
VoldemortConfig voldemortConfig,
AsyncOperationService asyncService,
Rebalancer rebalancer) {
this.errorCodeMapper = errorCodeMapper;
this.storageService = storageService;
this.metadataStore = metadataStore;
this.storeRepository = storeRepository;
this.voldemortConfig = voldemortConfig;
this.networkClassLoader = new NetworkClassLoader(Thread.currentThread()
.getContextClassLoader());
this.asyncService = asyncService;
this.rebalancer = rebalancer;
setFetcherClass(voldemortConfig);
}
private void setFetcherClass(VoldemortConfig voldemortConfig) {
if(voldemortConfig != null) {
String className = voldemortConfig.getAllProps().getString("file.fetcher.class", null);
if(className == null || className.trim().length() == 0) {
this.fileFetcher = null;
} else {
try {
logger.info("Loading fetcher " + className);
Class<?> cls = Class.forName(className.trim());
this.fileFetcher = (FileFetcher) ReflectUtils.callConstructor(cls,
new Class<?>[] {
VoldemortConfig.class,
storageService.getDynThrottleLimit()
.getClass() },
new Object[] {
voldemortConfig,
storageService.getDynThrottleLimit() });
} catch(Exception e) {
throw new VoldemortException("Error loading file fetcher class " + className, e);
}
}
} else {
this.fileFetcher = null;
}
}
public StreamRequestHandler handleRequest(final DataInputStream inputStream,
final DataOutputStream outputStream)
throws IOException {
// Another protocol buffers bug here, temp. work around
VoldemortAdminRequest.Builder request = VoldemortAdminRequest.newBuilder();
int size = inputStream.readInt();
if(logger.isTraceEnabled())
logger.trace("In handleRequest, request specified size of " + size + " bytes");
if(size < 0)
throw new IOException("In handleRequest, request specified size of " + size + " bytes");
byte[] input = new byte[size];
ByteUtils.read(inputStream, input);
request.mergeFrom(input);
switch(request.getType()) {
case GET_METADATA:
ProtoUtils.writeMessage(outputStream, handleGetMetadata(request.getGetMetadata()));
break;
case UPDATE_METADATA:
ProtoUtils.writeMessage(outputStream,
handleUpdateMetadata(request.getUpdateMetadata()));
break;
case DELETE_PARTITION_ENTRIES:
ProtoUtils.writeMessage(outputStream,
handleDeletePartitionEntries(request.getDeletePartitionEntries()));
break;
case FETCH_PARTITION_ENTRIES:
return handleFetchPartitionEntries(request.getFetchPartitionEntries());
case UPDATE_PARTITION_ENTRIES:
return handleUpdatePartitionEntries(request.getUpdatePartitionEntries());
case INITIATE_FETCH_AND_UPDATE:
ProtoUtils.writeMessage(outputStream,
handleFetchAndUpdate(request.getInitiateFetchAndUpdate()));
break;
case ASYNC_OPERATION_STATUS:
ProtoUtils.writeMessage(outputStream,
handleAsyncStatus(request.getAsyncOperationStatus()));
break;
case INITIATE_REBALANCE_NODE:
ProtoUtils.writeMessage(outputStream,
handleRebalanceNode(request.getInitiateRebalanceNode()));
break;
case INITIATE_REBALANCE_NODE_ON_DONOR:
ProtoUtils.writeMessage(outputStream,
handleRebalanceNodeOnDonor(request.getInitiateRebalanceNodeOnDonor()));
break;
case ASYNC_OPERATION_LIST:
ProtoUtils.writeMessage(outputStream,
handleAsyncOperationList(request.getAsyncOperationList()));
break;
case ASYNC_OPERATION_STOP:
ProtoUtils.writeMessage(outputStream,
handleAsyncOperationStop(request.getAsyncOperationStop()));
break;
case TRUNCATE_ENTRIES:
ProtoUtils.writeMessage(outputStream,
handleTruncateEntries(request.getTruncateEntries()));
break;
case ADD_STORE:
ProtoUtils.writeMessage(outputStream, handleAddStore(request.getAddStore()));
break;
case DELETE_STORE:
ProtoUtils.writeMessage(outputStream, handleDeleteStore(request.getDeleteStore()));
break;
case FETCH_STORE:
ProtoUtils.writeMessage(outputStream, handleFetchROStore(request.getFetchStore()));
break;
case SWAP_STORE:
ProtoUtils.writeMessage(outputStream, handleSwapROStore(request.getSwapStore()));
break;
case ROLLBACK_STORE:
ProtoUtils.writeMessage(outputStream,
handleRollbackStore(request.getRollbackStore()));
break;
case GET_RO_MAX_VERSION_DIR:
ProtoUtils.writeMessage(outputStream,
handleGetROMaxVersionDir(request.getGetRoMaxVersionDir()));
break;
case GET_RO_CURRENT_VERSION_DIR:
ProtoUtils.writeMessage(outputStream,
handleGetROCurrentVersionDir(request.getGetRoCurrentVersionDir()));
break;
case GET_RO_STORAGE_FORMAT:
ProtoUtils.writeMessage(outputStream,
handleGetROStorageFormat(request.getGetRoStorageFormat()));
break;
case FETCH_PARTITION_FILES:
return handleFetchROPartitionFiles(request.getFetchPartitionFiles());
case UPDATE_SLOP_ENTRIES:
return handleUpdateSlopEntries(request.getUpdateSlopEntries());
case FAILED_FETCH_STORE:
ProtoUtils.writeMessage(outputStream,
handleFailedROFetch(request.getFailedFetchStore()));
break;
case REBALANCE_STATE_CHANGE:
ProtoUtils.writeMessage(outputStream,
handleRebalanceStateChange(request.getRebalanceStateChange()));
break;
case DELETE_STORE_REBALANCE_STATE:
ProtoUtils.writeMessage(outputStream,
handleDeleteStoreRebalanceState(request.getDeleteStoreRebalanceState()));
break;
case REPAIR_JOB:
ProtoUtils.writeMessage(outputStream, handleRepairJob(request.getRepairJob()));
break;
case NATIVE_BACKUP:
ProtoUtils.writeMessage(outputStream, handleNativeBackup(request.getNativeBackup()));
break;
case RESERVE_MEMORY:
ProtoUtils.writeMessage(outputStream,
handleReserveMemory(request.getReserveMemory()));
break;
default:
throw new VoldemortException("Unkown operation " + request.getType());
}
return null;
}
private VAdminProto.DeleteStoreRebalanceStateResponse handleDeleteStoreRebalanceState(VAdminProto.DeleteStoreRebalanceStateRequest request) {
VAdminProto.DeleteStoreRebalanceStateResponse.Builder response = VAdminProto.DeleteStoreRebalanceStateResponse.newBuilder();
synchronized(rebalancer) {
try {
int nodeId = request.getNodeId();
String storeName = request.getStoreName();
logger.info("Removing rebalancing state for donor node " + nodeId + " and store "
+ storeName + " from stealer node " + metadataStore.getNodeId());
RebalancePartitionsInfo info = metadataStore.getRebalancerState().find(nodeId);
if(info == null) {
throw new VoldemortException("Could not find state for donor node " + nodeId);
}
HashMap<Integer, List<Integer>> replicaToPartition = info.getReplicaToAddPartitionList(storeName);
if(replicaToPartition == null) {
throw new VoldemortException("Could not find state for donor node " + nodeId
+ " and store " + storeName);
}
info.removeStore(storeName);
logger.info("Removed rebalancing state for donor node " + nodeId + " and store "
+ storeName + " from stealer node " + metadataStore.getNodeId());
if(info.getUnbalancedStoreList().isEmpty()) {
metadataStore.deleteRebalancingState(info);
logger.info("Removed entire rebalancing state for donor node " + nodeId
+ " from stealer node " + metadataStore.getNodeId());
}
} catch(VoldemortException e) {
response.setError(ProtoUtils.encodeError(errorCodeMapper, e));
logger.error("handleDeleteStoreRebalanceState failed for request("
+ request.toString() + ")",
e);
}
}
return response.build();
}
public VAdminProto.RebalanceStateChangeResponse handleRebalanceStateChange(VAdminProto.RebalanceStateChangeRequest request) {
VAdminProto.RebalanceStateChangeResponse.Builder response = VAdminProto.RebalanceStateChangeResponse.newBuilder();
synchronized(rebalancer) {
try {
// Retrieve all values first
List<RebalancePartitionsInfo> rebalancePartitionsInfo = Lists.newArrayList();
for(RebalancePartitionInfoMap map: request.getRebalancePartitionInfoListList()) {
rebalancePartitionsInfo.add(ProtoUtils.decodeRebalancePartitionInfoMap(map));
}
Cluster cluster = new ClusterMapper().readCluster(new StringReader(request.getClusterString()));
boolean swapRO = request.getSwapRo();
boolean changeClusterMetadata = request.getChangeClusterMetadata();
boolean changeRebalanceState = request.getChangeRebalanceState();
boolean rollback = request.getRollback();
rebalancer.rebalanceStateChange(cluster,
rebalancePartitionsInfo,
swapRO,
changeClusterMetadata,
changeRebalanceState,
rollback);
} catch(VoldemortException e) {
response.setError(ProtoUtils.encodeError(errorCodeMapper, e));
logger.error("handleRebalanceStateChange failed for request(" + request.toString()
+ ")", e);
}
}
return response.build();
}
public VAdminProto.AsyncOperationStatusResponse handleRebalanceNodeOnDonor(VAdminProto.InitiateRebalanceNodeOnDonorRequest request) {
VAdminProto.AsyncOperationStatusResponse.Builder response = VAdminProto.AsyncOperationStatusResponse.newBuilder();
try {
if(!voldemortConfig.isEnableRebalanceService())
throw new VoldemortException("Rebalance service is not enabled for node: "
+ metadataStore.getNodeId());
List<RebalancePartitionsInfo> rebalanceStealInfos = ProtoUtils.decodeRebalancePartitionInfoMap(request.getRebalancePartitionInfoList());
// Assert that all the plans we got have the same donor node
RebalanceUtils.assertSameDonor(rebalanceStealInfos, metadataStore.getNodeId());
int requestId = rebalancer.rebalanceNodeOnDonor(rebalanceStealInfos);
response.setRequestId(requestId)
.setDescription(rebalanceStealInfos.toString())
.setStatus("Started rebalancing on donor")
.setComplete(false);
} catch(VoldemortException e) {
response.setError(ProtoUtils.encodeError(errorCodeMapper, e));
logger.error("handleRebalanceNodeOnDonor failed for request(" + request.toString()
+ ")", e);
}
return response.build();
}
public VAdminProto.AsyncOperationStatusResponse handleRebalanceNode(VAdminProto.InitiateRebalanceNodeRequest request) {
VAdminProto.AsyncOperationStatusResponse.Builder response = VAdminProto.AsyncOperationStatusResponse.newBuilder();
try {
if(!voldemortConfig.isEnableRebalanceService())
throw new VoldemortException("Rebalance service is not enabled for node: "
+ metadataStore.getNodeId());
// We should be in rebalancing state to run this function
if(!metadataStore.getServerState()
.equals(MetadataStore.VoldemortState.REBALANCING_MASTER_SERVER)) {
response.setError(ProtoUtils.encodeError(errorCodeMapper,
new VoldemortException("Voldemort server "
+ metadataStore.getNodeId()
+ " not in rebalancing state")));
return response.build();
}
RebalancePartitionsInfo rebalanceStealInfo = ProtoUtils.decodeRebalancePartitionInfoMap(request.getRebalancePartitionInfo());
int requestId = rebalancer.rebalanceNode(rebalanceStealInfo);
response.setRequestId(requestId)
.setDescription(rebalanceStealInfo.toString())
.setStatus("Started rebalancing")
.setComplete(false);
} catch(VoldemortException e) {
response.setError(ProtoUtils.encodeError(errorCodeMapper, e));
logger.error("handleRebalanceNode failed for request(" + request.toString() + ")", e);
}
return response.build();
}
public VAdminProto.GetROCurrentVersionDirResponse handleGetROCurrentVersionDir(VAdminProto.GetROCurrentVersionDirRequest request) {
final List<String> storeNames = request.getStoreNameList();
VAdminProto.GetROCurrentVersionDirResponse.Builder response = VAdminProto.GetROCurrentVersionDirResponse.newBuilder();
try {
for(String storeName: storeNames) {
ReadOnlyStorageEngine store = getReadOnlyStorageEngine(metadataStore,
storeRepository,
storeName);
VAdminProto.ROStoreVersionDirMap storeResponse = VAdminProto.ROStoreVersionDirMap.newBuilder()
.setStoreName(storeName)
.setStoreDir(store.getCurrentDirPath())
.build();
response.addRoStoreVersions(storeResponse);
}
} catch(VoldemortException e) {
response.setError(ProtoUtils.encodeError(errorCodeMapper, e));
logger.error("handleGetROCurrentVersion failed for request(" + request.toString() + ")",
e);
}
return response.build();
}
public VAdminProto.GetROMaxVersionDirResponse handleGetROMaxVersionDir(VAdminProto.GetROMaxVersionDirRequest request) {
final List<String> storeNames = request.getStoreNameList();
VAdminProto.GetROMaxVersionDirResponse.Builder response = VAdminProto.GetROMaxVersionDirResponse.newBuilder();
try {
for(String storeName: storeNames) {
ReadOnlyStorageEngine store = getReadOnlyStorageEngine(metadataStore,
storeRepository,
storeName);
File storeDirPath = new File(store.getStoreDirPath());
if(!storeDirPath.exists())
throw new VoldemortException("Unable to locate the directory of the read-only store "
+ storeName);
File[] versionDirs = ReadOnlyUtils.getVersionDirs(storeDirPath);
File[] kthDir = ReadOnlyUtils.findKthVersionedDir(versionDirs,
versionDirs.length - 1,
versionDirs.length - 1);
VAdminProto.ROStoreVersionDirMap storeResponse = VAdminProto.ROStoreVersionDirMap.newBuilder()
.setStoreName(storeName)
.setStoreDir(kthDir[0].getAbsolutePath())
.build();
response.addRoStoreVersions(storeResponse);
}
} catch(VoldemortException e) {
response.setError(ProtoUtils.encodeError(errorCodeMapper, e));
logger.error("handleGetROMaxVersion failed for request(" + request.toString() + ")", e);
}
return response.build();
}
public VAdminProto.GetROStorageFormatResponse handleGetROStorageFormat(VAdminProto.GetROStorageFormatRequest request) {
final List<String> storeNames = request.getStoreNameList();
VAdminProto.GetROStorageFormatResponse.Builder response = VAdminProto.GetROStorageFormatResponse.newBuilder();
try {
for(String storeName: storeNames) {
ReadOnlyStorageEngine store = getReadOnlyStorageEngine(metadataStore,
storeRepository,
storeName);
VAdminProto.ROStoreVersionDirMap storeResponse = VAdminProto.ROStoreVersionDirMap.newBuilder()
.setStoreName(storeName)
.setStoreDir(store.getReadOnlyStorageFormat()
.getCode())
.build();
response.addRoStoreVersions(storeResponse);
}
} catch(VoldemortException e) {
response.setError(ProtoUtils.encodeError(errorCodeMapper, e));
logger.error("handleGetROStorageFormat failed for request(" + request.toString() + ")",
e);
}
return response.build();
}
public VAdminProto.FailedFetchStoreResponse handleFailedROFetch(VAdminProto.FailedFetchStoreRequest request) {
final String storeDir = request.getStoreDir();
final String storeName = request.getStoreName();
VAdminProto.FailedFetchStoreResponse.Builder response = VAdminProto.FailedFetchStoreResponse.newBuilder();
try {
if(!Utils.isReadableDir(storeDir))
throw new VoldemortException("Could not read folder " + storeDir
+ " correctly to delete it");
final ReadOnlyStorageEngine store = getReadOnlyStorageEngine(metadataStore,
storeRepository,
storeName);
if(store.getCurrentVersionId() == ReadOnlyUtils.getVersionId(new File(storeDir))) {
logger.warn("Cannot delete " + storeDir + " for " + storeName
+ " since it is the current dir");
return response.build();
}
logger.info("Deleting data from failed fetch for RO store '" + storeName
+ "' and directory '" + storeDir + "'");
// Lets delete the folder
Utils.rm(new File(storeDir));
logger.info("Successfully deleted data from failed fetch for RO store '" + storeName
+ "' and directory '" + storeDir + "'");
} catch(VoldemortException e) {
response.setError(ProtoUtils.encodeError(errorCodeMapper, e));
logger.error("handleFailedFetch failed for request(" + request.toString() + ")", e);
}
return response.build();
}
public StreamRequestHandler handleFetchROPartitionFiles(VAdminProto.FetchPartitionFilesRequest request) {
return new FetchPartitionFileStreamRequestHandler(request,
metadataStore,
voldemortConfig,
storeRepository);
}
public StreamRequestHandler handleUpdateSlopEntries(VAdminProto.UpdateSlopEntriesRequest request) {
return new UpdateSlopEntriesRequestHandler(request,
errorCodeMapper,
storeRepository,
voldemortConfig);
}
public StreamRequestHandler handleFetchPartitionEntries(VAdminProto.FetchPartitionEntriesRequest request) {
boolean fetchValues = request.hasFetchValues() && request.getFetchValues();
boolean fetchOrphaned = request.hasFetchOrphaned() && request.getFetchOrphaned();
StorageEngine<ByteArray, byte[], byte[]> storageEngine = AdminServiceRequestHandler.getStorageEngine(storeRepository,
request.getStore());
if(fetchValues) {
if(storageEngine.isPartitionScanSupported() && !fetchOrphaned)
return new PartitionScanFetchEntriesRequestHandler(request,
metadataStore,
errorCodeMapper,
voldemortConfig,
storeRepository,
networkClassLoader);
else
return new FullScanFetchEntriesRequestHandler(request,
metadataStore,
errorCodeMapper,
voldemortConfig,
storeRepository,
networkClassLoader);
} else {
if(storageEngine.isPartitionScanSupported() && !fetchOrphaned)
return new PartitionScanFetchKeysRequestHandler(request,
metadataStore,
errorCodeMapper,
voldemortConfig,
storeRepository,
networkClassLoader);
else
return new FullScanFetchKeysRequestHandler(request,
metadataStore,
errorCodeMapper,
voldemortConfig,
storeRepository,
networkClassLoader);
}
}
public StreamRequestHandler handleUpdatePartitionEntries(VAdminProto.UpdatePartitionEntriesRequest request) {
StorageEngine<ByteArray, byte[], byte[]> storageEngine = AdminServiceRequestHandler.getStorageEngine(storeRepository,
request.getStore());
if(storageEngine instanceof MysqlStorageEngine) {
// TODO This check is ugly. Need some generic capability to check
// which storage engine supports which operations.
return new UpdatePartitionEntriesStreamRequestHandler(request,
errorCodeMapper,
voldemortConfig,
storageEngine,
storeRepository,
networkClassLoader);
} else {
return new BufferedUpdatePartitionEntriesStreamRequestHandler(request,
errorCodeMapper,
voldemortConfig,
storageEngine,
storeRepository,
networkClassLoader);
}
}
public VAdminProto.AsyncOperationListResponse handleAsyncOperationList(VAdminProto.AsyncOperationListRequest request) {
VAdminProto.AsyncOperationListResponse.Builder response = VAdminProto.AsyncOperationListResponse.newBuilder();
boolean showComplete = request.getShowComplete();
try {
logger.info("Retrieving list of async operations "
+ ((showComplete) ? " [ including completed ids ]" : ""));
List<Integer> asyncIds = asyncService.getAsyncOperationList(showComplete);
logger.info("Retrieved list of async operations - " + asyncIds);
response.addAllRequestIds(asyncIds);
} catch(VoldemortException e) {
response.setError(ProtoUtils.encodeError(errorCodeMapper, e));
logger.error("handleAsyncOperationList failed for request(" + request.toString() + ")",
e);
}
return response.build();
}
public VAdminProto.AsyncOperationStopResponse handleAsyncOperationStop(VAdminProto.AsyncOperationStopRequest request) {
VAdminProto.AsyncOperationStopResponse.Builder response = VAdminProto.AsyncOperationStopResponse.newBuilder();
int requestId = request.getRequestId();
try {
logger.info("Stopping async id " + requestId);
asyncService.stopOperation(requestId);
logger.info("Successfully stopped async id " + requestId);
} catch(VoldemortException e) {
response.setError(ProtoUtils.encodeError(errorCodeMapper, e));
logger.error("handleAsyncOperationStop failed for request(" + request.toString() + ")",
e);
}
return response.build();
}
public VAdminProto.RollbackStoreResponse handleRollbackStore(VAdminProto.RollbackStoreRequest request) {
final String storeName = request.getStoreName();
final long pushVersion = request.getPushVersion();
VAdminProto.RollbackStoreResponse.Builder response = VAdminProto.RollbackStoreResponse.newBuilder();
try {
ReadOnlyStorageEngine store = getReadOnlyStorageEngine(metadataStore,
storeRepository,
storeName);
File rollbackVersionDir = new File(store.getStoreDirPath(), "version-" + pushVersion);
logger.info("Rolling back data for RO store '" + storeName + "' to version directory '"
+ rollbackVersionDir + "'");
store.rollback(rollbackVersionDir);
logger.info("Successfully rolled back data for RO store '" + storeName
+ "' to version directory '" + rollbackVersionDir + "'");
} catch(VoldemortException e) {
response.setError(ProtoUtils.encodeError(errorCodeMapper, e));
logger.error("handleRollbackStore failed for request(" + request.toString() + ")", e);
}
return response.build();
}
public VAdminProto.RepairJobResponse handleRepairJob(VAdminProto.RepairJobRequest request) {
VAdminProto.RepairJobResponse.Builder response = VAdminProto.RepairJobResponse.newBuilder();
try {
int requestId = asyncService.getUniqueRequestId();
asyncService.submitOperation(requestId, new AsyncOperation(requestId, "Repair Job") {
@Override
public void operate() {
RepairJob job = storeRepository.getRepairJob();
if(job != null) {
logger.info("Starting the repair job now on ID : "
+ metadataStore.getNodeId());
job.run();
} else
logger.error("RepairJob is not initialized.");
}
@Override
public void stop() {
status.setException(new VoldemortException("Repair job interrupted"));
}
});
} catch(VoldemortException e) {
response.setError(ProtoUtils.encodeError(errorCodeMapper, e));
logger.error("Repair job failed for request : " + request.toString() + ")", e);
}
return response.build();
}
/**
* Given a read-only store name and a directory, swaps it in while returning
* the directory path being swapped out
*
* @param storeName The name of the read-only store
* @param directory The directory being swapped in
* @return The directory path which was swapped out
* @throws VoldemortException
*/
private String swapStore(String storeName, String directory) throws VoldemortException {
ReadOnlyStorageEngine store = getReadOnlyStorageEngine(metadataStore,
storeRepository,
storeName);
if(!Utils.isReadableDir(directory))
throw new VoldemortException("Store directory '" + directory
+ "' is not a readable directory.");
String currentDirPath = store.getCurrentDirPath();
logger.info("Swapping RO store '" + storeName + "' to version directory '" + directory
+ "'");
store.swapFiles(directory);
logger.info("Swapping swapped RO store '" + storeName + "' to version directory '"
+ directory + "'");
return currentDirPath;
}
public VAdminProto.SwapStoreResponse handleSwapROStore(VAdminProto.SwapStoreRequest request) {
final String dir = request.getStoreDir();
final String storeName = request.getStoreName();
VAdminProto.SwapStoreResponse.Builder response = VAdminProto.SwapStoreResponse.newBuilder();
if(!metadataStore.getServerState().equals(MetadataStore.VoldemortState.NORMAL_SERVER)) {
response.setError(ProtoUtils.encodeError(errorCodeMapper,
new VoldemortException("Voldemort server "
+ metadataStore.getNodeId()
+ " not in normal state while swapping store "
+ storeName
+ " with directory "
+ dir)));
return response.build();
}
try {
response.setPreviousStoreDir(swapStore(storeName, dir));
return response.build();
} catch(VoldemortException e) {
response.setError(ProtoUtils.encodeError(errorCodeMapper, e));
logger.error("handleSwapStore failed for request(" + request.toString() + ")", e);
return response.build();
}
}
public VAdminProto.AsyncOperationStatusResponse handleFetchROStore(VAdminProto.FetchStoreRequest request) {
final String fetchUrl = request.getStoreDir();
final String storeName = request.getStoreName();
int requestId = asyncService.getUniqueRequestId();
VAdminProto.AsyncOperationStatusResponse.Builder response = VAdminProto.AsyncOperationStatusResponse.newBuilder()
.setRequestId(requestId)
.setComplete(false)
.setDescription("Fetch store")
.setStatus("started");
try {
final ReadOnlyStorageEngine store = getReadOnlyStorageEngine(metadataStore,
storeRepository,
storeName);
final long pushVersion;
if(request.hasPushVersion()) {
pushVersion = request.getPushVersion();
if(pushVersion <= store.getCurrentVersionId())
throw new VoldemortException("Version of push specified (" + pushVersion
+ ") should be greater than current version "
+ store.getCurrentVersionId() + " for store "
+ storeName + " on node "
+ metadataStore.getNodeId());
} else {
// Find the max version
long maxVersion;
File[] storeDirList = ReadOnlyUtils.getVersionDirs(new File(store.getStoreDirPath()));
if(storeDirList == null || storeDirList.length == 0) {
throw new VoldemortException("Push version required since no version folders exist for store "
+ storeName
+ " on node "
+ metadataStore.getNodeId());
} else {
maxVersion = ReadOnlyUtils.getVersionId(ReadOnlyUtils.findKthVersionedDir(storeDirList,
storeDirList.length - 1,
storeDirList.length - 1)[0]);
}
pushVersion = maxVersion + 1;
}
asyncService.submitOperation(requestId, new AsyncOperation(requestId, "Fetch store") {
private String fetchDirPath = null;
@Override
public void markComplete() {
if(fetchDirPath != null)
status.setStatus(fetchDirPath);
status.setComplete(true);
}
@Override
public void operate() {
File fetchDir = null;
if(fileFetcher == null) {
logger.warn("File fetcher class has not instantiated correctly. Assuming local file");
if(!Utils.isReadableDir(fetchUrl)) {
throw new VoldemortException("Fetch url " + fetchUrl
+ " is not readable");
}
fetchDir = new File(store.getStoreDirPath(), "version-"
+ Long.toString(pushVersion));
if(fetchDir.exists())
throw new VoldemortException("Version directory "
+ fetchDir.getAbsolutePath()
+ " already exists");
Utils.move(new File(fetchUrl), fetchDir);
} else {
logger.info("Started executing fetch of " + fetchUrl + " for RO store '"
+ storeName + "'");
updateStatus("0 MB copied at 0 MB/sec - 0 % complete");
try {
fileFetcher.setAsyncOperationStatus(status);
fetchDir = fileFetcher.fetch(fetchUrl, store.getStoreDirPath()
+ File.separator + "version-"
+ Long.toString(pushVersion));
if(fetchDir == null) {
String errorMessage = "File fetcher failed for "
+ fetchUrl
+ " and store '"
+ storeName
+ "' due to incorrect input path/checksum error";
updateStatus(errorMessage);
logger.error(errorMessage);
throw new VoldemortException(errorMessage);
} else {
String message = "Successfully executed fetch of " + fetchUrl
+ " for RO store '" + storeName + "'";
updateStatus(message);
logger.info(message);
}
} catch(VoldemortException ve) {
String errorMessage = "File fetcher failed for " + fetchUrl
+ " and store '" + storeName + "' Reason: \n"
+ ve.getMessage();
updateStatus(errorMessage);
logger.error(errorMessage);
throw new VoldemortException(errorMessage);
} catch(Exception e) {
throw new VoldemortException("Exception in Fetcher = " + e.getMessage());
}
}
fetchDirPath = fetchDir.getAbsolutePath();
}
@Override
public void stop() {
status.setException(new VoldemortException("Fetcher interrupted"));
}
});
} catch(VoldemortException e) {
response.setError(ProtoUtils.encodeError(errorCodeMapper, e));
logger.error("handleFetchStore failed for request(" + request.toString() + ")", e);
}
return response.build();
}
public VAdminProto.AsyncOperationStatusResponse handleFetchAndUpdate(VAdminProto.InitiateFetchAndUpdateRequest request) {
final int nodeId = request.getNodeId();
final HashMap<Integer, List<Integer>> replicaToPartitionList = ProtoUtils.decodePartitionTuple(request.getReplicaToPartitionList());
final VoldemortFilter filter = request.hasFilter() ? getFilterFromRequest(request.getFilter(),
voldemortConfig,
networkClassLoader)
: new DefaultVoldemortFilter();
final String storeName = request.getStore();
final boolean optimize = request.hasOptimize() ? request.getOptimize() : false;
final Cluster initialCluster = request.hasInitialCluster() ? new ClusterMapper().readCluster(new StringReader(request.getInitialCluster()))
: null;
int requestId = asyncService.getUniqueRequestId();
VAdminProto.AsyncOperationStatusResponse.Builder response = VAdminProto.AsyncOperationStatusResponse.newBuilder()
.setRequestId(requestId)
.setComplete(false)
.setDescription("Fetch and update")
.setStatus("Started");
final StoreDefinition storeDef = metadataStore.getStoreDef(storeName);
final boolean isReadOnlyStore = storeDef.getType()
.compareTo(ReadOnlyStorageConfiguration.TYPE_NAME) == 0;
try {
asyncService.submitOperation(requestId, new AsyncOperation(requestId,
"Fetch and Update") {
private final AtomicBoolean running = new AtomicBoolean(true);
@Override
public void stop() {
running.set(false);
logger.info("Stopping fetch and update for store " + storeName + " from node "
+ nodeId + "( " + replicaToPartitionList + " )");
}
@Override
public void operate() {
AdminClient adminClient = RebalanceUtils.createTempAdminClient(voldemortConfig,
metadataStore.getCluster(),
voldemortConfig.getClientMaxConnectionsPerNode());
try {
StorageEngine<ByteArray, byte[], byte[]> storageEngine = getStorageEngine(storeRepository,
storeName);
EventThrottler throttler = new EventThrottler(voldemortConfig.getStreamMaxWriteBytesPerSec());
if(isReadOnlyStore) {
ReadOnlyStorageEngine readOnlyStorageEngine = ((ReadOnlyStorageEngine) storageEngine);
String destinationDir = readOnlyStorageEngine.getCurrentDirPath();
logger.info("Fetching files for RO store '" + storeName
+ "' from node " + nodeId + " ( " + replicaToPartitionList
+ " )");
updateStatus("Fetching files for RO store '" + storeName
+ "' from node " + nodeId + " ( " + replicaToPartitionList
+ " )");
// TODO: Optimization to get rid of redundant
// copying of data which already exists on this
// node. This should simply copy and rename the
// existing replica file locally if they exist.
// Should not do rename only because then we won't
// be able to rollback
adminClient.readonlyOps.fetchPartitionFiles(nodeId,
storeName,
replicaToPartitionList,
destinationDir,
readOnlyStorageEngine.getChunkedFileSet()
.getChunkIdToNumChunks()
.keySet(),
running);
} else {
logger.info("Fetching entries for RW store '" + storeName
+ "' from node " + nodeId + " ( " + replicaToPartitionList
+ " )");
updateStatus("Fetching entries for RW store '" + storeName
+ "' from node " + nodeId + " ( " + replicaToPartitionList
+ " ) ");
// Optimization to get rid of redundant copying of
// data which already exists on this node
HashMap<Integer, List<Integer>> optimizedReplicaToPartitionList = Maps.newHashMap();
if(initialCluster != null && optimize
&& !storageEngine.isPartitionAware()
&& voldemortConfig.getRebalancingOptimization()) {
optimizedReplicaToPartitionList.putAll(RebalanceUtils.getOptimizedReplicaToPartitionList(metadataStore.getNodeId(),
initialCluster,
storeDef,
replicaToPartitionList));
logger.info("After running RW level optimization - Fetching entries for RW store '"
+ storeName
+ "' from node "
+ nodeId
+ " ( "
+ optimizedReplicaToPartitionList + " )");
updateStatus("After running RW level optimization - Fetching entries for RW store '"
+ storeName
+ "' from node "
+ nodeId
+ " ( "
+ optimizedReplicaToPartitionList + " )");
} else {
optimizedReplicaToPartitionList.putAll(replicaToPartitionList);
}
if(optimizedReplicaToPartitionList.size() > 0) {
Iterator<Pair<ByteArray, Versioned<byte[]>>> entriesIterator = adminClient.bulkFetchOps.fetchEntries(nodeId,
storeName,
optimizedReplicaToPartitionList,
filter,
false,
initialCluster,
0);
long numTuples = 0;
long startTime = System.currentTimeMillis();
while(running.get() && entriesIterator.hasNext()) {
Pair<ByteArray, Versioned<byte[]>> entry = entriesIterator.next();
ByteArray key = entry.getFirst();
Versioned<byte[]> value = entry.getSecond();
try {