-
Notifications
You must be signed in to change notification settings - Fork 3.3k
/
RSRpcServices.java
4003 lines (3743 loc) · 174 KB
/
RSRpcServices.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
/*
* 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.hadoop.hbase.regionserver;
import com.google.errorprone.annotations.RestrictedApi;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.net.BindException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.NavigableMap;
import java.util.Optional;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
import org.apache.hadoop.hbase.CacheEvictionStats;
import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.DroppedSnapshotException;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MultiActionResultTooLarge;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.RegionTooBusyException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.CheckAndMutate;
import org.apache.hadoop.hbase.client.CheckAndMutateResult;
import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Row;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.VersionInfoUtil;
import org.apache.hadoop.hbase.conf.ConfigurationObserver;
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
import org.apache.hadoop.hbase.io.ByteBuffAllocator;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.ipc.PriorityFunction;
import org.apache.hadoop.hbase.ipc.QosPriority;
import org.apache.hadoop.hbase.ipc.RpcCall;
import org.apache.hadoop.hbase.ipc.RpcCallContext;
import org.apache.hadoop.hbase.ipc.RpcCallback;
import org.apache.hadoop.hbase.ipc.RpcScheduler;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
import org.apache.hadoop.hbase.ipc.RpcServerFactory;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.log.HBaseMarkers;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterRpcServices;
import org.apache.hadoop.hbase.namequeues.NamedQueuePayload;
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
import org.apache.hadoop.hbase.namequeues.RpcLogDetails;
import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
import org.apache.hadoop.hbase.net.Address;
import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement;
import org.apache.hadoop.hbase.quotas.OperationQuota;
import org.apache.hadoop.hbase.quotas.QuotaUtil;
import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
import org.apache.hadoop.hbase.regionserver.LeaseManager.Lease;
import org.apache.hadoop.hbase.regionserver.LeaseManager.LeaseStillHeldException;
import org.apache.hadoop.hbase.regionserver.Region.Operation;
import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
import org.apache.hadoop.hbase.regionserver.handler.AssignRegionHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenPriorityRegionHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
import org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler;
import org.apache.hadoop.hbase.security.Superusers;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.access.AccessChecker;
import org.apache.hadoop.hbase.security.access.NoopAccessChecker;
import org.apache.hadoop.hbase.security.access.Permission;
import org.apache.hadoop.hbase.security.access.ZKPermissionWatcher;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.DNS;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALSplitUtil;
import org.apache.hadoop.hbase.wal.WALSplitUtil.MutationReplay;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearSlowLogResponses;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponseRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponses;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;
import org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
/**
* Implements the regionserver RPC services.
*/
@InterfaceAudience.Private
@SuppressWarnings("deprecation")
public class RSRpcServices implements HBaseRPCErrorHandler, AdminService.BlockingInterface,
ClientService.BlockingInterface, PriorityFunction, ConfigurationObserver {
protected static final Logger LOG = LoggerFactory.getLogger(RSRpcServices.class);
/** RPC scheduler to use for the region server. */
public static final String REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS =
"hbase.region.server.rpc.scheduler.factory.class";
/** RPC scheduler to use for the master. */
public static final String MASTER_RPC_SCHEDULER_FACTORY_CLASS =
"hbase.master.rpc.scheduler.factory.class";
/**
* Minimum allowable time limit delta (in milliseconds) that can be enforced during scans. This
* configuration exists to prevent the scenario where a time limit is specified to be so
* restrictive that the time limit is reached immediately (before any cells are scanned).
*/
private static final String REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA =
"hbase.region.server.rpc.minimum.scan.time.limit.delta";
/**
* Default value of {@link RSRpcServices#REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA}
*/
static final long DEFAULT_REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA = 10;
/*
* Whether to reject rows with size > threshold defined by {@link
* RSRpcServices#BATCH_ROWS_THRESHOLD_NAME}
*/
private static final String REJECT_BATCH_ROWS_OVER_THRESHOLD =
"hbase.rpc.rows.size.threshold.reject";
/*
* Default value of config {@link RSRpcServices#REJECT_BATCH_ROWS_OVER_THRESHOLD}
*/
private static final boolean DEFAULT_REJECT_BATCH_ROWS_OVER_THRESHOLD = false;
// Request counter. (Includes requests that are not serviced by regions.)
// Count only once for requests with multiple actions like multi/caching-scan/replayBatch
final LongAdder requestCount = new LongAdder();
// Request counter for rpc get
final LongAdder rpcGetRequestCount = new LongAdder();
// Request counter for rpc scan
final LongAdder rpcScanRequestCount = new LongAdder();
// Request counter for scans that might end up in full scans
final LongAdder rpcFullScanRequestCount = new LongAdder();
// Request counter for rpc multi
final LongAdder rpcMultiRequestCount = new LongAdder();
// Request counter for rpc mutate
final LongAdder rpcMutateRequestCount = new LongAdder();
// Server to handle client requests.
final RpcServerInterface rpcServer;
final InetSocketAddress isa;
protected final HRegionServer regionServer;
private final long maxScannerResultSize;
// The reference to the priority extraction function
private final PriorityFunction priority;
private ScannerIdGenerator scannerIdGenerator;
private final ConcurrentMap<String, RegionScannerHolder> scanners = new ConcurrentHashMap<>();
// Hold the name of a closed scanner for a while. This is used to keep compatible for old clients
// which may send next or close request to a region scanner which has already been exhausted. The
// entries will be removed automatically after scannerLeaseTimeoutPeriod.
private final Cache<String, String> closedScanners;
/**
* The lease timeout period for client scanners (milliseconds).
*/
private final int scannerLeaseTimeoutPeriod;
/**
* The RPC timeout period (milliseconds)
*/
private final int rpcTimeout;
/**
* The minimum allowable delta to use for the scan limit
*/
private final long minimumScanTimeLimitDelta;
/**
* Row size threshold for multi requests above which a warning is logged
*/
private final int rowSizeWarnThreshold;
/*
* Whether we should reject requests with very high no of rows i.e. beyond threshold defined by
* rowSizeWarnThreshold
*/
private final boolean rejectRowsWithSizeOverThreshold;
final AtomicBoolean clearCompactionQueues = new AtomicBoolean(false);
private AccessChecker accessChecker;
private ZKPermissionWatcher zkPermissionWatcher;
/**
* Services launched in RSRpcServices. By default they are on but you can use the below booleans
* to selectively enable/disable either Admin or Client Service (Rare is the case where you would
* ever turn off one or the other).
*/
public static final String REGIONSERVER_ADMIN_SERVICE_CONFIG =
"hbase.regionserver.admin.executorService";
public static final String REGIONSERVER_CLIENT_SERVICE_CONFIG =
"hbase.regionserver.client.executorService";
/**
* An Rpc callback for closing a RegionScanner.
*/
private static final class RegionScannerCloseCallBack implements RpcCallback {
private final RegionScanner scanner;
public RegionScannerCloseCallBack(RegionScanner scanner) {
this.scanner = scanner;
}
@Override
public void run() throws IOException {
this.scanner.close();
}
}
/**
* An Rpc callback for doing shipped() call on a RegionScanner.
*/
private class RegionScannerShippedCallBack implements RpcCallback {
private final String scannerName;
private final Shipper shipper;
private final Lease lease;
public RegionScannerShippedCallBack(String scannerName, Shipper shipper, Lease lease) {
this.scannerName = scannerName;
this.shipper = shipper;
this.lease = lease;
}
@Override
public void run() throws IOException {
this.shipper.shipped();
// We're done. On way out re-add the above removed lease. The lease was temp removed for this
// Rpc call and we are at end of the call now. Time to add it back.
if (scanners.containsKey(scannerName)) {
if (lease != null) {
regionServer.getLeaseManager().addLease(lease);
}
}
}
}
/**
* An RpcCallBack that creates a list of scanners that needs to perform callBack operation on
* completion of multiGets.
*/
static class RegionScannersCloseCallBack implements RpcCallback {
private final List<RegionScanner> scanners = new ArrayList<>();
public void addScanner(RegionScanner scanner) {
this.scanners.add(scanner);
}
@Override
public void run() {
for (RegionScanner scanner : scanners) {
try {
scanner.close();
} catch (IOException e) {
LOG.error("Exception while closing the scanner " + scanner, e);
}
}
}
}
/**
* Holder class which holds the RegionScanner, nextCallSeq and RpcCallbacks together.
*/
static final class RegionScannerHolder {
private final AtomicLong nextCallSeq = new AtomicLong(0);
private final RegionScanner s;
private final HRegion r;
private final RpcCallback closeCallBack;
private final RpcCallback shippedCallback;
private byte[] rowOfLastPartialResult;
private boolean needCursor;
private boolean fullRegionScan;
private final String clientIPAndPort;
private final String userName;
RegionScannerHolder(RegionScanner s, HRegion r, RpcCallback closeCallBack,
RpcCallback shippedCallback, boolean needCursor, boolean fullRegionScan,
String clientIPAndPort, String userName) {
this.s = s;
this.r = r;
this.closeCallBack = closeCallBack;
this.shippedCallback = shippedCallback;
this.needCursor = needCursor;
this.fullRegionScan = fullRegionScan;
this.clientIPAndPort = clientIPAndPort;
this.userName = userName;
}
long getNextCallSeq() {
return nextCallSeq.get();
}
boolean incNextCallSeq(long currentSeq) {
// Use CAS to prevent multiple scan request running on the same scanner.
return nextCallSeq.compareAndSet(currentSeq, currentSeq + 1);
}
// Should be called only when we need to print lease expired messages otherwise
// cache the String once made.
@Override
public String toString() {
return "clientIPAndPort=" + this.clientIPAndPort + ", userName=" + this.userName
+ ", regionInfo=" + this.r.getRegionInfo().getRegionNameAsString();
}
}
/**
* Instantiated as a scanner lease. If the lease times out, the scanner is closed
*/
private class ScannerListener implements LeaseListener {
private final String scannerName;
ScannerListener(final String n) {
this.scannerName = n;
}
@Override
public void leaseExpired() {
RegionScannerHolder rsh = scanners.remove(this.scannerName);
if (rsh == null) {
LOG.warn("Scanner lease {} expired but no outstanding scanner", this.scannerName);
return;
}
LOG.info("Scanner lease {} expired {}", this.scannerName, rsh);
RegionScanner s = rsh.s;
HRegion region = null;
try {
region = regionServer.getRegion(s.getRegionInfo().getRegionName());
if (region != null && region.getCoprocessorHost() != null) {
region.getCoprocessorHost().preScannerClose(s);
}
} catch (IOException e) {
LOG.error("Closing scanner {} {}", this.scannerName, rsh, e);
} finally {
try {
s.close();
if (region != null && region.getCoprocessorHost() != null) {
region.getCoprocessorHost().postScannerClose(s);
}
} catch (IOException e) {
LOG.error("Closing scanner {} {}", this.scannerName, rsh, e);
}
}
}
}
private static ResultOrException getResultOrException(final ClientProtos.Result r,
final int index) {
return getResultOrException(ResponseConverter.buildActionResult(r), index);
}
private static ResultOrException getResultOrException(final Exception e, final int index) {
return getResultOrException(ResponseConverter.buildActionResult(e), index);
}
private static ResultOrException getResultOrException(final ResultOrException.Builder builder,
final int index) {
return builder.setIndex(index).build();
}
/**
* Checks for the following pre-checks in order:
* <ol>
* <li>RegionServer is running</li>
* <li>If authorization is enabled, then RPC caller has ADMIN permissions</li>
* </ol>
* @param requestName name of rpc request. Used in reporting failures to provide context.
* @throws ServiceException If any of the above listed pre-check fails.
*/
private void rpcPreCheck(String requestName) throws ServiceException {
try {
checkOpen();
requirePermission(requestName, Permission.Action.ADMIN);
} catch (IOException ioe) {
throw new ServiceException(ioe);
}
}
private boolean isClientCellBlockSupport(RpcCallContext context) {
return context != null && context.isClientCellBlockSupported();
}
private void addResult(final MutateResponse.Builder builder, final Result result,
final HBaseRpcController rpcc, boolean clientCellBlockSupported) {
if (result == null) return;
if (clientCellBlockSupported) {
builder.setResult(ProtobufUtil.toResultNoData(result));
rpcc.setCellScanner(result.cellScanner());
} else {
ClientProtos.Result pbr = ProtobufUtil.toResult(result);
builder.setResult(pbr);
}
}
private void addResults(ScanResponse.Builder builder, List<Result> results,
HBaseRpcController controller, boolean isDefaultRegion, boolean clientCellBlockSupported) {
builder.setStale(!isDefaultRegion);
if (results.isEmpty()) {
return;
}
if (clientCellBlockSupported) {
for (Result res : results) {
builder.addCellsPerResult(res.size());
builder.addPartialFlagPerResult(res.mayHaveMoreCellsInRow());
}
controller.setCellScanner(CellUtil.createCellScanner(results));
} else {
for (Result res : results) {
ClientProtos.Result pbr = ProtobufUtil.toResult(res);
builder.addResults(pbr);
}
}
}
private CheckAndMutateResult checkAndMutate(HRegion region, List<ClientProtos.Action> actions,
CellScanner cellScanner, Condition condition, long nonceGroup,
ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException {
int countOfCompleteMutation = 0;
try {
if (!region.getRegionInfo().isMetaRegion()) {
regionServer.getMemStoreFlusher().reclaimMemStoreMemory();
}
List<Mutation> mutations = new ArrayList<>();
long nonce = HConstants.NO_NONCE;
for (ClientProtos.Action action : actions) {
if (action.hasGet()) {
throw new DoNotRetryIOException(
"Atomic put and/or delete only, not a Get=" + action.getGet());
}
MutationProto mutation = action.getMutation();
MutationType type = mutation.getMutateType();
switch (type) {
case PUT:
Put put = ProtobufUtil.toPut(mutation, cellScanner);
++countOfCompleteMutation;
checkCellSizeLimit(region, put);
spaceQuotaEnforcement.getPolicyEnforcement(region).check(put);
mutations.add(put);
break;
case DELETE:
Delete del = ProtobufUtil.toDelete(mutation, cellScanner);
++countOfCompleteMutation;
spaceQuotaEnforcement.getPolicyEnforcement(region).check(del);
mutations.add(del);
break;
case INCREMENT:
Increment increment = ProtobufUtil.toIncrement(mutation, cellScanner);
nonce = mutation.hasNonce() ? mutation.getNonce() : HConstants.NO_NONCE;
++countOfCompleteMutation;
checkCellSizeLimit(region, increment);
spaceQuotaEnforcement.getPolicyEnforcement(region).check(increment);
mutations.add(increment);
break;
case APPEND:
Append append = ProtobufUtil.toAppend(mutation, cellScanner);
nonce = mutation.hasNonce() ? mutation.getNonce() : HConstants.NO_NONCE;
++countOfCompleteMutation;
checkCellSizeLimit(region, append);
spaceQuotaEnforcement.getPolicyEnforcement(region).check(append);
mutations.add(append);
break;
default:
throw new DoNotRetryIOException("invalid mutation type : " + type);
}
}
if (mutations.size() == 0) {
return new CheckAndMutateResult(true, null);
} else {
CheckAndMutate checkAndMutate = ProtobufUtil.toCheckAndMutate(condition, mutations);
CheckAndMutateResult result = null;
if (region.getCoprocessorHost() != null) {
result = region.getCoprocessorHost().preCheckAndMutate(checkAndMutate);
}
if (result == null) {
result = region.checkAndMutate(checkAndMutate, nonceGroup, nonce);
if (region.getCoprocessorHost() != null) {
result = region.getCoprocessorHost().postCheckAndMutate(checkAndMutate, result);
}
}
return result;
}
} finally {
// Currently, the checkAndMutate isn't supported by batch so it won't mess up the cell scanner
// even if the malformed cells are not skipped.
for (int i = countOfCompleteMutation; i < actions.size(); ++i) {
skipCellsForMutation(actions.get(i), cellScanner);
}
}
}
/**
* Execute an append mutation.
* @return result to return to client if default operation should be bypassed as indicated by
* RegionObserver, null otherwise
*/
private Result append(final HRegion region, final OperationQuota quota,
final MutationProto mutation, final CellScanner cellScanner, long nonceGroup,
ActivePolicyEnforcement spaceQuota) throws IOException {
long before = EnvironmentEdgeManager.currentTime();
Append append = ProtobufUtil.toAppend(mutation, cellScanner);
checkCellSizeLimit(region, append);
spaceQuota.getPolicyEnforcement(region).check(append);
quota.addMutation(append);
long nonce = mutation.hasNonce() ? mutation.getNonce() : HConstants.NO_NONCE;
Result r = region.append(append, nonceGroup, nonce);
if (regionServer.getMetrics() != null) {
regionServer.getMetrics().updateAppend(region.getTableDescriptor().getTableName(),
EnvironmentEdgeManager.currentTime() - before);
}
return r == null ? Result.EMPTY_RESULT : r;
}
/**
* Execute an increment mutation.
*/
private Result increment(final HRegion region, final OperationQuota quota,
final MutationProto mutation, final CellScanner cells, long nonceGroup,
ActivePolicyEnforcement spaceQuota) throws IOException {
long before = EnvironmentEdgeManager.currentTime();
Increment increment = ProtobufUtil.toIncrement(mutation, cells);
checkCellSizeLimit(region, increment);
spaceQuota.getPolicyEnforcement(region).check(increment);
quota.addMutation(increment);
long nonce = mutation.hasNonce() ? mutation.getNonce() : HConstants.NO_NONCE;
Result r = region.increment(increment, nonceGroup, nonce);
final MetricsRegionServer metricsRegionServer = regionServer.getMetrics();
if (metricsRegionServer != null) {
metricsRegionServer.updateIncrement(region.getTableDescriptor().getTableName(),
EnvironmentEdgeManager.currentTime() - before);
}
return r == null ? Result.EMPTY_RESULT : r;
}
/**
* Run through the regionMutation <code>rm</code> and per Mutation, do the work, and then when
* done, add an instance of a {@link ResultOrException} that corresponds to each Mutation.
* @param cellsToReturn Could be null. May be allocated in this method. This is what this method
* returns as a 'result'.
* @param closeCallBack the callback to be used with multigets
* @param context the current RpcCallContext
* @return Return the <code>cellScanner</code> passed
*/
private List<CellScannable> doNonAtomicRegionMutation(final HRegion region,
final OperationQuota quota, final RegionAction actions, final CellScanner cellScanner,
final RegionActionResult.Builder builder, List<CellScannable> cellsToReturn, long nonceGroup,
final RegionScannersCloseCallBack closeCallBack, RpcCallContext context,
ActivePolicyEnforcement spaceQuotaEnforcement) {
// Gather up CONTIGUOUS Puts and Deletes in this mutations List. Idea is that rather than do
// one at a time, we instead pass them in batch. Be aware that the corresponding
// ResultOrException instance that matches each Put or Delete is then added down in the
// doNonAtomicBatchOp call. We should be staying aligned though the Put and Delete are
// deferred/batched
List<ClientProtos.Action> mutations = null;
long maxQuotaResultSize = Math.min(maxScannerResultSize, quota.getReadAvailable());
IOException sizeIOE = null;
Object lastBlock = null;
ClientProtos.ResultOrException.Builder resultOrExceptionBuilder =
ResultOrException.newBuilder();
boolean hasResultOrException = false;
for (ClientProtos.Action action : actions.getActionList()) {
hasResultOrException = false;
resultOrExceptionBuilder.clear();
try {
Result r = null;
if (
context != null && context.isRetryImmediatelySupported()
&& (context.getResponseCellSize() > maxQuotaResultSize
|| context.getResponseBlockSize() + context.getResponseExceptionSize()
> maxQuotaResultSize)
) {
// We're storing the exception since the exception and reason string won't
// change after the response size limit is reached.
if (sizeIOE == null) {
// We don't need the stack un-winding do don't throw the exception.
// Throwing will kill the JVM's JIT.
//
// Instead just create the exception and then store it.
sizeIOE = new MultiActionResultTooLarge("Max size exceeded" + " CellSize: "
+ context.getResponseCellSize() + " BlockSize: " + context.getResponseBlockSize());
// Only report the exception once since there's only one request that
// caused the exception. Otherwise this number will dominate the exceptions count.
rpcServer.getMetrics().exception(sizeIOE);
}
// Now that there's an exception is known to be created
// use it for the response.
//
// This will create a copy in the builder.
NameBytesPair pair = ResponseConverter.buildException(sizeIOE);
resultOrExceptionBuilder.setException(pair);
context.incrementResponseExceptionSize(pair.getSerializedSize());
resultOrExceptionBuilder.setIndex(action.getIndex());
builder.addResultOrException(resultOrExceptionBuilder.build());
skipCellsForMutation(action, cellScanner);
continue;
}
if (action.hasGet()) {
long before = EnvironmentEdgeManager.currentTime();
ClientProtos.Get pbGet = action.getGet();
// An asynchbase client, https://github.com/OpenTSDB/asynchbase, starts by trying to do
// a get closest before. Throwing the UnknownProtocolException signals it that it needs
// to switch and do hbase2 protocol (HBase servers do not tell clients what versions
// they are; its a problem for non-native clients like asynchbase. HBASE-20225.
if (pbGet.hasClosestRowBefore() && pbGet.getClosestRowBefore()) {
throw new UnknownProtocolException("Is this a pre-hbase-1.0.0 or asynchbase client? "
+ "Client is invoking getClosestRowBefore removed in hbase-2.0.0 replaced by "
+ "reverse Scan.");
}
try {
Get get = ProtobufUtil.toGet(pbGet);
if (context != null) {
r = get(get, (region), closeCallBack, context);
} else {
r = region.get(get);
}
} finally {
final MetricsRegionServer metricsRegionServer = regionServer.getMetrics();
if (metricsRegionServer != null) {
metricsRegionServer.updateGet(region.getTableDescriptor().getTableName(),
EnvironmentEdgeManager.currentTime() - before);
}
}
} else if (action.hasServiceCall()) {
hasResultOrException = true;
com.google.protobuf.Message result = execServiceOnRegion(region, action.getServiceCall());
ClientProtos.CoprocessorServiceResult.Builder serviceResultBuilder =
ClientProtos.CoprocessorServiceResult.newBuilder();
resultOrExceptionBuilder.setServiceResult(serviceResultBuilder
.setValue(serviceResultBuilder.getValueBuilder().setName(result.getClass().getName())
// TODO: Copy!!!
.setValue(UnsafeByteOperations.unsafeWrap(result.toByteArray()))));
} else if (action.hasMutation()) {
MutationType type = action.getMutation().getMutateType();
if (
type != MutationType.PUT && type != MutationType.DELETE && mutations != null
&& !mutations.isEmpty()
) {
// Flush out any Puts or Deletes already collected.
doNonAtomicBatchOp(builder, region, quota, mutations, cellScanner,
spaceQuotaEnforcement);
mutations.clear();
}
switch (type) {
case APPEND:
r = append(region, quota, action.getMutation(), cellScanner, nonceGroup,
spaceQuotaEnforcement);
break;
case INCREMENT:
r = increment(region, quota, action.getMutation(), cellScanner, nonceGroup,
spaceQuotaEnforcement);
break;
case PUT:
case DELETE:
// Collect the individual mutations and apply in a batch
if (mutations == null) {
mutations = new ArrayList<>(actions.getActionCount());
}
mutations.add(action);
break;
default:
throw new DoNotRetryIOException("Unsupported mutate type: " + type.name());
}
} else {
throw new HBaseIOException("Unexpected Action type");
}
if (r != null) {
ClientProtos.Result pbResult = null;
if (isClientCellBlockSupport(context)) {
pbResult = ProtobufUtil.toResultNoData(r);
// Hard to guess the size here. Just make a rough guess.
if (cellsToReturn == null) {
cellsToReturn = new ArrayList<>();
}
cellsToReturn.add(r);
} else {
pbResult = ProtobufUtil.toResult(r);
}
lastBlock = addSize(context, r, lastBlock);
hasResultOrException = true;
resultOrExceptionBuilder.setResult(pbResult);
}
// Could get to here and there was no result and no exception. Presumes we added
// a Put or Delete to the collecting Mutations List for adding later. In this
// case the corresponding ResultOrException instance for the Put or Delete will be added
// down in the doNonAtomicBatchOp method call rather than up here.
} catch (IOException ie) {
rpcServer.getMetrics().exception(ie);
hasResultOrException = true;
NameBytesPair pair = ResponseConverter.buildException(ie);
resultOrExceptionBuilder.setException(pair);
context.incrementResponseExceptionSize(pair.getSerializedSize());
}
if (hasResultOrException) {
// Propagate index.
resultOrExceptionBuilder.setIndex(action.getIndex());
builder.addResultOrException(resultOrExceptionBuilder.build());
}
}
// Finish up any outstanding mutations
if (!CollectionUtils.isEmpty(mutations)) {
doNonAtomicBatchOp(builder, region, quota, mutations, cellScanner, spaceQuotaEnforcement);
}
return cellsToReturn;
}
private void checkCellSizeLimit(final HRegion r, final Mutation m) throws IOException {
if (r.maxCellSize > 0) {
CellScanner cells = m.cellScanner();
while (cells.advance()) {
int size = PrivateCellUtil.estimatedSerializedSizeOf(cells.current());
if (size > r.maxCellSize) {
String msg = "Cell[" + cells.current() + "] with size " + size + " exceeds limit of "
+ r.maxCellSize + " bytes";
LOG.debug(msg);
throw new DoNotRetryIOException(msg);
}
}
}
}
private void doAtomicBatchOp(final RegionActionResult.Builder builder, final HRegion region,
final OperationQuota quota, final List<ClientProtos.Action> mutations, final CellScanner cells,
long nonceGroup, ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException {
// Just throw the exception. The exception will be caught and then added to region-level
// exception for RegionAction. Leaving the null to action result is ok since the null
// result is viewed as failure by hbase client. And the region-lever exception will be used
// to replaced the null result. see AsyncRequestFutureImpl#receiveMultiAction and
// AsyncBatchRpcRetryingCaller#onComplete for more details.
doBatchOp(builder, region, quota, mutations, cells, nonceGroup, spaceQuotaEnforcement, true);
}
private void doNonAtomicBatchOp(final RegionActionResult.Builder builder, final HRegion region,
final OperationQuota quota, final List<ClientProtos.Action> mutations, final CellScanner cells,
ActivePolicyEnforcement spaceQuotaEnforcement) {
try {
doBatchOp(builder, region, quota, mutations, cells, HConstants.NO_NONCE,
spaceQuotaEnforcement, false);
} catch (IOException e) {
// Set the exception for each action. The mutations in same RegionAction are group to
// different batch and then be processed individually. Hence, we don't set the region-level
// exception here for whole RegionAction.
for (Action mutation : mutations) {
builder.addResultOrException(getResultOrException(e, mutation.getIndex()));
}
}
}
/**
* Execute a list of mutations. nnn
*/
private void doBatchOp(final RegionActionResult.Builder builder, final HRegion region,
final OperationQuota quota, final List<ClientProtos.Action> mutations, final CellScanner cells,
long nonceGroup, ActivePolicyEnforcement spaceQuotaEnforcement, boolean atomic)
throws IOException {
Mutation[] mArray = new Mutation[mutations.size()];
long before = EnvironmentEdgeManager.currentTime();
boolean batchContainsPuts = false, batchContainsDelete = false;
try {
/**
* HBASE-17924 mutationActionMap is a map to map the relation between mutations and actions
* since mutation array may have been reoredered.In order to return the right result or
* exception to the corresponding actions, We need to know which action is the mutation belong
* to. We can't sort ClientProtos.Action array, since they are bonded to cellscanners.
*/
Map<Mutation, ClientProtos.Action> mutationActionMap = new HashMap<>();
int i = 0;
long nonce = HConstants.NO_NONCE;
for (ClientProtos.Action action : mutations) {
if (action.hasGet()) {
throw new DoNotRetryIOException(
"Atomic put and/or delete only, not a Get=" + action.getGet());
}
MutationProto m = action.getMutation();
Mutation mutation;
switch (m.getMutateType()) {
case PUT:
mutation = ProtobufUtil.toPut(m, cells);
batchContainsPuts = true;
break;
case DELETE:
mutation = ProtobufUtil.toDelete(m, cells);
batchContainsDelete = true;
break;
case INCREMENT:
mutation = ProtobufUtil.toIncrement(m, cells);
nonce = m.hasNonce() ? m.getNonce() : HConstants.NO_NONCE;
break;
case APPEND:
mutation = ProtobufUtil.toAppend(m, cells);
nonce = m.hasNonce() ? m.getNonce() : HConstants.NO_NONCE;
break;
default:
throw new DoNotRetryIOException("Invalid mutation type : " + m.getMutateType());
}
mutationActionMap.put(mutation, action);
mArray[i++] = mutation;
checkCellSizeLimit(region, mutation);
// Check if a space quota disallows this mutation
spaceQuotaEnforcement.getPolicyEnforcement(region).check(mutation);
quota.addMutation(mutation);
}
if (!region.getRegionInfo().isMetaRegion()) {
regionServer.getMemStoreFlusher().reclaimMemStoreMemory();
}