HBASE-26921 Rewrite the counting cells part in TestMultiVersions (#4316)
[hbase.git] / hbase-client / src / main / java / org / apache / hadoop / hbase / client / RegionReplicaUtil.java
blob09150f123fec61a44fe873f5848bb23386ec5f23
1 /**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
19 package org.apache.hadoop.hbase.client;
21 import java.util.ArrayList;
22 import java.util.Collection;
23 import java.util.Iterator;
24 import java.util.List;
26 import org.apache.hadoop.hbase.util.Bytes;
27 import org.apache.yetus.audience.InterfaceAudience;
29 /**
30 * Utility methods which contain the logic for regions and replicas.
32 @InterfaceAudience.Private
33 public class RegionReplicaUtil {
35 /**
36 * Whether or not the secondary region will wait for observing a flush / region open event
37 * from the primary region via async wal replication before enabling read requests. Since replayed
38 * edits from async wal replication from primary is not persisted in WAL, the memstore of the
39 * secondary region might be non-empty at the time of close or crash. For ensuring seqId's not
40 * "going back in time" in the secondary region replica, this should be enabled. However, in some
41 * cases the above semantics might be ok for some application classes.
42 * See HBASE-11580 for more context.
44 public static final String REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY
45 = "hbase.region.replica.wait.for.primary.flush";
46 protected static final boolean DEFAULT_REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH = true;
48 /**
49 * The default replicaId for the region
51 static final int DEFAULT_REPLICA_ID = 0;
53 /**
54 * Returns the RegionInfo for the given replicaId.
55 * RegionInfo's correspond to a range of a table, but more than one
56 * "instance" of the same range can be deployed which are differentiated by
57 * the replicaId.
58 * @param regionInfo
59 * @param replicaId the replicaId to use
60 * @return an RegionInfo object corresponding to the same range (table, start and
61 * end key), but for the given replicaId.
63 public static RegionInfo getRegionInfoForReplica(RegionInfo regionInfo, int replicaId) {
64 if (regionInfo.getReplicaId() == replicaId) {
65 return regionInfo;
67 return RegionInfoBuilder.newBuilder(regionInfo).setReplicaId(replicaId).build();
70 /**
71 * Returns the RegionInfo for the default replicaId (0). RegionInfo's correspond to
72 * a range of a table, but more than one "instance" of the same range can be
73 * deployed which are differentiated by the replicaId.
74 * @return an RegionInfo object corresponding to the same range (table, start and
75 * end key), but for the default replicaId.
77 public static RegionInfo getRegionInfoForDefaultReplica(RegionInfo regionInfo) {
78 return getRegionInfoForReplica(regionInfo, DEFAULT_REPLICA_ID);
81 /** @return true if this replicaId corresponds to default replica for the region */
82 public static boolean isDefaultReplica(int replicaId) {
83 return DEFAULT_REPLICA_ID == replicaId;
86 /** @return true if this region is a default replica for the region */
87 public static boolean isDefaultReplica(RegionInfo hri) {
88 return hri.getReplicaId() == DEFAULT_REPLICA_ID;
91 /**
92 * Removes the non-default replicas from the passed regions collection
93 * @param regions
95 public static void removeNonDefaultRegions(Collection<RegionInfo> regions) {
96 Iterator<RegionInfo> iterator = regions.iterator();
97 while (iterator.hasNext()) {
98 RegionInfo hri = iterator.next();
99 if (!RegionReplicaUtil.isDefaultReplica(hri)) {
100 iterator.remove();
105 public static boolean isReplicasForSameRegion(RegionInfo regionInfoA, RegionInfo regionInfoB) {
106 return compareRegionInfosWithoutReplicaId(regionInfoA, regionInfoB) == 0;
109 private static int compareRegionInfosWithoutReplicaId(RegionInfo regionInfoA,
110 RegionInfo regionInfoB) {
111 int result = regionInfoA.getTable().compareTo(regionInfoB.getTable());
112 if (result != 0) {
113 return result;
116 // Compare start keys.
117 result = Bytes.compareTo(regionInfoA.getStartKey(), regionInfoB.getStartKey());
118 if (result != 0) {
119 return result;
122 // Compare end keys.
123 result = Bytes.compareTo(regionInfoA.getEndKey(), regionInfoB.getEndKey());
125 if (result != 0) {
126 if (regionInfoA.getStartKey().length != 0
127 && regionInfoA.getEndKey().length == 0) {
128 return 1; // this is last region
130 if (regionInfoB.getStartKey().length != 0
131 && regionInfoB.getEndKey().length == 0) {
132 return -1; // o is the last region
134 return result;
137 // regionId is usually milli timestamp -- this defines older stamps
138 // to be "smaller" than newer stamps in sort order.
139 if (regionInfoA.getRegionId() > regionInfoB.getRegionId()) {
140 return 1;
141 } else if (regionInfoA.getRegionId() < regionInfoB.getRegionId()) {
142 return -1;
144 return 0;
148 * Create any replicas for the regions (the default replicas that was already created is passed to
149 * the method)
150 * @param regions existing regions
151 * @param oldReplicaCount existing replica count
152 * @param newReplicaCount updated replica count due to modify table
153 * @return the combined list of default and non-default replicas
155 public static List<RegionInfo> addReplicas(final List<RegionInfo> regions, int oldReplicaCount,
156 int newReplicaCount) {
157 if ((newReplicaCount - 1) <= 0) {
158 return regions;
160 List<RegionInfo> hRegionInfos = new ArrayList<>((newReplicaCount) * regions.size());
161 for (RegionInfo ri : regions) {
162 if (RegionReplicaUtil.isDefaultReplica(ri) &&
163 (!ri.isOffline() || (!ri.isSplit() && !ri.isSplitParent()))) {
164 // region level replica index starts from 0. So if oldReplicaCount was 2 then the max replicaId for
165 // the existing regions would be 1
166 for (int j = oldReplicaCount; j < newReplicaCount; j++) {
167 hRegionInfos.add(RegionReplicaUtil.getRegionInfoForReplica(ri, j));
171 hRegionInfos.addAll(regions);
172 return hRegionInfos;