You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2016/12/02 06:41:31 UTC
[1/6] hbase git commit: HBASE-16119 Procedure v2 - Reimplement Merge
region (Stephen Yuan Jiang)
Repository: hbase
Updated Branches:
refs/heads/master 00b302435 -> 0a2407784
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 85b7967..07e16c8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -96,8 +96,8 @@ import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.master.MasterRpcServices;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse;
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;
@@ -1384,25 +1384,28 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
@Override
@QosPriority(priority=HConstants.ADMIN_QOS)
- public CloseRegionForSplitResponse closeRegionForSplit(
+ public CloseRegionForSplitOrMergeResponse closeRegionForSplitOrMerge(
final RpcController controller,
- final CloseRegionForSplitRequest request) throws ServiceException {
+ final CloseRegionForSplitOrMergeRequest request) throws ServiceException {
try {
checkOpen();
- final String encodedRegionName = ProtobufUtil.getRegionEncodedName(request.getRegion());
+ List<String> encodedRegionNameList = new ArrayList<>();
+ for(int i = 0; i < request.getRegionCount(); i++) {
+ final String encodedRegionName = ProtobufUtil.getRegionEncodedName(request.getRegion(i));
- // Can be null if we're calling close on a region that's not online
- final Region parentRegion = regionServer.getFromOnlineRegions(encodedRegionName);
- if ((parentRegion != null) && (parentRegion.getCoprocessorHost() != null)) {
- parentRegion.getCoprocessorHost().preClose(false);
+ // Can be null if we're calling close on a region that's not online
+ final Region targetRegion = regionServer.getFromOnlineRegions(encodedRegionName);
+ if ((targetRegion != null) && (targetRegion.getCoprocessorHost() != null)) {
+ targetRegion.getCoprocessorHost().preClose(false);
+ encodedRegionNameList.add(encodedRegionName);
+ }
}
-
requestCount.increment();
- LOG.info("Close and offline " + encodedRegionName + " and prepare for split.");
- boolean closed = regionServer.closeAndOfflineRegionForSplit(encodedRegionName);
- CloseRegionForSplitResponse.Builder builder =
- CloseRegionForSplitResponse.newBuilder().setClosed(closed);
+ LOG.info("Close and offline " + encodedRegionNameList + " regions.");
+ boolean closed = regionServer.closeAndOfflineRegionForSplitOrMerge(encodedRegionNameList);
+ CloseRegionForSplitOrMergeResponse.Builder builder =
+ CloseRegionForSplitOrMergeResponse.newBuilder().setClosed(closed);
return builder.build();
} catch (IOException ie) {
throw new ServiceException(ie);
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index 2af65a4..65cedda 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -30,7 +30,6 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
@@ -1180,10 +1179,10 @@ public class TestAdmin1 {
gotException = false;
// Try merging a replica with another. Should fail.
try {
- // TODO convert this to version that is synchronous (See HBASE-16668)
- TEST_UTIL.getAdmin().mergeRegionsAsync(regions.get(1).getFirst().getEncodedNameAsBytes(),
- regions.get(2).getFirst().getEncodedNameAsBytes(), true)
- .get(60, TimeUnit.SECONDS);
+ TEST_UTIL.getHBaseAdmin().mergeRegionsSync(
+ regions.get(1).getFirst().getEncodedNameAsBytes(),
+ regions.get(2).getFirst().getEncodedNameAsBytes(),
+ true);
} catch (IllegalArgumentException m) {
gotException = true;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 5497a3f..7522e85 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -175,6 +175,8 @@ public class TestMasterObserver {
private boolean preGetTableNamesCalled;
private boolean preDispatchMergeCalled;
private boolean postDispatchMergeCalled;
+ private boolean preMergeRegionsCalled;
+ private boolean postMergeRegionsCalled;
public void enableBypass(boolean bypass) {
this.bypass = bypass;
@@ -261,6 +263,26 @@ public class TestMasterObserver {
preGetTableNamesCalled = false;
preDispatchMergeCalled = false;
postDispatchMergeCalled = false;
+ preMergeRegionsCalled = false;
+ postMergeRegionsCalled = false;
+ }
+
+ @Override
+ public void preMergeRegions(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ preMergeRegionsCalled = true;
+ }
+
+ @Override
+ public void postMergeRegions(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ postMergeRegionsCalled = true;
+ }
+
+ public boolean wasMergeRegionsCalled() {
+ return preMergeRegionsCalled && postMergeRegionsCalled;
}
@Override
@@ -1512,9 +1534,42 @@ public class TestMasterObserver {
}
@Override
- public void preRollBackSplitRegionAction(
+ public void postRollBackSplitRegionAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
}
+
+ @Override
+ public void preMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ }
+
+ @Override
+ public void postCompletedMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> c,
+ final HRegionInfo[] regionsToMerge,
+ final HRegionInfo mergedRegion) throws IOException {
+ }
+
+ @Override
+ public void preMergeRegionsCommitAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge,
+ final List<Mutation> metaEntries) throws IOException {
+ }
+
+ @Override
+ public void postMergeRegionsCommitAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge,
+ final HRegionInfo mergedRegion) throws IOException {
+ }
+
+ @Override
+ public void postRollBackMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ }
}
private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
@@ -1593,7 +1648,7 @@ public class TestMasterObserver {
admin.mergeRegionsAsync(regions.get(0).getRegionInfo().getEncodedNameAsBytes(),
regions.get(1).getRegionInfo().getEncodedNameAsBytes(), true);
assertTrue("Coprocessor should have been called on region merge",
- cp.wasDispatchMergeCalled());
+ cp.wasMergeRegionsCalled());
tableCreationLatch = new CountDownLatch(1);
admin.disableTable(tableName);
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java
index ecf9da1..4a62bff 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@@ -61,6 +62,7 @@ public class TestRegionServerObserver {
* Test verifies the hooks in regions merge.
* @throws Exception
*/
+ @Ignore
@Test
public void testCoprocessorHooksInRegionsMerge() throws Exception {
final int NUM_MASTERS = 1;
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 2630068..b52f5df 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -287,6 +287,15 @@ public class MockNoopMasterServices implements MasterServices, Server {
}
@Override
+ public long mergeRegions(
+ final HRegionInfo[] regionsToMerge,
+ final boolean forcible,
+ final long nonceGroup,
+ final long nonce) throws IOException {
+ return -1;
+ }
+
+ @Override
public long splitRegion(
final HRegionInfo regionInfo,
final byte[] splitRow,
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 1594b6d..ec8054e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -51,8 +51,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
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.CloseRegionForSplitRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse;
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.FlushRegionRequest;
@@ -500,9 +500,9 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
}
@Override
- public CloseRegionForSplitResponse closeRegionForSplit(
+ public CloseRegionForSplitOrMergeResponse closeRegionForSplitOrMerge(
RpcController controller,
- CloseRegionForSplitRequest request) throws ServiceException {
+ CloseRegionForSplitOrMergeRequest request) throws ServiceException {
return null;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMergeTableRegionsProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMergeTableRegionsProcedure.java
new file mode 100644
index 0000000..1915f69
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMergeTableRegionsProcedure.java
@@ -0,0 +1,263 @@
+/**
+ * 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.master.procedure;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestMergeTableRegionsProcedure {
+ private static final Log LOG = LogFactory.getLog(TestMergeTableRegionsProcedure.class);
+
+ protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+ private static long nonceGroup = HConstants.NO_NONCE;
+ private static long nonce = HConstants.NO_NONCE;
+
+ private static final int initialRegionCount = 4;
+ private final static byte[] FAMILY = Bytes.toBytes("FAMILY");
+ final static Configuration conf = UTIL.getConfiguration();
+ private static Admin admin;
+
+ private static void setupConf(Configuration conf) {
+ // Reduce the maximum attempts to speed up the test
+ conf.setInt("hbase.assignment.maximum.attempts", 3);
+ conf.setInt("hbase.master.maximum.ping.server.attempts", 3);
+ conf.setInt("hbase.master.ping.server.retry.sleep.interval", 1);
+
+ conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+ }
+
+ @BeforeClass
+ public static void setupCluster() throws Exception {
+ setupConf(conf);
+ UTIL.startMiniCluster(1);
+ admin = UTIL.getHBaseAdmin();
+ }
+
+ @AfterClass
+ public static void cleanupTest() throws Exception {
+ try {
+ UTIL.shutdownMiniCluster();
+ } catch (Exception e) {
+ LOG.warn("failure shutting down cluster", e);
+ }
+ }
+
+ @Before
+ public void setup() throws Exception {
+ resetProcExecutorTestingKillFlag();
+ nonceGroup =
+ MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
+ nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
+ // Turn off balancer so it doesn't cut in and mess up our placements.
+ UTIL.getHBaseAdmin().setBalancerRunning(false, true);
+ // Turn off the meta scanner so it don't remove parent on us.
+ UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(false);
+ resetProcExecutorTestingKillFlag();
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ resetProcExecutorTestingKillFlag();
+ for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
+ LOG.info("Tear down, remove table=" + htd.getTableName());
+ UTIL.deleteTable(htd.getTableName());
+ }
+ }
+
+ private void resetProcExecutorTestingKillFlag() {
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+ assertTrue("expected executor to be running", procExec.isRunning());
+ }
+
+ /**
+ * This tests two region merges
+ */
+ @Test(timeout=60000)
+ public void testMergeTwoRegions() throws Exception {
+ final TableName tableName = TableName.valueOf("testMergeTwoRegions");
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ List<HRegionInfo> tableRegions = createTable(tableName);
+
+ HRegionInfo[] regionsToMerge = new HRegionInfo[2];
+ regionsToMerge[0] = tableRegions.get(0);
+ regionsToMerge[1] = tableRegions.get(1);
+
+ long procId = procExec.submitProcedure(new MergeTableRegionsProcedure(
+ procExec.getEnvironment(), regionsToMerge, true));
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+
+ assertRegionCount(tableName, initialRegionCount - 1);
+ }
+
+ /**
+ * This tests two concurrent region merges
+ */
+ @Test(timeout=60000)
+ public void testMergeRegionsConcurrently() throws Exception {
+ final TableName tableName = TableName.valueOf("testMergeRegionsConcurrently");
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ List<HRegionInfo> tableRegions = createTable(tableName);
+
+ HRegionInfo[] regionsToMerge1 = new HRegionInfo[2];
+ HRegionInfo[] regionsToMerge2 = new HRegionInfo[2];
+ regionsToMerge1[0] = tableRegions.get(0);
+ regionsToMerge1[1] = tableRegions.get(1);
+ regionsToMerge2[0] = tableRegions.get(2);
+ regionsToMerge2[1] = tableRegions.get(3);
+
+ long procId1 = procExec.submitProcedure(new MergeTableRegionsProcedure(
+ procExec.getEnvironment(), regionsToMerge1, true));
+ long procId2 = procExec.submitProcedure(new MergeTableRegionsProcedure(
+ procExec.getEnvironment(), regionsToMerge2, true));
+ ProcedureTestingUtility.waitProcedure(procExec, procId1);
+ ProcedureTestingUtility.waitProcedure(procExec, procId2);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
+ assertRegionCount(tableName, initialRegionCount - 2);
+ }
+
+ @Test(timeout=60000)
+ public void testMergeRegionsTwiceWithSameNonce() throws Exception {
+ final TableName tableName = TableName.valueOf("testMergeRegionsTwiceWithSameNonce");
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ List<HRegionInfo> tableRegions = createTable(tableName);
+
+ HRegionInfo[] regionsToMerge = new HRegionInfo[2];
+ regionsToMerge[0] = tableRegions.get(0);
+ regionsToMerge[1] = tableRegions.get(1);
+
+ long procId1 = procExec.submitProcedure(new MergeTableRegionsProcedure(
+ procExec.getEnvironment(), regionsToMerge, true), nonceGroup, nonce);
+ long procId2 = procExec.submitProcedure(new MergeTableRegionsProcedure(
+ procExec.getEnvironment(), regionsToMerge, true), nonceGroup, nonce);
+ assertEquals(procId1, procId2);
+
+ ProcedureTestingUtility.waitProcedure(procExec, procId1);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+ // The second proc should succeed too - because it is the same proc.
+ ProcedureTestingUtility.waitProcedure(procExec, procId2);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
+
+ assertRegionCount(tableName, initialRegionCount - 1);
+ }
+
+ @Test(timeout=60000)
+ public void testRecoveryAndDoubleExecution() throws Exception {
+ final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ List<HRegionInfo> tableRegions = createTable(tableName);
+
+ ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+ HRegionInfo[] regionsToMerge = new HRegionInfo[2];
+ regionsToMerge[0] = tableRegions.get(0);
+ regionsToMerge[1] = tableRegions.get(1);
+
+ long procId = procExec.submitProcedure(
+ new MergeTableRegionsProcedure(procExec.getEnvironment(), regionsToMerge, true));
+
+ // Restart the executor and execute the step twice
+ int numberOfSteps = MergeTableRegionsState.values().length;
+ MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+
+ assertRegionCount(tableName, initialRegionCount - 1);
+ }
+
+ @Test(timeout = 60000)
+ public void testRollbackAndDoubleExecution() throws Exception {
+ final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ List<HRegionInfo> tableRegions = createTable(tableName);
+
+ ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+ HRegionInfo[] regionsToMerge = new HRegionInfo[2];
+ regionsToMerge[0] = tableRegions.get(0);
+ regionsToMerge[1] = tableRegions.get(1);
+
+ long procId = procExec.submitProcedure(
+ new MergeTableRegionsProcedure(procExec.getEnvironment(), regionsToMerge, true));
+
+ // Failing before MERGE_TABLE_REGIONS_UPDATE_META we should trigger the rollback
+ // NOTE: the 6 (number before MERGE_TABLE_REGIONS_UPDATE_META step) is
+ // hardcoded, so you have to look at this test at least once when you add a new step.
+ int numberOfSteps = 6;
+ MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
+ }
+
+ private List<HRegionInfo> createTable(final TableName tableName)
+ throws Exception {
+ HTableDescriptor desc = new HTableDescriptor(tableName);
+ desc.addFamily(new HColumnDescriptor(FAMILY));
+ byte[][] splitRows = new byte[initialRegionCount - 1][];
+ for (int i = 0; i < splitRows.length; ++i) {
+ splitRows[i] = Bytes.toBytes(String.format("%d", i));
+ }
+ admin.createTable(desc, splitRows);
+ return assertRegionCount(tableName, initialRegionCount);
+ }
+
+ public List<HRegionInfo> assertRegionCount(final TableName tableName, final int nregions)
+ throws Exception {
+ UTIL.waitUntilNoRegionsInTransition();
+ List<HRegionInfo> tableRegions = admin.getTableRegions(tableName);
+ assertEquals(nregions, tableRegions.size());
+ return tableRegions;
+ }
+
+ private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+ return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSplitTableRegionProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSplitTableRegionProcedure.java
index fcce0fb..55e38ba 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSplitTableRegionProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSplitTableRegionProcedure.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.CompactionState;
@@ -311,8 +310,6 @@ public class TestSplitTableRegionProcedure {
HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
- int splitRowNum = startRowNum + rowCount / 2;
- byte[] splitKey = Bytes.toBytes("" + splitRowNum);
assertTrue("not able to find a splittable region", regions != null);
assertTrue("not able to find a splittable region", regions.length == 1);
@@ -330,7 +327,7 @@ public class TestSplitTableRegionProcedure {
}
}
- @Test(timeout = 600000)
+ @Test(timeout = 60000)
public void testRollbackAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index a25c157..f8a1a93 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -66,6 +66,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
import org.apache.hadoop.hbase.mapreduce.TableInputFormatBase;
import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.TableNamespaceManager;
@@ -73,9 +74,7 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.quotas.QuotaExceededException;
import org.apache.hadoop.hbase.quotas.QuotaUtil;
import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
@@ -104,11 +103,11 @@ public class TestNamespaceAuditor {
@BeforeClass
public static void before() throws Exception {
- UTIL.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
- CustomObserver.class.getName());
- UTIL.getConfiguration().set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
- MasterSyncObserver.class.getName());
Configuration conf = UTIL.getConfiguration();
+ conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, CustomObserver.class.getName());
+ conf.setStrings(
+ CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
+ MasterSyncObserver.class.getName(), CPMasterObserver.class.getName());
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
conf.setClass("hbase.coprocessor.regionserver.classes", CPRegionServerObserver.class,
@@ -309,6 +308,33 @@ public class TestNamespaceAuditor {
}
}
+ public static class CPMasterObserver extends BaseMasterObserver {
+ private volatile boolean shouldFailMerge = false;
+
+ public void failMerge(boolean fail) {
+ shouldFailMerge = fail;
+ }
+
+ private boolean triggered = false;
+
+ public synchronized void waitUtilTriggered() throws InterruptedException {
+ while (!triggered) {
+ wait();
+ }
+ }
+
+ @Override
+ public synchronized void preMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ triggered = true;
+ notifyAll();
+ if (shouldFailMerge) {
+ throw new IOException("fail merge");
+ }
+ }
+ }
+
@Test
public void testRegionMerge() throws Exception {
String nsp1 = prefix + "_regiontest";
@@ -414,18 +440,17 @@ public class TestNamespaceAuditor {
// fail region merge through Coprocessor hook
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
- HRegionServer regionServer = cluster.getRegionServer(0);
- RegionServerCoprocessorHost cpHost = regionServer.getRegionServerCoprocessorHost();
- Coprocessor coprocessor = cpHost.findCoprocessor(CPRegionServerObserver.class.getName());
- CPRegionServerObserver regionServerObserver = (CPRegionServerObserver) coprocessor;
- regionServerObserver.failMerge(true);
- regionServerObserver.triggered = false;
+ MasterCoprocessorHost cpHost = cluster.getMaster().getMasterCoprocessorHost();
+ Coprocessor coprocessor = cpHost.findCoprocessor(CPMasterObserver.class.getName());
+ CPMasterObserver masterObserver = (CPMasterObserver) coprocessor;
+ masterObserver.failMerge(true);
+ masterObserver.triggered = false;
ADMIN.mergeRegionsAsync(
hris.get(1).getEncodedNameAsBytes(),
hris.get(2).getEncodedNameAsBytes(),
false);
- regionServerObserver.waitUtilTriggered();
+ masterObserver.waitUtilTriggered();
hris = ADMIN.getTableRegions(tableTwo);
assertEquals(initialRegions, hris.size());
Collections.sort(hris);
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
index 2cb4ecc..6fcccaf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
@@ -55,12 +55,10 @@ import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
-import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
@Category({ ReplicationTests.class, LargeTests.class })
public class TestSerialReplication {
@@ -270,7 +268,7 @@ public class TestSerialReplication {
}
List<Pair<HRegionInfo, ServerName>> regions =
MetaTableAccessor.getTableRegionsAndLocations(utility1.getConnection(), tableName);
- utility1.getHBaseAdmin().mergeRegions(regions.get(0).getFirst().getRegionName(),
+ utility1.getHBaseAdmin().mergeRegionsAsync(regions.get(0).getFirst().getRegionName(),
regions.get(1).getFirst().getRegionName(), true);
waitTableHasRightNumberOfRegions(tableName, 1);
for (int i = 11; i < 100; i += 10) {
[2/6] hbase git commit: HBASE-16119 Procedure v2 - Reimplement Merge
region (Stephen Yuan Jiang)
Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-protocol-shaded/src/main/protobuf/Admin.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
index 47d39be..36221c24 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
@@ -119,14 +119,14 @@ message CloseRegionResponse {
}
/**
- * Closes the specified region and create
- * child region.
+ * Closes the specified region(s) for
+ * split or merge
*/
-message CloseRegionForSplitRequest {
- required RegionSpecifier region = 1;
+message CloseRegionForSplitOrMergeRequest {
+ repeated RegionSpecifier region = 1;
}
-message CloseRegionForSplitResponse {
+message CloseRegionForSplitOrMergeResponse {
required bool closed = 1;
}
@@ -295,8 +295,8 @@ service AdminService {
rpc CloseRegion(CloseRegionRequest)
returns(CloseRegionResponse);
- rpc CloseRegionForSplit(CloseRegionForSplitRequest)
- returns(CloseRegionForSplitResponse);
+ rpc CloseRegionForSplitOrMerge(CloseRegionForSplitOrMergeRequest)
+ returns(CloseRegionForSplitOrMergeResponse);
rpc FlushRegion(FlushRegionRequest)
returns(FlushRegionResponse);
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 9e6d1ed..b283ed9 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -93,6 +93,20 @@ message DispatchMergingRegionsResponse {
optional uint64 proc_id = 1;
}
+/**
+ * Merging the specified regions in a table.
+ */
+message MergeTableRegionsRequest {
+ repeated RegionSpecifier region = 1;
+ optional bool forcible = 3 [default = false];
+ optional uint64 nonce_group = 4 [default = 0];
+ optional uint64 nonce = 5 [default = 0];
+}
+
+message MergeTableRegionsResponse {
+ optional uint64 proc_id = 1;
+}
+
message AssignRegionRequest {
required RegionSpecifier region = 1;
}
@@ -593,6 +607,10 @@ service MasterService {
rpc DispatchMergingRegions(DispatchMergingRegionsRequest)
returns(DispatchMergingRegionsResponse);
+ /** Master merge the regions */
+ rpc MergeTableRegions(MergeTableRegionsRequest)
+ returns(MergeTableRegionsResponse);
+
/** Assign a region to a server chosen at random. */
rpc AssignRegion(AssignRegionRequest)
returns(AssignRegionResponse);
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index 8926605..23d914e 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -277,11 +277,32 @@ message DispatchMergingRegionsStateData {
optional bool forcible = 4;
}
+enum MergeTableRegionsState {
+ MERGE_TABLE_REGIONS_PREPARE = 1;
+ MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 2;
+ MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION = 3;
+ MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE = 4;
+ MERGE_TABLE_REGIONS_CLOSE_REGIONS = 5;
+ MERGE_TABLE_REGIONS_CREATE_MERGED_REGION = 6;
+ MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION = 7;
+ MERGE_TABLE_REGIONS_UPDATE_META = 8;
+ MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 9;
+ MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 10;
+ MERGE_TABLE_REGIONS_POST_OPERATION = 11;
+}
+
+message MergeTableRegionsStateData {
+ required UserInformation user_info = 1;
+ repeated RegionInfo region_info = 2;
+ required RegionInfo merged_region_info = 3;
+ optional bool forcible = 4 [default = false];
+}
+
enum SplitTableRegionState {
SPLIT_TABLE_REGION_PREPARE = 1;
SPLIT_TABLE_REGION_PRE_OPERATION = 2;
SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE = 3;
- SPLIT_TABLE_REGION_CLOSED_PARENT_REGION = 4;
+ SPLIT_TABLE_REGION_CLOSE_PARENT_REGION = 4;
SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS = 5;
SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR = 6;
SPLIT_TABLE_REGION_UPDATE_META = 7;
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index 70167bb..2065939 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -1030,6 +1030,18 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService
}
@Override
+ public void preMergeRegions(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ }
+
+ @Override
+ public void postMergeRegions(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ }
+
+ @Override
public void preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<HostAndPort>
servers, String targetGroup) throws IOException {
}
@@ -1133,7 +1145,40 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService
}
@Override
- public void preRollBackSplitRegionAction(
+ public void postRollBackSplitRegionAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
}
+
+ @Override
+ public void preMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ }
+
+ @Override
+ public void postCompletedMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> c,
+ final HRegionInfo[] regionsToMerge,
+ final HRegionInfo mergedRegion) throws IOException {
+ }
+
+ @Override
+ public void preMergeRegionsCommitAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge,
+ final List<Mutation> metaEntries) throws IOException {
+ }
+
+ @Override
+ public void postMergeRegionsCommitAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge,
+ final HRegionInfo mergedRegion) throws IOException {
+ }
+
+ @Override
+ public void postRollBackMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.java
index 21381e8..93b2085 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
-import org.apache.hadoop.hbase.regionserver.Region;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
@@ -480,6 +479,18 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver
}
@Override
+ public void preMergeRegions(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ }
+
+ @Override
+ public void postMergeRegions(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ }
+
+ @Override
public void preAbortProcedure(
ObserverContext<MasterCoprocessorEnvironment> ctx,
final ProcedureExecutor<MasterProcedureEnv> procEnv,
@@ -831,7 +842,40 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver
}
@Override
- public void preRollBackSplitRegionAction(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+ public void postRollBackSplitRegionAction(final ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
}
+
+ @Override
+ public void preMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ }
+
+ @Override
+ public void postCompletedMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> c,
+ final HRegionInfo[] regionsToMerge,
+ final HRegionInfo mergedRegion) throws IOException {
+ }
+
+ @Override
+ public void preMergeRegionsCommitAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge,
+ final List<Mutation> metaEntries) throws IOException {
+ }
+
+ @Override
+ public void postMergeRegionsCommitAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge,
+ final HRegionInfo mergedRegion) throws IOException {
+ }
+
+ @Override
+ public void postRollBackMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
index 4d24a84..23afe4b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
-import org.apache.hadoop.hbase.regionserver.Region;
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.CONFIG})
@InterfaceStability.Evolving
@@ -755,6 +754,18 @@ public class BaseMasterObserver implements MasterObserver {
}
@Override
+ public void preMergeRegions(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ }
+
+ @Override
+ public void postMergeRegions(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ }
+
+ @Override
public void preAbortProcedure(
ObserverContext<MasterCoprocessorEnvironment> ctx,
final ProcedureExecutor<MasterProcedureEnv> procEnv,
@@ -852,11 +863,44 @@ public class BaseMasterObserver implements MasterObserver {
}
@Override
- public void preRollBackSplitRegionAction(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+ public void postRollBackSplitRegionAction(final ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
}
@Override
+ public void preMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ }
+
+ @Override
+ public void postCompletedMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> c,
+ final HRegionInfo[] regionsToMerge,
+ final HRegionInfo mergedRegion) throws IOException {
+ }
+
+ @Override
+ public void preMergeRegionsCommitAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge,
+ final List<Mutation> metaEntries) throws IOException {
+ }
+
+ @Override
+ public void postMergeRegionsCommitAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge,
+ final HRegionInfo mergedRegion) throws IOException {
+ }
+
+ @Override
+ public void postRollBackMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException {
+ }
+
+ @Override
public void preBalance(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index e90f753..9abcd52 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaMutationAnnotation;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerName;
@@ -41,6 +42,7 @@ import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
@@ -1196,14 +1198,74 @@ public interface MasterObserver extends Coprocessor {
throws IOException;
/**
- * This will be called before the roll back of the split region is completed
+ * This will be called after the roll back of the split region is completed
* @param ctx the environment to interact with the framework and master
* @throws IOException
*/
- void preRollBackSplitRegionAction(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+ void postRollBackSplitRegionAction(final ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException;
/**
+ * Called before the regions merge.
+ * Call {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} to skip the merge.
+ * @throws IOException if an error occurred on the coprocessor
+ * @param ctx
+ * @param regionsToMerge
+ * @throws IOException
+ */
+ void preMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException;
+
+ /**
+ * called after the regions merge.
+ * @param c
+ * @param regionsToMerge
+ * @param mergedRegion
+ * @throws IOException
+ */
+ void postCompletedMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> c,
+ final HRegionInfo[] regionsToMerge,
+ final HRegionInfo mergedRegion) throws IOException;
+
+ /**
+ * This will be called before PONR step as part of regions merge transaction. Calling
+ * {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} rollback the merge
+ * @param ctx
+ * @param regionsToMerge
+ * @param metaEntries mutations to execute on hbase:meta atomically with regions merge updates.
+ * Any puts or deletes to execute on hbase:meta can be added to the mutations.
+ * @throws IOException
+ */
+ void preMergeRegionsCommitAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge,
+ @MetaMutationAnnotation List<Mutation> metaEntries) throws IOException;
+
+ /**
+ * This will be called after PONR step as part of regions merge transaction.
+ * @param ctx
+ * @param regionsToMerge
+ * @param mergedRegion
+ * @throws IOException
+ */
+ void postMergeRegionsCommitAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge,
+ final HRegionInfo mergedRegion) throws IOException;
+
+ /**
+ * This will be called after the roll back of the regions merge.
+ * @param ctx
+ * @param regionsToMerge
+ * @throws IOException
+ */
+ void postRollBackMergeRegionsAction(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException;
+
+ /**
* Called prior to modifying the flag used to enable/disable region balancing.
* @param ctx the coprocessor instance's environment
* @param newValue the new flag value submitted in the call
@@ -1651,6 +1713,27 @@ public interface MasterObserver extends Coprocessor {
final HRegionInfo regionA, final HRegionInfo regionB) throws IOException;
/**
+ * Called before merge regions request.
+ * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
+ * @param ctx coprocessor environment
+ * @param regionsToMerge regions to be merged
+ * @throws IOException if an error occurred on the coprocessor
+ */
+ void preMergeRegions(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ final HRegionInfo[] regionsToMerge) throws IOException;
+
+ /**
+ * called after merge regions request.
+ * @param c coprocessor environment
+ * @param regionsToMerge regions to be merged
+ * @throws IOException if an error occurred on the coprocessor
+ */
+ void postMergeRegions(
+ final ObserverContext<MasterCoprocessorEnvironment> c,
+ final HRegionInfo[] regionsToMerge) throws IOException;
+
+ /**
* Called before servers are moved to target region server group
* @param ctx the environment to interact with the framework and master
* @param servers set of servers to move
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index 3540b19..a8061a1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -2630,6 +2630,39 @@ public class AssignmentManager {
return null;
}
+ public void assignMergedRegion(
+ final HRegionInfo mergedRegion,
+ final HRegionInfo daughterAHRI,
+ final HRegionInfo daughterBHRI) throws InterruptedException, IOException {
+ //Offline the daughter regions
+ regionOffline(daughterAHRI, State.MERGED);
+ regionOffline(daughterBHRI, State.MERGED);
+
+ //Set merged region to offline
+ regionStates.prepareAssignMergedRegion(mergedRegion);
+
+ // Assign merged region
+ invokeAssign(mergedRegion);
+
+ Callable<Object> mergeReplicasCallable = new Callable<Object>() {
+ @Override
+ public Object call() {
+ doMergingOfReplicas(mergedRegion, daughterAHRI, daughterBHRI);
+ return null;
+ }
+ };
+ threadPoolExecutorService.submit(mergeReplicasCallable);
+
+ // wait for assignment completion
+ ArrayList<HRegionInfo> regionAssignSet = new ArrayList<HRegionInfo>(1);
+ regionAssignSet.add(mergedRegion);
+ while (!waitForAssignment(regionAssignSet, true, regionAssignSet.size(), Long.MAX_VALUE)) {
+ LOG.debug("The merged region " + mergedRegion + " is still in transition. ");
+ }
+
+ regionStateListener.onRegionMerged(mergedRegion);
+ }
+
private String onRegionMerged(final RegionState current, final HRegionInfo hri,
final ServerName serverName, final RegionStateTransition transition) {
// The region must be in merging_new state, and the daughters must be
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 5f2e2a6..710c48a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -109,6 +109,7 @@ import org.apache.hadoop.hbase.master.procedure.DispatchMergingRegionsProcedure;
import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.MergeTableRegionsProcedure;
import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
@@ -1420,6 +1421,50 @@ public class HMaster extends HRegionServer implements MasterServices {
}
@Override
+ public long mergeRegions(
+ final HRegionInfo[] regionsToMerge,
+ final boolean forcible,
+ final long nonceGroup,
+ final long nonce) throws IOException {
+ checkInitialized();
+
+ assert(regionsToMerge.length == 2);
+
+ TableName tableName = regionsToMerge[0].getTable();
+ if (tableName == null || regionsToMerge[1].getTable() == null) {
+ throw new UnknownRegionException ("Can't merge regions without table associated");
+ }
+
+ if (!tableName.equals(regionsToMerge[1].getTable())) {
+ throw new IOException (
+ "Cannot merge regions from two different tables " + regionsToMerge[0].getTable()
+ + " and " + regionsToMerge[1].getTable());
+ }
+
+ if (regionsToMerge[0].compareTo(regionsToMerge[1]) == 0) {
+ throw new MergeRegionException(
+ "Cannot merge a region to itself " + regionsToMerge[0] + ", " + regionsToMerge[1]);
+ }
+
+ if (cpHost != null) {
+ cpHost.preMergeRegions(regionsToMerge);
+ }
+
+ LOG.info(getClientIdAuditPrefix() + " Merge regions "
+ + regionsToMerge[0].getEncodedName() + " and " + regionsToMerge[1].getEncodedName());
+
+ long procId = this.procedureExecutor.submitProcedure(
+ new MergeTableRegionsProcedure(procedureExecutor.getEnvironment(), regionsToMerge, forcible),
+ nonceGroup,
+ nonce);
+
+ if (cpHost != null) {
+ cpHost.postMergeRegions(regionsToMerge);
+ }
+ return procId;
+ }
+
+ @Override
public long splitRegion(
final HRegionInfo regionInfo,
final byte[] splitRow,
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index d0ac765..a18068d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaMutationAnnotation;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerName;
@@ -792,6 +793,28 @@ public class MasterCoprocessorHost
});
}
+ public void preMergeRegions(final HRegionInfo[] regionsToMerge)
+ throws IOException {
+ execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+ @Override
+ public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
+ throws IOException {
+ oserver.preMergeRegions(ctx, regionsToMerge);
+ }
+ });
+ }
+
+ public void postMergeRegions(final HRegionInfo[] regionsToMerge)
+ throws IOException {
+ execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+ @Override
+ public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
+ throws IOException {
+ oserver.postMergeRegions(ctx, regionsToMerge);
+ }
+ });
+ }
+
public boolean preBalance() throws IOException {
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
@@ -928,16 +951,110 @@ public class MasterCoprocessorHost
}
/**
- * Invoked just before the rollback of a failed split is started
+ * Invoked just after the rollback of a failed split
* @param user the user
* @throws IOException
*/
- public void preRollBackSplitAction(final User user) throws IOException {
+ public void postRollBackSplitRegionAction(final User user) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
- oserver.preRollBackSplitRegionAction(ctx);
+ oserver.postRollBackSplitRegionAction(ctx);
+ }
+ });
+ }
+
+ /**
+ * Invoked just before a merge
+ * @param regionsToMerge the regions to merge
+ * @param user the user
+ * @throws IOException
+ */
+ public boolean preMergeRegionsAction(
+ final HRegionInfo[] regionsToMerge, final User user) throws IOException {
+ return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+ @Override
+ public void call(MasterObserver oserver,
+ ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+ oserver.preMergeRegionsAction(ctx, regionsToMerge);
+ }
+ });
+ }
+
+ /**
+ * Invoked after completing merge regions operation
+ * @param regionsToMerge the regions to merge
+ * @param mergedRegion the new merged region
+ * @param user the user
+ * @throws IOException
+ */
+ public void postCompletedMergeRegionsAction(
+ final HRegionInfo[] regionsToMerge,
+ final HRegionInfo mergedRegion,
+ final User user) throws IOException {
+ execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+ @Override
+ public void call(MasterObserver oserver,
+ ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+ oserver.postCompletedMergeRegionsAction(ctx, regionsToMerge, mergedRegion);
+ }
+ });
+ }
+
+ /**
+ * Invoked before merge regions operation writes the new region to hbase:meta
+ * @param regionsToMerge the regions to merge
+ * @param metaEntries the meta entry
+ * @param user the user
+ * @throws IOException
+ */
+ public boolean preMergeRegionsCommit(
+ final HRegionInfo[] regionsToMerge,
+ final @MetaMutationAnnotation List<Mutation> metaEntries,
+ final User user) throws IOException {
+ return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+ @Override
+ public void call(MasterObserver oserver,
+ ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+ oserver.preMergeRegionsCommitAction(ctx, regionsToMerge, metaEntries);
+ }
+ });
+ }
+
+ /**
+ * Invoked after merge regions operation writes the new region to hbase:meta
+ * @param regionsToMerge the regions to merge
+ * @param mergedRegion the new merged region
+ * @param user the user
+ * @throws IOException
+ */
+ public void postMergeRegionsCommit(
+ final HRegionInfo[] regionsToMerge,
+ final HRegionInfo mergedRegion,
+ final User user) throws IOException {
+ execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+ @Override
+ public void call(MasterObserver oserver,
+ ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+ oserver.postMergeRegionsCommitAction(ctx, regionsToMerge, mergedRegion);
+ }
+ });
+ }
+
+ /**
+ * Invoked after rollback merge regions operation
+ * @param regionsToMerge the regions to merge
+ * @param user the user
+ * @throws IOException
+ */
+ public void postRollBackMergeRegionsAction(
+ final HRegionInfo[] regionsToMerge, final User user) throws IOException {
+ execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
+ @Override
+ public void call(MasterObserver oserver,
+ ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+ oserver.postRollBackMergeRegionsAction(ctx, regionsToMerge);
}
});
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 97eb209..709b3f2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -553,6 +553,46 @@ public class MasterRpcServices extends RSRpcServices
}
@Override
+ public MergeTableRegionsResponse mergeTableRegions(
+ RpcController c, MergeTableRegionsRequest request) throws ServiceException {
+ try {
+ master.checkInitialized();
+ } catch (IOException ioe) {
+ throw new ServiceException(ioe);
+ }
+
+ RegionStates regionStates = master.getAssignmentManager().getRegionStates();
+
+ assert(request.getRegionCount() == 2);
+ HRegionInfo[] regionsToMerge = new HRegionInfo[request.getRegionCount()];
+ for (int i = 0; i < request.getRegionCount(); i++) {
+ final byte[] encodedNameOfRegion = request.getRegion(i).getValue().toByteArray();
+ if (request.getRegion(i).getType() != RegionSpecifierType.ENCODED_REGION_NAME) {
+ LOG.warn("MergeRegions specifier type: expected: "
+ + RegionSpecifierType.ENCODED_REGION_NAME + " actual: region " + i + " ="
+ + request.getRegion(i).getType());
+ }
+ RegionState regionState = regionStates.getRegionState(Bytes.toString(encodedNameOfRegion));
+ if (regionState == null) {
+ throw new ServiceException(
+ new UnknownRegionException(Bytes.toStringBinary(encodedNameOfRegion)));
+ }
+ regionsToMerge[i] = regionState.getRegion();
+ }
+
+ try {
+ long procId = master.mergeRegions(
+ regionsToMerge,
+ request.getForcible(),
+ request.getNonceGroup(),
+ request.getNonce());
+ return MergeTableRegionsResponse.newBuilder().setProcId(procId).build();
+ } catch (IOException ioe) {
+ throw new ServiceException(ioe);
+ }
+ }
+
+ @Override
public SplitTableRegionResponse splitRegion(
final RpcController controller,
final SplitTableRegionRequest request) throws ServiceException {
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index fa1c33d..a4c27f3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -265,6 +265,21 @@ public interface MasterServices extends Server {
throws IOException;
/**
+ * Merge regions in a table.
+ * @param regionsToMerge daughter regions to merge
+ * @param forcible whether to force to merge even two regions are not adjacent
+ * @param nonceGroup used to detect duplicate
+ * @param nonce used to detect duplicate
+ * @return procedure Id
+ * @throws IOException
+ */
+ long mergeRegions(
+ final HRegionInfo[] regionsToMerge,
+ final boolean forcible,
+ final long nonceGroup,
+ final long nonce) throws IOException;
+
+ /**
* Split a region.
* @param regionInfo region to split
* @param splitRow split point
@@ -273,7 +288,7 @@ public interface MasterServices extends Server {
* @return procedure Id
* @throws IOException
*/
- public long splitRegion(
+ long splitRegion(
final HRegionInfo regionInfo,
final byte [] splitRow,
final long nonceGroup,
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
index b199374..7c2df61 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
@@ -896,6 +896,14 @@ public class RegionStates {
}
}
+ public void prepareAssignMergedRegion(HRegionInfo mergedRegion) {
+ synchronized (this) {
+ if (isRegionInState(mergedRegion, State.MERGING_NEW)) {
+ updateRegionState(mergedRegion, State.OFFLINE, null);
+ }
+ }
+ }
+
void splitRegion(HRegionInfo p,
HRegionInfo a, HRegionInfo b, ServerName sn) throws IOException {
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index a567e1d..b76cd7e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -823,23 +823,22 @@ public class ServerManager {
* A region server could reject the close request because it either does not
* have the specified region or the region is being split.
* @param server server to close a region
- * @param regionToClose the info of the region to close
+ * @param regionToClose the info of the region(s) to close
* @throws IOException
*/
- public boolean sendRegionCloseForSplit(
+ public boolean sendRegionCloseForSplitOrMerge(
final ServerName server,
- final HRegionInfo regionToClose) throws IOException {
+ final HRegionInfo... regionToClose) throws IOException {
if (server == null) {
throw new NullPointerException("Passed server is null");
}
AdminService.BlockingInterface admin = getRsAdmin(server);
if (admin == null) {
- throw new IOException("Attempting to send CLOSE For Split RPC to server " +
- server.toString() + " for region " + regionToClose.getRegionNameAsString() +
- " failed because no RPC connection found to this server");
+ throw new IOException("Attempting to send CLOSE For Split or Merge RPC to server " +
+ server.toString() + " failed because no RPC connection found to this server.");
}
HBaseRpcController controller = newRpcController();
- return ProtobufUtil.closeRegionForSplit(controller, admin, server, regionToClose);
+ return ProtobufUtil.closeRegionForSplitOrMerge(controller, admin, server, regionToClose);
}
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
new file mode 100644
index 0000000..c313700
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
@@ -0,0 +1,907 @@
+/**
+ * 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.master.procedure;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaMutationAnnotation;
+import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.UnknownRegionException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.exceptions.MergeRegionException;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.CatalogJanitor;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.RegionStates;
+import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+/**
+ * The procedure to Merge a region in a table.
+ */
+@InterfaceAudience.Private
+public class MergeTableRegionsProcedure
+ extends AbstractStateMachineTableProcedure<MergeTableRegionsState> {
+ private static final Log LOG = LogFactory.getLog(MergeTableRegionsProcedure.class);
+
+ private Boolean traceEnabled;
+ private AssignmentManager assignmentManager;
+ private int timeout;
+ private ServerName regionLocation;
+ private String regionsToMergeListFullName;
+ private String regionsToMergeListEncodedName;
+
+ private HRegionInfo [] regionsToMerge;
+ private HRegionInfo mergedRegionInfo;
+ private boolean forcible;
+
+ public MergeTableRegionsProcedure() {
+ this.traceEnabled = isTraceEnabled();
+ this.assignmentManager = null;
+ this.timeout = -1;
+ this.regionLocation = null;
+ this.regionsToMergeListFullName = null;
+ this.regionsToMergeListEncodedName = null;
+ }
+
+ public MergeTableRegionsProcedure(
+ final MasterProcedureEnv env,
+ final HRegionInfo[] regionsToMerge,
+ final boolean forcible) throws IOException {
+ super(env);
+ this.traceEnabled = isTraceEnabled();
+ this.assignmentManager = getAssignmentManager(env);
+ // For now, we only merge 2 regions. It could be extended to more than 2 regions in
+ // the future.
+ assert(regionsToMerge.length == 2);
+ assert(regionsToMerge[0].getTable() == regionsToMerge[1].getTable());
+ this.regionsToMerge = regionsToMerge;
+ this.forcible = forcible;
+
+ this.timeout = -1;
+ this.regionsToMergeListFullName = getRegionsToMergeListFullNameString();
+ this.regionsToMergeListEncodedName = getRegionsToMergeListEncodedNameString();
+
+ // Check daughter regions and make sure that we have valid daughter regions before
+ // doing the real work.
+ checkDaughterRegions();
+ // WARN: make sure there is no parent region of the two merging regions in
+ // hbase:meta If exists, fixing up daughters would cause daughter regions(we
+ // have merged one) online again when we restart master, so we should clear
+ // the parent region to prevent the above case
+ // Since HBASE-7721, we don't need fix up daughters any more. so here do
+ // nothing
+ setupMergedRegionInfo();
+ }
+
+ @Override
+ protected Flow executeFromState(
+ final MasterProcedureEnv env,
+ final MergeTableRegionsState state) throws InterruptedException {
+ if (isTraceEnabled()) {
+ LOG.trace(this + " execute state=" + state);
+ }
+
+ try {
+ switch (state) {
+ case MERGE_TABLE_REGIONS_PREPARE:
+ prepareMergeRegion(env);
+ setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS);
+ break;
+ case MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS:
+ if (MoveRegionsToSameRS(env)) {
+ setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION);
+ } else {
+ LOG.info("Cancel merging regions " + getRegionsToMergeListFullNameString()
+ + ", because can't move them to the same RS");
+ setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_POST_OPERATION);
+ }
+ break;
+ case MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION:
+ preMergeRegions(env);
+ setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE);
+ break;
+ case MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE:
+ setRegionStateToMerging(env);
+ setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_CLOSE_REGIONS);
+ break;
+ case MERGE_TABLE_REGIONS_CLOSE_REGIONS:
+ closeRegionsForMerge(env);
+ setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_CREATE_MERGED_REGION);
+ break;
+ case MERGE_TABLE_REGIONS_CREATE_MERGED_REGION:
+ createMergedRegion(env);
+ setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION);
+ break;
+ case MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION:
+ preMergeRegionsCommit(env);
+ setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_UPDATE_META);
+ break;
+ case MERGE_TABLE_REGIONS_UPDATE_META:
+ updateMetaForMergedRegions(env);
+ setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION);
+ break;
+ case MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION:
+ postMergeRegionsCommit(env);
+ setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_OPEN_MERGED_REGION);
+ break;
+ case MERGE_TABLE_REGIONS_OPEN_MERGED_REGION:
+ openMergedRegions(env);
+ setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_POST_OPERATION);
+ break;
+ case MERGE_TABLE_REGIONS_POST_OPERATION:
+ postCompletedMergeRegions(env);
+ return Flow.NO_MORE_STATE;
+ default:
+ throw new UnsupportedOperationException(this + " unhandled state=" + state);
+ }
+ } catch (IOException e) {
+ LOG.warn("Error trying to merge regions " + getRegionsToMergeListFullNameString() +
+ " in the table " + getTableName() + " (in state=" + state + ")", e);
+
+ setFailure("master-merge-regions", e);
+ }
+ return Flow.HAS_MORE_STATE;
+ }
+
+ @Override
+ protected void rollbackState(
+ final MasterProcedureEnv env,
+ final MergeTableRegionsState state) throws IOException, InterruptedException {
+ if (isTraceEnabled()) {
+ LOG.trace(this + " rollback state=" + state);
+ }
+
+ try {
+ switch (state) {
+ case MERGE_TABLE_REGIONS_POST_OPERATION:
+ case MERGE_TABLE_REGIONS_OPEN_MERGED_REGION:
+ case MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION:
+ case MERGE_TABLE_REGIONS_UPDATE_META:
+ String msg = this + " We are in the " + state + " state."
+ + " It is complicated to rollback the merge operation that region server is working on."
+ + " Rollback is not supported and we should let the merge operation to complete";
+ LOG.warn(msg);
+ // PONR
+ throw new UnsupportedOperationException(this + " unhandled state=" + state);
+ case MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION:
+ break;
+ case MERGE_TABLE_REGIONS_CREATE_MERGED_REGION:
+ cleanupMergedRegion(env);
+ break;
+ case MERGE_TABLE_REGIONS_CLOSE_REGIONS:
+ rollbackCloseRegionsForMerge(env);
+ break;
+ case MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE:
+ setRegionStateToRevertMerging(env);
+ break;
+ case MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION:
+ postRollBackMergeRegions(env);
+ break;
+ case MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS:
+ break; // nothing to rollback
+ case MERGE_TABLE_REGIONS_PREPARE:
+ break; // nothing to rollback
+ default:
+ throw new UnsupportedOperationException(this + " unhandled state=" + state);
+ }
+ } catch (Exception e) {
+ // This will be retried. Unless there is a bug in the code,
+ // this should be just a "temporary error" (e.g. network down)
+ LOG.warn("Failed rollback attempt step " + state + " for merging the regions "
+ + getRegionsToMergeListFullNameString() + " in table " + getTableName(), e);
+ throw e;
+ }
+ }
+
+ /*
+ * Check whether we are in the state that can be rollback
+ */
+ @Override
+ protected boolean isRollbackSupported(final MergeTableRegionsState state) {
+ switch (state) {
+ case MERGE_TABLE_REGIONS_POST_OPERATION:
+ case MERGE_TABLE_REGIONS_OPEN_MERGED_REGION:
+ case MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION:
+ case MERGE_TABLE_REGIONS_UPDATE_META:
+ // It is not safe to rollback if we reach to these states.
+ return false;
+ default:
+ break;
+ }
+ return true;
+ }
+
+ @Override
+ protected MergeTableRegionsState getState(final int stateId) {
+ return MergeTableRegionsState.valueOf(stateId);
+ }
+
+ @Override
+ protected int getStateId(final MergeTableRegionsState state) {
+ return state.getNumber();
+ }
+
+ @Override
+ protected MergeTableRegionsState getInitialState() {
+ return MergeTableRegionsState.MERGE_TABLE_REGIONS_PREPARE;
+ }
+
+ @Override
+ public void serializeStateData(final OutputStream stream) throws IOException {
+ super.serializeStateData(stream);
+
+ MasterProcedureProtos.MergeTableRegionsStateData.Builder mergeTableRegionsMsg =
+ MasterProcedureProtos.MergeTableRegionsStateData.newBuilder()
+ .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
+ .setMergedRegionInfo(HRegionInfo.convert(mergedRegionInfo))
+ .setForcible(forcible);
+ for (HRegionInfo hri: regionsToMerge) {
+ mergeTableRegionsMsg.addRegionInfo(HRegionInfo.convert(hri));
+ }
+ mergeTableRegionsMsg.build().writeDelimitedTo(stream);
+ }
+
+ @Override
+ public void deserializeStateData(final InputStream stream) throws IOException {
+ super.deserializeStateData(stream);
+
+ MasterProcedureProtos.MergeTableRegionsStateData mergeTableRegionsMsg =
+ MasterProcedureProtos.MergeTableRegionsStateData.parseDelimitedFrom(stream);
+ setUser(MasterProcedureUtil.toUserInfo(mergeTableRegionsMsg.getUserInfo()));
+
+ assert(mergeTableRegionsMsg.getRegionInfoCount() == 2);
+ regionsToMerge = new HRegionInfo[mergeTableRegionsMsg.getRegionInfoCount()];
+ for (int i = 0; i < regionsToMerge.length; i++) {
+ regionsToMerge[i] = HRegionInfo.convert(mergeTableRegionsMsg.getRegionInfo(i));
+ }
+
+ mergedRegionInfo = HRegionInfo.convert(mergeTableRegionsMsg.getMergedRegionInfo());
+ }
+
+ @Override
+ public void toStringClassDetails(StringBuilder sb) {
+ sb.append(getClass().getSimpleName());
+ sb.append(" (table=");
+ sb.append(getTableName());
+ sb.append(" regions=");
+ sb.append(getRegionsToMergeListFullNameString());
+ sb.append(" forcible=");
+ sb.append(forcible);
+ sb.append(")");
+ }
+
+ @Override
+ protected boolean acquireLock(final MasterProcedureEnv env) {
+ if (env.waitInitialized(this)) {
+ return false;
+ }
+ return !env.getProcedureQueue().waitRegions(
+ this, getTableName(), regionsToMerge[0], regionsToMerge[1]);
+ }
+
+ @Override
+ protected void releaseLock(final MasterProcedureEnv env) {
+ env.getProcedureQueue().wakeRegions(this, getTableName(), regionsToMerge[0], regionsToMerge[1]);
+ }
+
+ @Override
+ public TableName getTableName() {
+ return regionsToMerge[0].getTable();
+ }
+
+ @Override
+ public TableOperationType getTableOperationType() {
+ return TableOperationType.MERGE;
+ }
+
+ /**
+ * check daughter regions
+ * @throws IOException
+ */
+ private void checkDaughterRegions() throws IOException {
+ // Note: the following logic assumes that we only have 2 regions to merge. In the future,
+ // if we want to extend to more than 2 regions, the code needs to modify a little bit.
+ //
+ if (regionsToMerge[0].getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID ||
+ regionsToMerge[1].getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+ throw new MergeRegionException("Can't merge non-default replicas");
+ }
+
+ if (!HRegionInfo.areAdjacent(regionsToMerge[0], regionsToMerge[1])) {
+ String msg = "Trying to merge non-adjacent regions "
+ + getRegionsToMergeListFullNameString() + " where forcible = " + forcible;
+ LOG.warn(msg);
+ if (!forcible) {
+ throw new DoNotRetryIOException(msg);
+ }
+ }
+ }
+
+ /**
+ * Prepare merge and do some check
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void prepareMergeRegion(final MasterProcedureEnv env) throws IOException {
+ // Note: the following logic assumes that we only have 2 regions to merge. In the future,
+ // if we want to extend to more than 2 regions, the code needs to modify a little bit.
+ //
+ CatalogJanitor catalogJanitor = env.getMasterServices().getCatalogJanitor();
+ boolean regionAHasMergeQualifier = !catalogJanitor.cleanMergeQualifier(regionsToMerge[0]);
+ if (regionAHasMergeQualifier
+ || !catalogJanitor.cleanMergeQualifier(regionsToMerge[1])) {
+ String msg = "Skip merging regions " + getRegionsToMergeListFullNameString()
+ + ", because region "
+ + (regionAHasMergeQualifier ? regionsToMerge[0].getEncodedName() : regionsToMerge[1]
+ .getEncodedName()) + " has merge qualifier";
+ LOG.warn(msg);
+ throw new MergeRegionException(msg);
+ }
+
+ RegionStates regionStates = getAssignmentManager(env).getRegionStates();
+ RegionState regionStateA = regionStates.getRegionState(regionsToMerge[0].getEncodedName());
+ RegionState regionStateB = regionStates.getRegionState(regionsToMerge[1].getEncodedName());
+ if (regionStateA == null || regionStateB == null) {
+ throw new UnknownRegionException(
+ regionStateA == null ?
+ regionsToMerge[0].getEncodedName() : regionsToMerge[1].getEncodedName());
+ }
+
+ if (!regionStateA.isOpened() || !regionStateB.isOpened()) {
+ throw new MergeRegionException(
+ "Unable to merge regions not online " + regionStateA + ", " + regionStateB);
+ }
+ }
+
+ /**
+ * Create merged region info through the specified two regions
+ */
+ private void setupMergedRegionInfo() {
+ long rid = EnvironmentEdgeManager.currentTime();
+ // Regionid is timestamp. Merged region's id can't be less than that of
+ // merging regions else will insert at wrong location in hbase:meta
+ if (rid < regionsToMerge[0].getRegionId() || rid < regionsToMerge[1].getRegionId()) {
+ LOG.warn("Clock skew; merging regions id are " + regionsToMerge[0].getRegionId()
+ + " and " + regionsToMerge[1].getRegionId() + ", but current time here is " + rid);
+ rid = Math.max(regionsToMerge[0].getRegionId(), regionsToMerge[1].getRegionId()) + 1;
+ }
+
+ byte[] startKey = null;
+ byte[] endKey = null;
+ // Choose the smaller as start key
+ if (regionsToMerge[0].compareTo(regionsToMerge[1]) <= 0) {
+ startKey = regionsToMerge[0].getStartKey();
+ } else {
+ startKey = regionsToMerge[1].getStartKey();
+ }
+ // Choose the bigger as end key
+ if (Bytes.equals(regionsToMerge[0].getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
+ || (!Bytes.equals(regionsToMerge[1].getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
+ && Bytes.compareTo(regionsToMerge[0].getEndKey(), regionsToMerge[1].getEndKey()) > 0)) {
+ endKey = regionsToMerge[0].getEndKey();
+ } else {
+ endKey = regionsToMerge[1].getEndKey();
+ }
+
+ // Merged region is sorted between two merging regions in META
+ mergedRegionInfo = new HRegionInfo(getTableName(), startKey, endKey, false, rid);
+ }
+
+ /**
+ * Move all regions to the same region server
+ * @param env MasterProcedureEnv
+ * @return whether target regions hosted by the same RS
+ * @throws IOException
+ */
+ private boolean MoveRegionsToSameRS(final MasterProcedureEnv env) throws IOException {
+ // Make sure regions are on the same regionserver before send merge
+ // regions request to region server.
+ //
+ boolean onSameRS = isRegionsOnTheSameServer(env);
+ if (!onSameRS) {
+ // Note: the following logic assumes that we only have 2 regions to merge. In the future,
+ // if we want to extend to more than 2 regions, the code needs to modify a little bit.
+ //
+ RegionStates regionStates = getAssignmentManager(env).getRegionStates();
+ ServerName regionLocation2 = regionStates.getRegionServerOfRegion(regionsToMerge[1]);
+
+ RegionLoad loadOfRegionA = getRegionLoad(env, regionLocation, regionsToMerge[0]);
+ RegionLoad loadOfRegionB = getRegionLoad(env, regionLocation2, regionsToMerge[1]);
+ if (loadOfRegionA != null && loadOfRegionB != null
+ && loadOfRegionA.getRequestsCount() < loadOfRegionB.getRequestsCount()) {
+ // switch regionsToMerge[0] and regionsToMerge[1]
+ HRegionInfo tmpRegion = this.regionsToMerge[0];
+ this.regionsToMerge[0] = this.regionsToMerge[1];
+ this.regionsToMerge[1] = tmpRegion;
+ ServerName tmpLocation = regionLocation;
+ regionLocation = regionLocation2;
+ regionLocation2 = tmpLocation;
+ }
+
+ long startTime = EnvironmentEdgeManager.currentTime();
+
+ RegionPlan regionPlan = new RegionPlan(regionsToMerge[1], regionLocation2, regionLocation);
+ LOG.info("Moving regions to same server for merge: " + regionPlan.toString());
+ getAssignmentManager(env).balance(regionPlan);
+ do {
+ try {
+ Thread.sleep(20);
+ // Make sure check RIT first, then get region location, otherwise
+ // we would make a wrong result if region is online between getting
+ // region location and checking RIT
+ boolean isRIT = regionStates.isRegionInTransition(regionsToMerge[1]);
+ regionLocation2 = regionStates.getRegionServerOfRegion(regionsToMerge[1]);
+ onSameRS = regionLocation.equals(regionLocation2);
+ if (onSameRS || !isRIT) {
+ // Regions are on the same RS, or regionsToMerge[1] is not in
+ // RegionInTransition any more
+ break;
+ }
+ } catch (InterruptedException e) {
+ InterruptedIOException iioe = new InterruptedIOException();
+ iioe.initCause(e);
+ throw iioe;
+ }
+ } while ((EnvironmentEdgeManager.currentTime() - startTime) <= getTimeout(env));
+ }
+ return onSameRS;
+ }
+
+ /**
+ * Pre merge region action
+ * @param env MasterProcedureEnv
+ **/
+ private void preMergeRegions(final MasterProcedureEnv env) throws IOException {
+ final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+ if (cpHost != null) {
+ boolean ret = cpHost.preMergeRegionsAction(regionsToMerge, getUser());
+ if (ret) {
+ throw new IOException(
+ "Coprocessor bypassing regions " + getRegionsToMergeListFullNameString() + " merge.");
+ }
+ }
+ }
+
+ /**
+ * Action after rollback a merge table regions action.
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void postRollBackMergeRegions(final MasterProcedureEnv env) throws IOException {
+ final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+ if (cpHost != null) {
+ cpHost.postRollBackMergeRegionsAction(regionsToMerge, getUser());
+ }
+ }
+
+ /**
+ * Set the region states to MERGING state
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ public void setRegionStateToMerging(final MasterProcedureEnv env) throws IOException {
+ RegionStateTransition.Builder transition = RegionStateTransition.newBuilder();
+ transition.setTransitionCode(TransitionCode.READY_TO_MERGE);
+ transition.addRegionInfo(HRegionInfo.convert(mergedRegionInfo));
+ transition.addRegionInfo(HRegionInfo.convert(regionsToMerge[0]));
+ transition.addRegionInfo(HRegionInfo.convert(regionsToMerge[1]));
+ if (env.getMasterServices().getAssignmentManager().onRegionTransition(
+ getServerName(env), transition.build()) != null) {
+ throw new IOException("Failed to update region state to MERGING for "
+ + getRegionsToMergeListFullNameString());
+ }
+ }
+
+ /**
+ * Rollback the region state change
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void setRegionStateToRevertMerging(final MasterProcedureEnv env) throws IOException {
+ RegionStateTransition.Builder transition = RegionStateTransition.newBuilder();
+ transition.setTransitionCode(TransitionCode.MERGE_REVERTED);
+ transition.addRegionInfo(HRegionInfo.convert(mergedRegionInfo));
+ transition.addRegionInfo(HRegionInfo.convert(regionsToMerge[0]));
+ transition.addRegionInfo(HRegionInfo.convert(regionsToMerge[1]));
+ String msg = env.getMasterServices().getAssignmentManager().onRegionTransition(
+ getServerName(env), transition.build());
+ if (msg != null) {
+ // If daughter regions are online, the msg is coming from RPC retry. Ignore it.
+ RegionStates regionStates = getAssignmentManager(env).getRegionStates();
+ if (!regionStates.isRegionOnline(regionsToMerge[0]) ||
+ !regionStates.isRegionOnline(regionsToMerge[1])) {
+ throw new IOException("Failed to update region state for "
+ + getRegionsToMergeListFullNameString()
+ + " as part of operation for reverting merge. Error message: " + msg);
+ }
+ }
+ }
+
+ /**
+ * Create merged region
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void createMergedRegion(final MasterProcedureEnv env) throws IOException {
+ final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+ final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable());
+ final FileSystem fs = mfs.getFileSystem();
+ HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
+ env.getMasterConfiguration(), fs, tabledir, regionsToMerge[0], false);
+ regionFs.createMergesDir();
+
+ mergeStoreFiles(env, regionFs, regionFs.getMergesDir());
+ HRegionFileSystem regionFs2 = HRegionFileSystem.openRegionFromFileSystem(
+ env.getMasterConfiguration(), fs, tabledir, regionsToMerge[1], false);
+ mergeStoreFiles(env, regionFs2, regionFs.getMergesDir());
+
+ regionFs.commitMergedRegion(mergedRegionInfo);
+ }
+
+ /**
+ * Create reference file(s) of merging regions under the merges directory
+ * @param env MasterProcedureEnv
+ * @param regionFs region file system
+ * @param mergedDir the temp directory of merged region
+ * @throws IOException
+ */
+ private void mergeStoreFiles(
+ final MasterProcedureEnv env, final HRegionFileSystem regionFs, final Path mergedDir)
+ throws IOException {
+ final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+ final Configuration conf = env.getMasterConfiguration();
+ final HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
+
+ for (String family: regionFs.getFamilies()) {
+ final HColumnDescriptor hcd = htd.getFamily(family.getBytes());
+ final Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(family);
+
+ if (storeFiles != null && storeFiles.size() > 0) {
+ final CacheConfig cacheConf = new CacheConfig(conf, hcd);
+ for (StoreFileInfo storeFileInfo: storeFiles) {
+ // Create reference file(s) of the region in mergedDir
+ regionFs.mergeStoreFile(
+ mergedRegionInfo,
+ family,
+ new StoreFile(
+ mfs.getFileSystem(), storeFileInfo, conf, cacheConf, hcd.getBloomFilterType()),
+ mergedDir);
+ }
+ }
+ }
+ }
+
+ /**
+ * Clean up merged region
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void cleanupMergedRegion(final MasterProcedureEnv env) throws IOException {
+ final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+ final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable());
+ final FileSystem fs = mfs.getFileSystem();
+ HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
+ env.getMasterConfiguration(), fs, tabledir, regionsToMerge[0], false);
+ regionFs.cleanupMergedRegion(mergedRegionInfo);
+ }
+
+ /**
+ * RPC to region server that host the regions to merge, ask for close these regions
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void closeRegionsForMerge(final MasterProcedureEnv env) throws IOException {
+ boolean success = env.getMasterServices().getServerManager().sendRegionCloseForSplitOrMerge(
+ getServerName(env), regionsToMerge[0], regionsToMerge[1]);
+ if (!success) {
+ throw new IOException("Close regions " + getRegionsToMergeListFullNameString()
+ + " for merging failed. Check region server log for more details.");
+ }
+ }
+
+ /**
+ * Rollback close regions
+ * @param env MasterProcedureEnv
+ **/
+ private void rollbackCloseRegionsForMerge(final MasterProcedureEnv env) throws IOException {
+ // Check whether the region is closed; if so, open it in the same server
+ RegionStates regionStates = getAssignmentManager(env).getRegionStates();
+ for(int i = 1; i < regionsToMerge.length; i++) {
+ RegionState state = regionStates.getRegionState(regionsToMerge[i]);
+ if (state != null && (state.isClosing() || state.isClosed())) {
+ env.getMasterServices().getServerManager().sendRegionOpen(
+ getServerName(env),
+ regionsToMerge[i],
+ ServerName.EMPTY_SERVER_LIST);
+ }
+ }
+ }
+
+ /**
+ * Post merge region action
+ * @param env MasterProcedureEnv
+ **/
+ private void preMergeRegionsCommit(final MasterProcedureEnv env) throws IOException {
+ final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+ if (cpHost != null) {
+ @MetaMutationAnnotation
+ final List<Mutation> metaEntries = new ArrayList<Mutation>();
+ boolean ret = cpHost.preMergeRegionsCommit(regionsToMerge, metaEntries, getUser());
+
+ if (ret) {
+ throw new IOException(
+ "Coprocessor bypassing regions " + getRegionsToMergeListFullNameString() + " merge.");
+ }
+ try {
+ for (Mutation p : metaEntries) {
+ HRegionInfo.parseRegionName(p.getRow());
+ }
+ } catch (IOException e) {
+ LOG.error("Row key of mutation from coprocessor is not parsable as region name."
+ + "Mutations from coprocessor should only be for hbase:meta table.", e);
+ throw e;
+ }
+ }
+ }
+
+ /**
+ * Add merged region to META and delete original regions.
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void updateMetaForMergedRegions(final MasterProcedureEnv env) throws IOException {
+ RegionStateTransition.Builder transition = RegionStateTransition.newBuilder();
+ transition.setTransitionCode(TransitionCode.MERGE_PONR);
+ transition.addRegionInfo(HRegionInfo.convert(mergedRegionInfo));
+ transition.addRegionInfo(HRegionInfo.convert(regionsToMerge[0]));
+ transition.addRegionInfo(HRegionInfo.convert(regionsToMerge[1]));
+ // Add merged region and delete original regions
+ // as an atomic update. See HBASE-7721. This update to hbase:meta makes the region
+ // will determine whether the region is merged or not in case of failures.
+ if (env.getMasterServices().getAssignmentManager().onRegionTransition(
+ getServerName(env), transition.build()) != null) {
+ throw new IOException("Failed to update meta to add merged region that merges "
+ + getRegionsToMergeListFullNameString());
+ }
+ }
+
+ /**
+ * Post merge region action
+ * @param env MasterProcedureEnv
+ **/
+ private void postMergeRegionsCommit(final MasterProcedureEnv env) throws IOException {
+ final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+ if (cpHost != null) {
+ cpHost.postMergeRegionsCommit(regionsToMerge, mergedRegionInfo, getUser());
+ }
+ }
+
+ /**
+ * Assign merged region
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ * @throws InterruptedException
+ **/
+ private void openMergedRegions(final MasterProcedureEnv env)
+ throws IOException, InterruptedException {
+ // Check whether the merged region is already opened; if so,
+ // this is retry and we should just ignore.
+ RegionState regionState =
+ getAssignmentManager(env).getRegionStates().getRegionState(mergedRegionInfo);
+ if (regionState != null && regionState.isOpened()) {
+ LOG.info("Skip opening merged region " + mergedRegionInfo.getRegionNameAsString()
+ + " as it is already opened.");
+ return;
+ }
+
+ // TODO: The new AM should provide an API to force assign the merged region to the same RS
+ // as daughter regions; if the RS is unavailable, then assign to a different RS.
+ env.getMasterServices().getAssignmentManager().assignMergedRegion(
+ mergedRegionInfo, regionsToMerge[0], regionsToMerge[1]);
+ }
+
+ /**
+ * Post merge region action
+ * @param env MasterProcedureEnv
+ **/
+ private void postCompletedMergeRegions(final MasterProcedureEnv env) throws IOException {
+ final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+ if (cpHost != null) {
+ cpHost.postCompletedMergeRegionsAction(regionsToMerge, mergedRegionInfo, getUser());
+ }
+ }
+
+ private RegionLoad getRegionLoad(
+ final MasterProcedureEnv env,
+ final ServerName sn,
+ final HRegionInfo hri) {
+ ServerManager serverManager = env.getMasterServices().getServerManager();
+ ServerLoad load = serverManager.getLoad(sn);
+ if (load != null) {
+ Map<byte[], RegionLoad> regionsLoad = load.getRegionsLoad();
+ if (regionsLoad != null) {
+ return regionsLoad.get(hri.getRegionName());
+ }
+ }
+ return null;
+ }
+
+ /**
+ * The procedure could be restarted from a different machine. If the variable is null, we need to
+ * retrieve it.
+ * @param env MasterProcedureEnv
+ * @return whether target regions hosted by the same RS
+ */
+ private boolean isRegionsOnTheSameServer(final MasterProcedureEnv env) throws IOException{
+ Boolean onSameRS = true;
+ int i = 0;
+ RegionStates regionStates = getAssignmentManager(env).getRegionStates();
+ regionLocation = regionStates.getRegionServerOfRegion(regionsToMerge[i]);
+ if (regionLocation != null) {
+ for(i = 1; i < regionsToMerge.length; i++) {
+ ServerName regionLocation2 = regionStates.getRegionServerOfRegion(regionsToMerge[i]);
+ if (regionLocation2 != null) {
+ if (onSameRS) {
+ onSameRS = regionLocation.equals(regionLocation2);
+ }
+ } else {
+ // At least one region is not online, merge will fail, no need to continue.
+ break;
+ }
+ }
+ if (i == regionsToMerge.length) {
+ // Finish checking all regions, return the result;
+ return onSameRS;
+ }
+ }
+
+ // If reaching here, at least one region is not online.
+ String msg = "Skip merging regions " + getRegionsToMergeListFullNameString() +
+ ", because region " + regionsToMerge[i].getEncodedName() + " is not online now.";
+ LOG.warn(msg);
+ throw new IOException(msg);
+ }
+
+ /**
+ * The procedure could be restarted from a different machine. If the variable is null, we need to
+ * retrieve it.
+ * @param env MasterProcedureEnv
+ * @return assignmentManager
+ */
+ private AssignmentManager getAssignmentManager(final MasterProcedureEnv env) {
+ if (assignmentManager == null) {
+ assignmentManager = env.getMasterServices().getAssignmentManager();
+ }
+ return assignmentManager;
+ }
+
+ /**
+ * The procedure could be restarted from a different machine. If the variable is null, we need to
+ * retrieve it.
+ * @param env MasterProcedureEnv
+ * @return timeout value
+ */
+ private int getTimeout(final MasterProcedureEnv env) {
+ if (timeout == -1) {
+ timeout = env.getMasterConfiguration().getInt(
+ "hbase.master.regionmerge.timeout", regionsToMerge.length * 60 * 1000);
+ }
+ return timeout;
+ }
+
+ /**
+ * The procedure could be restarted from a different machine. If the variable is null, we need to
+ * retrieve it.
+ * @param env MasterProcedureEnv
+ * @return serverName
+ */
+ private ServerName getServerName(final MasterProcedureEnv env) {
+ if (regionLocation == null) {
+ regionLocation =
+ getAssignmentManager(env).getRegionStates().getRegionServerOfRegion(regionsToMerge[0]);
+ }
+ return regionLocation;
+ }
+
+ /**
+ * The procedure could be restarted from a different machine. If the variable is null, we need to
+ * retrieve it.
+ * @param fullName whether return only encoded name
+ * @return region names in a list
+ */
+ private String getRegionsToMergeListFullNameString() {
+ if (regionsToMergeListFullName == null) {
+ StringBuilder sb = new StringBuilder("[");
+ int i = 0;
+ while(i < regionsToMerge.length - 1) {
+ sb.append(regionsToMerge[i].getRegionNameAsString() + ", ");
+ i++;
+ }
+ sb.append(regionsToMerge[i].getRegionNameAsString() + " ]");
+ regionsToMergeListFullName = sb.toString();
+ }
+ return regionsToMergeListFullName;
+ }
+
+ /**
+ * The procedure could be restarted from a different machine. If the variable is null, we need to
+ * retrieve it.
+ * @return encoded region names
+ */
+ private String getRegionsToMergeListEncodedNameString() {
+ if (regionsToMergeListEncodedName == null) {
+ StringBuilder sb = new StringBuilder("[");
+ int i = 0;
+ while(i < regionsToMerge.length - 1) {
+ sb.append(regionsToMerge[i].getEncodedName() + ", ");
+ i++;
+ }
+ sb.append(regionsToMerge[i].getEncodedName() + " ]");
+ regionsToMergeListEncodedName = sb.toString();
+ }
+ return regionsToMergeListEncodedName;
+ }
+
+ /**
+ * The procedure could be restarted from a different machine. If the variable is null, we need to
+ * retrieve it.
+ * @return traceEnabled
+ */
+ private Boolean isTraceEnabled() {
+ if (traceEnabled == null) {
+ traceEnabled = LOG.isTraceEnabled();
+ }
+ return traceEnabled;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java
index 883ac9a..4730ad8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java
@@ -161,9 +161,9 @@ public class SplitTableRegionProcedure
break;
case SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE:
setRegionStateToSplitting(env);
- setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CLOSED_PARENT_REGION);
+ setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CLOSE_PARENT_REGION);
break;
- case SPLIT_TABLE_REGION_CLOSED_PARENT_REGION:
+ case SPLIT_TABLE_REGION_CLOSE_PARENT_REGION:
closeParentRegionForSplit(env);
setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS);
break;
@@ -242,14 +242,14 @@ public class SplitTableRegionProcedure
case SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS:
// Doing nothing, as re-open parent region would clean up daughter region directories.
break;
- case SPLIT_TABLE_REGION_CLOSED_PARENT_REGION:
+ case SPLIT_TABLE_REGION_CLOSE_PARENT_REGION:
openParentRegion(env);
break;
case SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE:
setRegionStateToRevertSplitting(env);
break;
case SPLIT_TABLE_REGION_PRE_OPERATION:
- preSplitRegionRollback(env);
+ postRollBackSplitRegion(env);
break;
case SPLIT_TABLE_REGION_PREPARE:
break; // nothing to do
@@ -408,15 +408,14 @@ public class SplitTableRegionProcedure
}
/**
- * Action during rollback a pre split table region.
+ * Action after rollback a split table region action.
* @param env MasterProcedureEnv
- * @param state the procedure state
* @throws IOException
*/
- private void preSplitRegionRollback(final MasterProcedureEnv env) throws IOException {
+ private void postRollBackSplitRegion(final MasterProcedureEnv env) throws IOException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
- cpHost.preRollBackSplitAction(getUser());
+ cpHost.postRollBackSplitRegionAction(getUser());
}
}
@@ -458,14 +457,13 @@ public class SplitTableRegionProcedure
}
/**
- * RPC to region server that host the parent region, ask for close the parent regions and
- * creating daughter regions
+ * RPC to region server that host the parent region, ask for close the parent regions
* @param env MasterProcedureEnv
* @throws IOException
*/
@VisibleForTesting
public void closeParentRegionForSplit(final MasterProcedureEnv env) throws IOException {
- boolean success = env.getMasterServices().getServerManager().sendRegionCloseForSplit(
+ boolean success = env.getMasterServices().getServerManager().sendRegionCloseForSplitOrMerge(
getParentRegionState(env).getServerName(), parentHRI);
if (!success) {
throw new IOException("Close parent region " + parentHRI + " for splitting failed."
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 50382a4..d4e80c3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -647,7 +647,7 @@ public class HRegionFileSystem {
// Merge Helpers
// ===========================================================================
/** @return {@link Path} to the temp directory used during merge operations */
- Path getMergesDir() {
+ public Path getMergesDir() {
return new Path(getRegionDir(), REGION_MERGES_DIR);
}
@@ -667,7 +667,7 @@ public class HRegionFileSystem {
* @param mergedRegion {@link HRegionInfo}
* @throws IOException
*/
- void cleanupMergedRegion(final HRegionInfo mergedRegion) throws IOException {
+ public void cleanupMergedRegion(final HRegionInfo mergedRegion) throws IOException {
Path regionDir = new Path(this.tableDir, mergedRegion.getEncodedName());
if (this.fs.exists(regionDir) && !this.fs.delete(regionDir, true)) {
throw new IOException("Failed delete of " + regionDir);
@@ -679,7 +679,7 @@ public class HRegionFileSystem {
* @throws IOException If merges dir already exists or we fail to create it.
* @see HRegionFileSystem#cleanupMergesDir()
*/
- void createMergesDir() throws IOException {
+ public void createMergesDir() throws IOException {
Path mergesdir = getMergesDir();
if (fs.exists(mergesdir)) {
LOG.info("The " + mergesdir
@@ -703,7 +703,7 @@ public class HRegionFileSystem {
* @return Path to created reference.
* @throws IOException
*/
- Path mergeStoreFile(final HRegionInfo mergedRegion, final String familyName,
+ public Path mergeStoreFile(final HRegionInfo mergedRegion, final String familyName,
final StoreFile f, final Path mergedDir)
throws IOException {
Path referenceDir = new Path(new Path(mergedDir,
@@ -728,7 +728,7 @@ public class HRegionFileSystem {
* @param mergedRegionInfo merged region {@link HRegionInfo}
* @throws IOException
*/
- void commitMergedRegion(final HRegionInfo mergedRegionInfo) throws IOException {
+ public void commitMergedRegion(final HRegionInfo mergedRegionInfo) throws IOException {
Path regionDir = new Path(this.tableDir, mergedRegionInfo.getEncodedName());
Path mergedRegionTmpDir = this.getMergesDir(mergedRegionInfo);
// Move the tmp dir in the expected location
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 56fc6eb..3e4a23e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -48,7 +48,6 @@ import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import javax.management.MalformedObjectNameException;
@@ -3026,54 +3025,58 @@ public class HRegionServer extends HasThread implements
}
/**
- * Close and offline the region for split
+ * Close and offline the region for split or merge
*
- * @param parentRegionEncodedName the name of the region to close
- * @return True if closed the region successfully.
+ * @param regionEncodedName the name of the region(s) to close
+ * @return true if closed the region successfully.
* @throws IOException
*/
- protected boolean closeAndOfflineRegionForSplit(
- final String parentRegionEncodedName) throws IOException {
- Region parentRegion = this.getFromOnlineRegions(parentRegionEncodedName);
- if (parentRegion != null) {
- Map<byte[], List<StoreFile>> hstoreFilesToSplit = null;
- Exception exceptionToThrow = null;
- try{
- hstoreFilesToSplit = ((HRegion)parentRegion).close(false);
- } catch (Exception e) {
- exceptionToThrow = e;
- }
- if (exceptionToThrow == null && hstoreFilesToSplit == null) {
- // The region was closed by someone else
- exceptionToThrow =
- new IOException("Failed to close region: already closed by another thread");
- }
+ protected boolean closeAndOfflineRegionForSplitOrMerge(
+ final List<String> regionEncodedName) throws IOException {
+ for (int i = 0; i < regionEncodedName.size(); ++i) {
+ Region regionToClose = this.getFromOnlineRegions(regionEncodedName.get(i));
+ if (regionToClose != null) {
+ Map<byte[], List<StoreFile>> hstoreFiles = null;
+ Exception exceptionToThrow = null;
+ try{
+ hstoreFiles = ((HRegion)regionToClose).close(false);
+ } catch (Exception e) {
+ exceptionToThrow = e;
+ }
+ if (exceptionToThrow == null && hstoreFiles == null) {
+ // The region was closed by someone else
+ exceptionToThrow =
+ new IOException("Failed to close region: already closed by another thread");
+ }
- if (exceptionToThrow != null) {
- if (exceptionToThrow instanceof IOException) throw (IOException)exceptionToThrow;
- throw new IOException(exceptionToThrow);
- }
- if (parentRegion.getTableDesc().hasSerialReplicationScope()) {
- // For serial replication, we need add a final barrier on this region. But the splitting may
- // be reverted, so we should make sure if we reopen this region, the open barrier is same as
- // this final barrier
- long seq = parentRegion.getMaxFlushedSeqId();
- if (seq == HConstants.NO_SEQNUM) {
- // No edits in WAL for this region; get the sequence number when the region was opened.
- seq = parentRegion.getOpenSeqNum();
+ if (exceptionToThrow != null) {
+ if (exceptionToThrow instanceof IOException) throw (IOException)exceptionToThrow;
+ throw new IOException(exceptionToThrow);
+ }
+ if (regionToClose.getTableDesc().hasSerialReplicationScope()) {
+ // For serial replication, we need add a final barrier on this region. But the splitting
+ // or merging may be reverted, so we should make sure if we reopen this region, the open
+ // barrier is same as this final barrier
+ long seq = regionToClose.getMaxFlushedSeqId();
if (seq == HConstants.NO_SEQNUM) {
- // This region has no data
- seq = 0;
+ // No edits in WAL for this region; get the sequence number when the region was opened.
+ seq = regionToClose.getOpenSeqNum();
+ if (seq == HConstants.NO_SEQNUM) {
+ // This region has no data
+ seq = 0;
+ }
+ } else {
+ seq++;
}
- } else {
- seq++;
+ Put finalBarrier = MetaTableAccessor.makeBarrierPut(
+ Bytes.toBytes(regionEncodedName.get(i)),
+ seq,
+ regionToClose.getTableDesc().getTableName().getName());
+ MetaTableAccessor.putToMetaTable(getConnection(), finalBarrier);
}
- Put finalBarrier = MetaTableAccessor.makeBarrierPut(Bytes.toBytes(parentRegionEncodedName),
- seq, parentRegion.getTableDesc().getTableName().getName());
- MetaTableAccessor.putToMetaTable(getConnection(), finalBarrier);
+ // Offline the region
+ this.removeFromOnlineRegions(regionToClose, null);
}
- // Offline the region
- this.removeFromOnlineRegions(parentRegion, null);
}
return true;
}
[4/6] hbase git commit: HBASE-16119 Procedure v2 - Reimplement Merge
region (Stephen Yuan Jiang)
Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
index 0be98ff..8f5992c 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
@@ -1833,6 +1833,177 @@ public final class MasterProcedureProtos {
}
/**
+ * Protobuf enum {@code hbase.pb.MergeTableRegionsState}
+ */
+ public enum MergeTableRegionsState
+ implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
+ /**
+ * <code>MERGE_TABLE_REGIONS_PREPARE = 1;</code>
+ */
+ MERGE_TABLE_REGIONS_PREPARE(1),
+ /**
+ * <code>MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 2;</code>
+ */
+ MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS(2),
+ /**
+ * <code>MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION = 3;</code>
+ */
+ MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION(3),
+ /**
+ * <code>MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE = 4;</code>
+ */
+ MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE(4),
+ /**
+ * <code>MERGE_TABLE_REGIONS_CLOSE_REGIONS = 5;</code>
+ */
+ MERGE_TABLE_REGIONS_CLOSE_REGIONS(5),
+ /**
+ * <code>MERGE_TABLE_REGIONS_CREATE_MERGED_REGION = 6;</code>
+ */
+ MERGE_TABLE_REGIONS_CREATE_MERGED_REGION(6),
+ /**
+ * <code>MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION = 7;</code>
+ */
+ MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION(7),
+ /**
+ * <code>MERGE_TABLE_REGIONS_UPDATE_META = 8;</code>
+ */
+ MERGE_TABLE_REGIONS_UPDATE_META(8),
+ /**
+ * <code>MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 9;</code>
+ */
+ MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION(9),
+ /**
+ * <code>MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 10;</code>
+ */
+ MERGE_TABLE_REGIONS_OPEN_MERGED_REGION(10),
+ /**
+ * <code>MERGE_TABLE_REGIONS_POST_OPERATION = 11;</code>
+ */
+ MERGE_TABLE_REGIONS_POST_OPERATION(11),
+ ;
+
+ /**
+ * <code>MERGE_TABLE_REGIONS_PREPARE = 1;</code>
+ */
+ public static final int MERGE_TABLE_REGIONS_PREPARE_VALUE = 1;
+ /**
+ * <code>MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 2;</code>
+ */
+ public static final int MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS_VALUE = 2;
+ /**
+ * <code>MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION = 3;</code>
+ */
+ public static final int MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION_VALUE = 3;
+ /**
+ * <code>MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE = 4;</code>
+ */
+ public static final int MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE_VALUE = 4;
+ /**
+ * <code>MERGE_TABLE_REGIONS_CLOSE_REGIONS = 5;</code>
+ */
+ public static final int MERGE_TABLE_REGIONS_CLOSE_REGIONS_VALUE = 5;
+ /**
+ * <code>MERGE_TABLE_REGIONS_CREATE_MERGED_REGION = 6;</code>
+ */
+ public static final int MERGE_TABLE_REGIONS_CREATE_MERGED_REGION_VALUE = 6;
+ /**
+ * <code>MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION = 7;</code>
+ */
+ public static final int MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION_VALUE = 7;
+ /**
+ * <code>MERGE_TABLE_REGIONS_UPDATE_META = 8;</code>
+ */
+ public static final int MERGE_TABLE_REGIONS_UPDATE_META_VALUE = 8;
+ /**
+ * <code>MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 9;</code>
+ */
+ public static final int MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION_VALUE = 9;
+ /**
+ * <code>MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 10;</code>
+ */
+ public static final int MERGE_TABLE_REGIONS_OPEN_MERGED_REGION_VALUE = 10;
+ /**
+ * <code>MERGE_TABLE_REGIONS_POST_OPERATION = 11;</code>
+ */
+ public static final int MERGE_TABLE_REGIONS_POST_OPERATION_VALUE = 11;
+
+
+ public final int getNumber() {
+ return value;
+ }
+
+ /**
+ * @deprecated Use {@link #forNumber(int)} instead.
+ */
+ @java.lang.Deprecated
+ public static MergeTableRegionsState valueOf(int value) {
+ return forNumber(value);
+ }
+
+ public static MergeTableRegionsState forNumber(int value) {
+ switch (value) {
+ case 1: return MERGE_TABLE_REGIONS_PREPARE;
+ case 2: return MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS;
+ case 3: return MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION;
+ case 4: return MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE;
+ case 5: return MERGE_TABLE_REGIONS_CLOSE_REGIONS;
+ case 6: return MERGE_TABLE_REGIONS_CREATE_MERGED_REGION;
+ case 7: return MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION;
+ case 8: return MERGE_TABLE_REGIONS_UPDATE_META;
+ case 9: return MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION;
+ case 10: return MERGE_TABLE_REGIONS_OPEN_MERGED_REGION;
+ case 11: return MERGE_TABLE_REGIONS_POST_OPERATION;
+ default: return null;
+ }
+ }
+
+ public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<MergeTableRegionsState>
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
+ MergeTableRegionsState> internalValueMap =
+ new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<MergeTableRegionsState>() {
+ public MergeTableRegionsState findValueByNumber(int number) {
+ return MergeTableRegionsState.forNumber(number);
+ }
+ };
+
+ public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
+ getValueDescriptor() {
+ return getDescriptor().getValues().get(ordinal());
+ }
+ public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptorForType() {
+ return getDescriptor();
+ }
+ public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(15);
+ }
+
+ private static final MergeTableRegionsState[] VALUES = values();
+
+ public static MergeTableRegionsState valueOf(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+ if (desc.getType() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "EnumValueDescriptor is not for this type.");
+ }
+ return VALUES[desc.getIndex()];
+ }
+
+ private final int value;
+
+ private MergeTableRegionsState(int value) {
+ this.value = value;
+ }
+
+ // @@protoc_insertion_point(enum_scope:hbase.pb.MergeTableRegionsState)
+ }
+
+ /**
* Protobuf enum {@code hbase.pb.SplitTableRegionState}
*/
public enum SplitTableRegionState
@@ -1850,9 +2021,9 @@ public final class MasterProcedureProtos {
*/
SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE(3),
/**
- * <code>SPLIT_TABLE_REGION_CLOSED_PARENT_REGION = 4;</code>
+ * <code>SPLIT_TABLE_REGION_CLOSE_PARENT_REGION = 4;</code>
*/
- SPLIT_TABLE_REGION_CLOSED_PARENT_REGION(4),
+ SPLIT_TABLE_REGION_CLOSE_PARENT_REGION(4),
/**
* <code>SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS = 5;</code>
*/
@@ -1892,9 +2063,9 @@ public final class MasterProcedureProtos {
*/
public static final int SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE_VALUE = 3;
/**
- * <code>SPLIT_TABLE_REGION_CLOSED_PARENT_REGION = 4;</code>
+ * <code>SPLIT_TABLE_REGION_CLOSE_PARENT_REGION = 4;</code>
*/
- public static final int SPLIT_TABLE_REGION_CLOSED_PARENT_REGION_VALUE = 4;
+ public static final int SPLIT_TABLE_REGION_CLOSE_PARENT_REGION_VALUE = 4;
/**
* <code>SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS = 5;</code>
*/
@@ -1938,7 +2109,7 @@ public final class MasterProcedureProtos {
case 1: return SPLIT_TABLE_REGION_PREPARE;
case 2: return SPLIT_TABLE_REGION_PRE_OPERATION;
case 3: return SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE;
- case 4: return SPLIT_TABLE_REGION_CLOSED_PARENT_REGION;
+ case 4: return SPLIT_TABLE_REGION_CLOSE_PARENT_REGION;
case 5: return SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS;
case 6: return SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR;
case 7: return SPLIT_TABLE_REGION_UPDATE_META;
@@ -1971,7 +2142,7 @@ public final class MasterProcedureProtos {
}
public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
getDescriptor() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(15);
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(16);
}
private static final SplitTableRegionState[] VALUES = values();
@@ -2132,7 +2303,7 @@ public final class MasterProcedureProtos {
}
public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
getDescriptor() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(16);
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(17);
}
private static final ServerCrashState[] VALUES = values();
@@ -21109,8 +21280,8 @@ public final class MasterProcedureProtos {
}
- public interface SplitTableRegionStateDataOrBuilder extends
- // @@protoc_insertion_point(interface_extends:hbase.pb.SplitTableRegionStateData)
+ public interface MergeTableRegionsStateDataOrBuilder extends
+ // @@protoc_insertion_point(interface_extends:hbase.pb.MergeTableRegionsStateData)
org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
/**
@@ -21127,55 +21298,65 @@ public final class MasterProcedureProtos {
org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
/**
- * <code>required .hbase.pb.RegionInfo parent_region_info = 2;</code>
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
*/
- boolean hasParentRegionInfo();
+ java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo>
+ getRegionInfoList();
/**
- * <code>required .hbase.pb.RegionInfo parent_region_info = 2;</code>
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
*/
- org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getParentRegionInfo();
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
/**
- * <code>required .hbase.pb.RegionInfo parent_region_info = 2;</code>
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
*/
- org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getParentRegionInfoOrBuilder();
+ int getRegionInfoCount();
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
+ getRegionInfoOrBuilderList();
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+ int index);
/**
- * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
*/
- java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo>
- getChildRegionInfoList();
+ boolean hasMergedRegionInfo();
/**
- * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
*/
- org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getChildRegionInfo(int index);
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getMergedRegionInfo();
/**
- * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
*/
- int getChildRegionInfoCount();
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getMergedRegionInfoOrBuilder();
+
/**
- * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ * <code>optional bool forcible = 4 [default = false];</code>
*/
- java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
- getChildRegionInfoOrBuilderList();
+ boolean hasForcible();
/**
- * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ * <code>optional bool forcible = 4 [default = false];</code>
*/
- org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getChildRegionInfoOrBuilder(
- int index);
+ boolean getForcible();
}
/**
- * Protobuf type {@code hbase.pb.SplitTableRegionStateData}
+ * Protobuf type {@code hbase.pb.MergeTableRegionsStateData}
*/
- public static final class SplitTableRegionStateData extends
+ public static final class MergeTableRegionsStateData extends
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
- // @@protoc_insertion_point(message_implements:hbase.pb.SplitTableRegionStateData)
- SplitTableRegionStateDataOrBuilder {
- // Use SplitTableRegionStateData.newBuilder() to construct.
- private SplitTableRegionStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+ // @@protoc_insertion_point(message_implements:hbase.pb.MergeTableRegionsStateData)
+ MergeTableRegionsStateDataOrBuilder {
+ // Use MergeTableRegionsStateData.newBuilder() to construct.
+ private MergeTableRegionsStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
super(builder);
}
- private SplitTableRegionStateData() {
- childRegionInfo_ = java.util.Collections.emptyList();
+ private MergeTableRegionsStateData() {
+ regionInfo_ = java.util.Collections.emptyList();
+ forcible_ = false;
}
@java.lang.Override
@@ -21183,7 +21364,7 @@ public final class MasterProcedureProtos {
getUnknownFields() {
return this.unknownFields;
}
- private SplitTableRegionStateData(
+ private MergeTableRegionsStateData(
org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
@@ -21220,25 +21401,30 @@ public final class MasterProcedureProtos {
break;
}
case 18: {
+ if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+ regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo>();
+ mutable_bitField0_ |= 0x00000002;
+ }
+ regionInfo_.add(
+ input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
+ break;
+ }
+ case 26: {
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder subBuilder = null;
if (((bitField0_ & 0x00000002) == 0x00000002)) {
- subBuilder = parentRegionInfo_.toBuilder();
+ subBuilder = mergedRegionInfo_.toBuilder();
}
- parentRegionInfo_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry);
+ mergedRegionInfo_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry);
if (subBuilder != null) {
- subBuilder.mergeFrom(parentRegionInfo_);
- parentRegionInfo_ = subBuilder.buildPartial();
+ subBuilder.mergeFrom(mergedRegionInfo_);
+ mergedRegionInfo_ = subBuilder.buildPartial();
}
bitField0_ |= 0x00000002;
break;
}
- case 26: {
- if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
- childRegionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo>();
- mutable_bitField0_ |= 0x00000004;
- }
- childRegionInfo_.add(
- input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
+ case 32: {
+ bitField0_ |= 0x00000004;
+ forcible_ = input.readBool();
break;
}
}
@@ -21249,8 +21435,8 @@ public final class MasterProcedureProtos {
throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
e).setUnfinishedMessage(this);
} finally {
- if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
- childRegionInfo_ = java.util.Collections.unmodifiableList(childRegionInfo_);
+ if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+ regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
}
this.unknownFields = unknownFields.build();
makeExtensionsImmutable();
@@ -21258,14 +21444,14 @@ public final class MasterProcedureProtos {
}
public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_SplitTableRegionStateData_descriptor;
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_descriptor;
}
protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
internalGetFieldAccessorTable() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_SplitTableRegionStateData_fieldAccessorTable
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_fieldAccessorTable
.ensureFieldAccessorsInitialized(
- org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionStateData.Builder.class);
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.Builder.class);
}
private int bitField0_;
@@ -21290,60 +21476,75 @@ public final class MasterProcedureProtos {
return userInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance() : userInfo_;
}
- public static final int PARENT_REGION_INFO_FIELD_NUMBER = 2;
- private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parentRegionInfo_;
+ public static final int REGION_INFO_FIELD_NUMBER = 2;
+ private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_;
/**
- * <code>required .hbase.pb.RegionInfo parent_region_info = 2;</code>
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
*/
- public boolean hasParentRegionInfo() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
+ public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
+ return regionInfo_;
}
/**
- * <code>required .hbase.pb.RegionInfo parent_region_info = 2;</code>
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
*/
- public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getParentRegionInfo() {
- return parentRegionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : parentRegionInfo_;
+ public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
+ getRegionInfoOrBuilderList() {
+ return regionInfo_;
}
/**
- * <code>required .hbase.pb.RegionInfo parent_region_info = 2;</code>
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
*/
- public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getParentRegionInfoOrBuilder() {
- return parentRegionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : parentRegionInfo_;
+ public int getRegionInfoCount() {
+ return regionInfo_.size();
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
+ return regionInfo_.get(index);
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+ int index) {
+ return regionInfo_.get(index);
}
- public static final int CHILD_REGION_INFO_FIELD_NUMBER = 3;
- private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> childRegionInfo_;
+ public static final int MERGED_REGION_INFO_FIELD_NUMBER = 3;
+ private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo mergedRegionInfo_;
/**
- * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
*/
- public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> getChildRegionInfoList() {
- return childRegionInfo_;
+ public boolean hasMergedRegionInfo() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
}
/**
- * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
*/
- public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
- getChildRegionInfoOrBuilderList() {
- return childRegionInfo_;
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getMergedRegionInfo() {
+ return mergedRegionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : mergedRegionInfo_;
}
/**
- * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
*/
- public int getChildRegionInfoCount() {
- return childRegionInfo_.size();
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getMergedRegionInfoOrBuilder() {
+ return mergedRegionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : mergedRegionInfo_;
}
+
+ public static final int FORCIBLE_FIELD_NUMBER = 4;
+ private boolean forcible_;
/**
- * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ * <code>optional bool forcible = 4 [default = false];</code>
*/
- public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getChildRegionInfo(int index) {
- return childRegionInfo_.get(index);
+ public boolean hasForcible() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
}
/**
- * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ * <code>optional bool forcible = 4 [default = false];</code>
*/
- public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getChildRegionInfoOrBuilder(
- int index) {
- return childRegionInfo_.get(index);
+ public boolean getForcible() {
+ return forcible_;
}
private byte memoizedIsInitialized = -1;
@@ -21356,7 +21557,7 @@ public final class MasterProcedureProtos {
memoizedIsInitialized = 0;
return false;
}
- if (!hasParentRegionInfo()) {
+ if (!hasMergedRegionInfo()) {
memoizedIsInitialized = 0;
return false;
}
@@ -21364,16 +21565,16 @@ public final class MasterProcedureProtos {
memoizedIsInitialized = 0;
return false;
}
- if (!getParentRegionInfo().isInitialized()) {
- memoizedIsInitialized = 0;
- return false;
- }
- for (int i = 0; i < getChildRegionInfoCount(); i++) {
- if (!getChildRegionInfo(i).isInitialized()) {
+ for (int i = 0; i < getRegionInfoCount(); i++) {
+ if (!getRegionInfo(i).isInitialized()) {
memoizedIsInitialized = 0;
return false;
}
}
+ if (!getMergedRegionInfo().isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
memoizedIsInitialized = 1;
return true;
}
@@ -21383,11 +21584,1252 @@ public final class MasterProcedureProtos {
if (((bitField0_ & 0x00000001) == 0x00000001)) {
output.writeMessage(1, getUserInfo());
}
+ for (int i = 0; i < regionInfo_.size(); i++) {
+ output.writeMessage(2, regionInfo_.get(i));
+ }
if (((bitField0_ & 0x00000002) == 0x00000002)) {
- output.writeMessage(2, getParentRegionInfo());
+ output.writeMessage(3, getMergedRegionInfo());
}
- for (int i = 0; i < childRegionInfo_.size(); i++) {
- output.writeMessage(3, childRegionInfo_.get(i));
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeBool(4, forcible_);
+ }
+ unknownFields.writeTo(output);
+ }
+
+ public int getSerializedSize() {
+ int size = memoizedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+ .computeMessageSize(1, getUserInfo());
+ }
+ for (int i = 0; i < regionInfo_.size(); i++) {
+ size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+ .computeMessageSize(2, regionInfo_.get(i));
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+ .computeMessageSize(3, getMergedRegionInfo());
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+ .computeBoolSize(4, forcible_);
+ }
+ size += unknownFields.getSerializedSize();
+ memoizedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ public boolean equals(final java.lang.Object obj) {
+ if (obj == this) {
+ return true;
+ }
+ if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData)) {
+ return super.equals(obj);
+ }
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData) obj;
+
+ boolean result = true;
+ result = result && (hasUserInfo() == other.hasUserInfo());
+ if (hasUserInfo()) {
+ result = result && getUserInfo()
+ .equals(other.getUserInfo());
+ }
+ result = result && getRegionInfoList()
+ .equals(other.getRegionInfoList());
+ result = result && (hasMergedRegionInfo() == other.hasMergedRegionInfo());
+ if (hasMergedRegionInfo()) {
+ result = result && getMergedRegionInfo()
+ .equals(other.getMergedRegionInfo());
+ }
+ result = result && (hasForcible() == other.hasForcible());
+ if (hasForcible()) {
+ result = result && (getForcible()
+ == other.getForcible());
+ }
+ result = result && unknownFields.equals(other.unknownFields);
+ return result;
+ }
+
+ @java.lang.Override
+ public int hashCode() {
+ if (memoizedHashCode != 0) {
+ return memoizedHashCode;
+ }
+ int hash = 41;
+ hash = (19 * hash) + getDescriptorForType().hashCode();
+ if (hasUserInfo()) {
+ hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
+ hash = (53 * hash) + getUserInfo().hashCode();
+ }
+ if (getRegionInfoCount() > 0) {
+ hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
+ hash = (53 * hash) + getRegionInfoList().hashCode();
+ }
+ if (hasMergedRegionInfo()) {
+ hash = (37 * hash) + MERGED_REGION_INFO_FIELD_NUMBER;
+ hash = (53 * hash) + getMergedRegionInfo().hashCode();
+ }
+ if (hasForcible()) {
+ hash = (37 * hash) + FORCIBLE_FIELD_NUMBER;
+ hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
+ getForcible());
+ }
+ hash = (29 * hash) + unknownFields.hashCode();
+ memoizedHashCode = hash;
+ return hash;
+ }
+
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(byte[] data)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(
+ byte[] data,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(
+ java.io.InputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseDelimitedWithIOException(PARSER, input);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseDelimitedFrom(
+ java.io.InputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input, extensionRegistry);
+ }
+
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder() {
+ return DEFAULT_INSTANCE.toBuilder();
+ }
+ public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData prototype) {
+ return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() {
+ return this == DEFAULT_INSTANCE
+ ? new Builder() : new Builder().mergeFrom(this);
+ }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * Protobuf type {@code hbase.pb.MergeTableRegionsStateData}
+ */
+ public static final class Builder extends
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+ // @@protoc_insertion_point(builder_implements:hbase.pb.MergeTableRegionsStateData)
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateDataOrBuilder {
+ public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_descriptor;
+ }
+
+ protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.Builder.class);
+ }
+
+ // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .alwaysUseFieldBuilders) {
+ getUserInfoFieldBuilder();
+ getRegionInfoFieldBuilder();
+ getMergedRegionInfoFieldBuilder();
+ }
+ }
+ public Builder clear() {
+ super.clear();
+ if (userInfoBuilder_ == null) {
+ userInfo_ = null;
+ } else {
+ userInfoBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ if (regionInfoBuilder_ == null) {
+ regionInfo_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000002);
+ } else {
+ regionInfoBuilder_.clear();
+ }
+ if (mergedRegionInfoBuilder_ == null) {
+ mergedRegionInfo_ = null;
+ } else {
+ mergedRegionInfoBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000004);
+ forcible_ = false;
+ bitField0_ = (bitField0_ & ~0x00000008);
+ return this;
+ }
+
+ public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_descriptor;
+ }
+
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData getDefaultInstanceForType() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.getDefaultInstance();
+ }
+
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData build() {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData buildPartial() {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ if (userInfoBuilder_ == null) {
+ result.userInfo_ = userInfo_;
+ } else {
+ result.userInfo_ = userInfoBuilder_.build();
+ }
+ if (regionInfoBuilder_ == null) {
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
+ bitField0_ = (bitField0_ & ~0x00000002);
+ }
+ result.regionInfo_ = regionInfo_;
+ } else {
+ result.regionInfo_ = regionInfoBuilder_.build();
+ }
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ if (mergedRegionInfoBuilder_ == null) {
+ result.mergedRegionInfo_ = mergedRegionInfo_;
+ } else {
+ result.mergedRegionInfo_ = mergedRegionInfoBuilder_.build();
+ }
+ if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.forcible_ = forcible_;
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder clone() {
+ return (Builder) super.clone();
+ }
+ public Builder setField(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+ Object value) {
+ return (Builder) super.setField(field, value);
+ }
+ public Builder clearField(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+ return (Builder) super.clearField(field);
+ }
+ public Builder clearOneof(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+ return (Builder) super.clearOneof(oneof);
+ }
+ public Builder setRepeatedField(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+ int index, Object value) {
+ return (Builder) super.setRepeatedField(field, index, value);
+ }
+ public Builder addRepeatedField(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+ Object value) {
+ return (Builder) super.addRepeatedField(field, value);
+ }
+ public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+ if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData) {
+ return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData other) {
+ if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.getDefaultInstance()) return this;
+ if (other.hasUserInfo()) {
+ mergeUserInfo(other.getUserInfo());
+ }
+ if (regionInfoBuilder_ == null) {
+ if (!other.regionInfo_.isEmpty()) {
+ if (regionInfo_.isEmpty()) {
+ regionInfo_ = other.regionInfo_;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ } else {
+ ensureRegionInfoIsMutable();
+ regionInfo_.addAll(other.regionInfo_);
+ }
+ onChanged();
+ }
+ } else {
+ if (!other.regionInfo_.isEmpty()) {
+ if (regionInfoBuilder_.isEmpty()) {
+ regionInfoBuilder_.dispose();
+ regionInfoBuilder_ = null;
+ regionInfo_ = other.regionInfo_;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ regionInfoBuilder_ =
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+ getRegionInfoFieldBuilder() : null;
+ } else {
+ regionInfoBuilder_.addAllMessages(other.regionInfo_);
+ }
+ }
+ }
+ if (other.hasMergedRegionInfo()) {
+ mergeMergedRegionInfo(other.getMergedRegionInfo());
+ }
+ if (other.hasForcible()) {
+ setForcible(other.getForcible());
+ }
+ this.mergeUnknownFields(other.unknownFields);
+ onChanged();
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ if (!hasUserInfo()) {
+ return false;
+ }
+ if (!hasMergedRegionInfo()) {
+ return false;
+ }
+ if (!getUserInfo().isInitialized()) {
+ return false;
+ }
+ for (int i = 0; i < getRegionInfoCount(); i++) {
+ if (!getRegionInfo(i).isInitialized()) {
+ return false;
+ }
+ }
+ if (!getMergedRegionInfo().isInitialized()) {
+ return false;
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData) e.getUnfinishedMessage();
+ throw e.unwrapIOException();
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_ = null;
+ private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+ org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ public boolean hasUserInfo() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+ if (userInfoBuilder_ == null) {
+ return userInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance() : userInfo_;
+ } else {
+ return userInfoBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ public Builder setUserInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation value) {
+ if (userInfoBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ userInfo_ = value;
+ onChanged();
+ } else {
+ userInfoBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ public Builder setUserInfo(
+ org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder builderForValue) {
+ if (userInfoBuilder_ == null) {
+ userInfo_ = builderForValue.build();
+ onChanged();
+ } else {
+ userInfoBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ public Builder mergeUserInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation value) {
+ if (userInfoBuilder_ == null) {
+ if (((bitField0_ & 0x00000001) == 0x00000001) &&
+ userInfo_ != null &&
+ userInfo_ != org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance()) {
+ userInfo_ =
+ org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.newBuilder(userInfo_).mergeFrom(value).buildPartial();
+ } else {
+ userInfo_ = value;
+ }
+ onChanged();
+ } else {
+ userInfoBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ public Builder clearUserInfo() {
+ if (userInfoBuilder_ == null) {
+ userInfo_ = null;
+ onChanged();
+ } else {
+ userInfoBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder getUserInfoBuilder() {
+ bitField0_ |= 0x00000001;
+ onChanged();
+ return getUserInfoFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+ if (userInfoBuilder_ != null) {
+ return userInfoBuilder_.getMessageOrBuilder();
+ } else {
+ return userInfo_ == null ?
+ org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance() : userInfo_;
+ }
+ }
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+ org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>
+ getUserInfoFieldBuilder() {
+ if (userInfoBuilder_ == null) {
+ userInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+ org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
+ getUserInfo(),
+ getParentForChildren(),
+ isClean());
+ userInfo_ = null;
+ }
+ return userInfoBuilder_;
+ }
+
+ private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_ =
+ java.util.Collections.emptyList();
+ private void ensureRegionInfoIsMutable() {
+ if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+ regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo>(regionInfo_);
+ bitField0_ |= 0x00000002;
+ }
+ }
+
+ private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
+
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
+ if (regionInfoBuilder_ == null) {
+ return java.util.Collections.unmodifiableList(regionInfo_);
+ } else {
+ return regionInfoBuilder_.getMessageList();
+ }
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public int getRegionInfoCount() {
+ if (regionInfoBuilder_ == null) {
+ return regionInfo_.size();
+ } else {
+ return regionInfoBuilder_.getCount();
+ }
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
+ if (regionInfoBuilder_ == null) {
+ return regionInfo_.get(index);
+ } else {
+ return regionInfoBuilder_.getMessage(index);
+ }
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public Builder setRegionInfo(
+ int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+ if (regionInfoBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureRegionInfoIsMutable();
+ regionInfo_.set(index, value);
+ onChanged();
+ } else {
+ regionInfoBuilder_.setMessage(index, value);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public Builder setRegionInfo(
+ int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+ if (regionInfoBuilder_ == null) {
+ ensureRegionInfoIsMutable();
+ regionInfo_.set(index, builderForValue.build());
+ onChanged();
+ } else {
+ regionInfoBuilder_.setMessage(index, builderForValue.build());
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public Builder addRegionInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+ if (regionInfoBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureRegionInfoIsMutable();
+ regionInfo_.add(value);
+ onChanged();
+ } else {
+ regionInfoBuilder_.addMessage(value);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public Builder addRegionInfo(
+ int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+ if (regionInfoBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureRegionInfoIsMutable();
+ regionInfo_.add(index, value);
+ onChanged();
+ } else {
+ regionInfoBuilder_.addMessage(index, value);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public Builder addRegionInfo(
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+ if (regionInfoBuilder_ == null) {
+ ensureRegionInfoIsMutable();
+ regionInfo_.add(builderForValue.build());
+ onChanged();
+ } else {
+ regionInfoBuilder_.addMessage(builderForValue.build());
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public Builder addRegionInfo(
+ int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+ if (regionInfoBuilder_ == null) {
+ ensureRegionInfoIsMutable();
+ regionInfo_.add(index, builderForValue.build());
+ onChanged();
+ } else {
+ regionInfoBuilder_.addMessage(index, builderForValue.build());
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public Builder addAllRegionInfo(
+ java.lang.Iterable<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> values) {
+ if (regionInfoBuilder_ == null) {
+ ensureRegionInfoIsMutable();
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
+ values, regionInfo_);
+ onChanged();
+ } else {
+ regionInfoBuilder_.addAllMessages(values);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public Builder clearRegionInfo() {
+ if (regionInfoBuilder_ == null) {
+ regionInfo_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000002);
+ onChanged();
+ } else {
+ regionInfoBuilder_.clear();
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public Builder removeRegionInfo(int index) {
+ if (regionInfoBuilder_ == null) {
+ ensureRegionInfoIsMutable();
+ regionInfo_.remove(index);
+ onChanged();
+ } else {
+ regionInfoBuilder_.remove(index);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder(
+ int index) {
+ return getRegionInfoFieldBuilder().getBuilder(index);
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+ int index) {
+ if (regionInfoBuilder_ == null) {
+ return regionInfo_.get(index); } else {
+ return regionInfoBuilder_.getMessageOrBuilder(index);
+ }
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
+ getRegionInfoOrBuilderList() {
+ if (regionInfoBuilder_ != null) {
+ return regionInfoBuilder_.getMessageOrBuilderList();
+ } else {
+ return java.util.Collections.unmodifiableList(regionInfo_);
+ }
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder addRegionInfoBuilder() {
+ return getRegionInfoFieldBuilder().addBuilder(
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance());
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder addRegionInfoBuilder(
+ int index) {
+ return getRegionInfoFieldBuilder().addBuilder(
+ index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance());
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+ */
+ public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder>
+ getRegionInfoBuilderList() {
+ return getRegionInfoFieldBuilder().getBuilderList();
+ }
+ private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
+ getRegionInfoFieldBuilder() {
+ if (regionInfoBuilder_ == null) {
+ regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
+ regionInfo_,
+ ((bitField0_ & 0x00000002) == 0x00000002),
+ getParentForChildren(),
+ isClean());
+ regionInfo_ = null;
+ }
+ return regionInfoBuilder_;
+ }
+
+ private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo mergedRegionInfo_ = null;
+ private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> mergedRegionInfoBuilder_;
+ /**
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
+ */
+ public boolean hasMergedRegionInfo() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getMergedRegionInfo() {
+ if (mergedRegionInfoBuilder_ == null) {
+ return mergedRegionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : mergedRegionInfo_;
+ } else {
+ return mergedRegionInfoBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
+ */
+ public Builder setMergedRegionInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+ if (mergedRegionInfoBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ mergedRegionInfo_ = value;
+ onChanged();
+ } else {
+ mergedRegionInfoBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000004;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
+ */
+ public Builder setMergedRegionInfo(
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+ if (mergedRegionInfoBuilder_ == null) {
+ mergedRegionInfo_ = builderForValue.build();
+ onChanged();
+ } else {
+ mergedRegionInfoBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000004;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
+ */
+ public Builder mergeMergedRegionInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+ if (mergedRegionInfoBuilder_ == null) {
+ if (((bitField0_ & 0x00000004) == 0x00000004) &&
+ mergedRegionInfo_ != null &&
+ mergedRegionInfo_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance()) {
+ mergedRegionInfo_ =
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.newBuilder(mergedRegionInfo_).mergeFrom(value).buildPartial();
+ } else {
+ mergedRegionInfo_ = value;
+ }
+ onChanged();
+ } else {
+ mergedRegionInfoBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000004;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
+ */
+ public Builder clearMergedRegionInfo() {
+ if (mergedRegionInfoBuilder_ == null) {
+ mergedRegionInfo_ = null;
+ onChanged();
+ } else {
+ mergedRegionInfoBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000004);
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder getMergedRegionInfoBuilder() {
+ bitField0_ |= 0x00000004;
+ onChanged();
+ return getMergedRegionInfoFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getMergedRegionInfoOrBuilder() {
+ if (mergedRegionInfoBuilder_ != null) {
+ return mergedRegionInfoBuilder_.getMessageOrBuilder();
+ } else {
+ return mergedRegionInfo_ == null ?
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : mergedRegionInfo_;
+ }
+ }
+ /**
+ * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
+ */
+ private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
+ getMergedRegionInfoFieldBuilder() {
+ if (mergedRegionInfoBuilder_ == null) {
+ mergedRegionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
+ getMergedRegionInfo(),
+ getParentForChildren(),
+ isClean());
+ mergedRegionInfo_ = null;
+ }
+ return mergedRegionInfoBuilder_;
+ }
+
+ private boolean forcible_ ;
+ /**
+ * <code>optional bool forcible = 4 [default = false];</code>
+ */
+ public boolean hasForcible() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional bool forcible = 4 [default = false];</code>
+ */
+ public boolean getForcible() {
+ return forcible_;
+ }
+ /**
+ * <code>optional bool forcible = 4 [default = false];</code>
+ */
+ public Builder setForcible(boolean value) {
+ bitField0_ |= 0x00000008;
+ forcible_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional bool forcible = 4 [default = false];</code>
+ */
+ public Builder clearForcible() {
+ bitField0_ = (bitField0_ & ~0x00000008);
+ forcible_ = false;
+ onChanged();
+ return this;
+ }
+ public final Builder setUnknownFields(
+ final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+ return super.setUnknownFields(unknownFields);
+ }
+
+ public final Builder mergeUnknownFields(
+ final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+ return super.mergeUnknownFields(unknownFields);
+ }
+
+
+ // @@protoc_insertion_point(builder_scope:hbase.pb.MergeTableRegionsStateData)
+ }
+
+ // @@protoc_insertion_point(class_scope:hbase.pb.MergeTableRegionsStateData)
+ private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData DEFAULT_INSTANCE;
+ static {
+ DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData();
+ }
+
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData getDefaultInstance() {
+ return DEFAULT_INSTANCE;
+ }
+
+ @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<MergeTableRegionsStateData>
+ PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<MergeTableRegionsStateData>() {
+ public MergeTableRegionsStateData parsePartialFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return new MergeTableRegionsStateData(input, extensionRegistry);
+ }
+ };
+
+ public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<MergeTableRegionsStateData> parser() {
+ return PARSER;
+ }
+
+ @java.lang.Override
+ public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<MergeTableRegionsStateData> getParserForType() {
+ return PARSER;
+ }
+
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData getDefaultInstanceForType() {
+ return DEFAULT_INSTANCE;
+ }
+
+ }
+
+ public interface SplitTableRegionStateDataOrBuilder extends
+ // @@protoc_insertion_point(interface_extends:hbase.pb.SplitTableRegionStateData)
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ boolean hasUserInfo();
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation getUserInfo();
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
+
+ /**
+ * <code>required .hbase.pb.RegionInfo parent_region_info = 2;</code>
+ */
+ boolean hasParentRegionInfo();
+ /**
+ * <code>required .hbase.pb.RegionInfo parent_region_info = 2;</code>
+ */
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getParentRegionInfo();
+ /**
+ * <code>required .hbase.pb.RegionInfo parent_region_info = 2;</code>
+ */
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getParentRegionInfoOrBuilder();
+
+ /**
+ * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ */
+ java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo>
+ getChildRegionInfoList();
+ /**
+ * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ */
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getChildRegionInfo(int index);
+ /**
+ * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ */
+ int getChildRegionInfoCount();
+ /**
+ * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ */
+ java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
+ getChildRegionInfoOrBuilderList();
+ /**
+ * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ */
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getChildRegionInfoOrBuilder(
+ int index);
+ }
+ /**
+ * Protobuf type {@code hbase.pb.SplitTableRegionStateData}
+ */
+ public static final class SplitTableRegionStateData extends
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+ // @@protoc_insertion_point(message_implements:hbase.pb.SplitTableRegionStateData)
+ SplitTableRegionStateDataOrBuilder {
+ // Use SplitTableRegionStateData.newBuilder() to construct.
+ private SplitTableRegionStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+ super(builder);
+ }
+ private SplitTableRegionStateData() {
+ childRegionInfo_ = java.util.Collections.emptyList();
+ }
+
+ @java.lang.Override
+ public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private SplitTableRegionStateData(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ this();
+ int mutable_bitField0_ = 0;
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ case 10: {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ subBuilder = userInfo_.toBuilder();
+ }
+ userInfo_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(userInfo_);
+ userInfo_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000001;
+ break;
+ }
+ case 18: {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ subBuilder = parentRegionInfo_.toBuilder();
+ }
+ parentRegionInfo_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(parentRegionInfo_);
+ parentRegionInfo_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000002;
+ break;
+ }
+ case 26: {
+ if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+ childRegionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo>();
+ mutable_bitField0_ |= 0x00000004;
+ }
+ childRegionInfo_.add(
+ input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
+ break;
+ }
+ }
+ }
+ } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+ e).setUnfinishedMessage(this);
+ } finally {
+ if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+ childRegionInfo_ = java.util.Collections.unmodifiableList(childRegionInfo_);
+ }
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_SplitTableRegionStateData_descriptor;
+ }
+
+ protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_SplitTableRegionStateData_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionStateData.Builder.class);
+ }
+
+ private int bitField0_;
+ public static final int USER_INFO_FIELD_NUMBER = 1;
+ private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_;
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ public boolean hasUserInfo() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+ return userInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance() : userInfo_;
+ }
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+ return userInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance() : userInfo_;
+ }
+
+ public static final int PARENT_REGION_INFO_FIELD_NUMBER = 2;
+ private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parentRegionInfo_;
+ /**
+ * <code>required .hbase.pb.RegionInfo parent_region_info = 2;</code>
+ */
+ public boolean hasParentRegionInfo() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>required .hbase.pb.RegionInfo parent_region_info = 2;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getParentRegionInfo() {
+ return parentRegionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : parentRegionInfo_;
+ }
+ /**
+ * <code>required .hbase.pb.RegionInfo parent_region_info = 2;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getParentRegionInfoOrBuilder() {
+ return parentRegionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : parentRegionInfo_;
+ }
+
+ public static final int CHILD_REGION_INFO_FIELD_NUMBER = 3;
+ private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> childRegionInfo_;
+ /**
+ * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ */
+ public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> getChildRegionInfoList() {
+ return childRegionInfo_;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ */
+ public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
+ getChildRegionInfoOrBuilderList() {
+ return childRegionInfo_;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ */
+ public int getChildRegionInfoCount() {
+ return childRegionInfo_.size();
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getChildRegionInfo(int index) {
+ return childRegionInfo_.get(index);
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo child_region_info = 3;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getChildRegionInfoOrBuilder(
+ int index) {
+ return childRegionInfo_.get(index);
+ }
+
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized == 1) return true;
+ if (isInitialized == 0) return false;
+
+ if (!hasUserInfo()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasParentRegionInfo()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!getUserInfo().isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!getParentRegionInfo().isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ for (int i = 0; i < getChildRegionInfoCount(); i++) {
+ if (!getChildRegionInfo(i).isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeMessage(1, getUserInfo());
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeMessage(2, getParentRegionInfo());
+ }
+ for (int i = 0; i < childRegionInfo_.size(); i++) {
+ output.writeMessage(3, childRegionInfo_.get(i));
}
unknownFields.writeTo(output);
}
@@ -23985,6 +25427,11 @@ public final class MasterProcedureProtos {
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
internal_static_hbase_pb_DispatchMergingRegionsStateData_fieldAccessorTable;
private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+ internal_static_hbase_pb_MergeTableRegionsStateData_descriptor;
+ private static final
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+ internal_static_hbase_pb_MergeTableRegionsStateData_fieldAccessorTable;
+ private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
internal_static_hbase_pb_SplitTableRegionStateData_descriptor;
private static final
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
@@ -24082,124 +25529,141 @@ public final class MasterProcedureProtos {
"e.pb.UserInformation\022\'\n\ntable_name\030\002 \002(\013" +
"2\023.hbase.pb.TableName\022)\n\013region_info\030\003 \003" +
"(\0132\024.hbase.pb.RegionInfo\022\020\n\010forcible\030\004 \001" +
- "(\010\"\254\001\n\031SplitTableRegionStateData\022,\n\tuser",
- "_info\030\001 \002(\0132\031.hbase.pb.UserInformation\0220" +
- "\n\022parent_region_info\030\002 \002(\0132\024.hbase.pb.Re" +
- "gionInfo\022/\n\021child_region_info\030\003 \003(\0132\024.hb" +
- "ase.pb.RegionInfo\"\201\002\n\024ServerCrashStateDa" +
- "ta\022)\n\013server_name\030\001 \002(\0132\024.hbase.pb.Serve" +
- "rName\022\036\n\026distributed_log_replay\030\002 \001(\010\0227\n" +
- "\031regions_on_crashed_server\030\003 \003(\0132\024.hbase" +
- ".pb.RegionInfo\022.\n\020regions_assigned\030\004 \003(\013" +
- "2\024.hbase.pb.RegionInfo\022\025\n\rcarrying_meta\030" +
- "\005 \001(\010\022\036\n\020should_split_wal\030\006 \001(\010:\004true*\330\001",
- "\n\020CreateTableState\022\036\n\032CREATE_TABLE_PRE_O" +
- "PERATION\020\001\022 \n\034CREATE_TABLE_WRITE_FS_LAYO" +
- "UT\020\002\022\034\n\030CREATE_TABLE_ADD_TO_META\020\003\022\037\n\033CR" +
- "EATE_TABLE_ASSIGN_REGIONS\020\004\022\"\n\036CREATE_TA" +
- "BLE_UPDATE_DESC_CACHE\020\005\022\037\n\033CREATE_TABLE_" +
- "POST_OPERATION\020\006*\207\002\n\020ModifyTableState\022\030\n" +
- "\024MODIFY_TABLE_PREPARE\020\001\022\036\n\032MODIFY_TABLE_" +
- "PRE_OPERATION\020\002\022(\n$MODIFY_TABLE_UPDATE_T" +
- "ABLE_DESCRIPTOR\020\003\022&\n\"MODIFY_TABLE_REMOVE" +
- "_REPLICA_COLUMN\020\004\022!\n\035MODIFY_TABLE_DELETE",
- "_FS_LAYOUT\020\005\022\037\n\033MODIFY_TABLE_POST_OPERAT" +
- "ION\020\006\022#\n\037MODIFY_TABLE_REOPEN_ALL_REGIONS" +
- "\020\007*\212\002\n\022TruncateTableState\022 \n\034TRUNCATE_TA" +
- "BLE_PRE_OPERATION\020\001\022#\n\037TRUNCATE_TABLE_RE" +
- "MOVE_FROM_META\020\002\022\"\n\036TRUNCATE_TABLE_CLEAR" +
- "_FS_LAYOUT\020\003\022#\n\037TRUNCATE_TABLE_CREATE_FS" +
- "_LAYOUT\020\004\022\036\n\032TRUNCATE_TABLE_ADD_TO_META\020" +
- "\005\022!\n\035TRUNCATE_TABLE_ASSIGN_REGIONS\020\006\022!\n\035" +
- "TRUNCATE_TABLE_POST_OPERATION\020\007*\337\001\n\020Dele" +
- "teTableState\022\036\n\032DELETE_TABLE_PRE_OPERATI",
- "ON\020\001\022!\n\035DELETE_TABLE_REMOVE_FROM_META\020\002\022" +
- " \n\034DELETE_TABLE_CLEAR_FS_LAYOUT\020\003\022\"\n\036DEL" +
- "ETE_TABLE_UPDATE_DESC_CACHE\020\004\022!\n\035DELETE_" +
- "TABLE_UNASSIGN_REGIONS\020\005\022\037\n\033DELETE_TABLE" +
- "_POST_OPERATION\020\006*\320\001\n\024CreateNamespaceSta" +
- "te\022\034\n\030CREATE_NAMESPACE_PREPARE\020\001\022%\n!CREA" +
- "TE_NAMESPACE_CREATE_DIRECTORY\020\002\022)\n%CREAT" +
- "E_NAMESPACE_INSERT_INTO_NS_TABLE\020\003\022\036\n\032CR" +
- "EATE_NAMESPACE_UPDATE_ZK\020\004\022(\n$CREATE_NAM" +
- "ESPACE_SET_NAMESPACE_QUOTA\020\005*z\n\024ModifyNa",
- "mespaceState\022\034\n\030MODIFY_NAMESPACE_PREPARE" +
- "\020\001\022$\n MODIFY_NAMESPACE_UPDATE_NS_TABLE\020\002" +
- "\022\036\n\032MODIFY_NAMESPACE_UPDATE_ZK\020\003*\332\001\n\024Del" +
- "eteNamespaceState\022\034\n\030DELETE_NAMESPACE_PR" +
- "EPARE\020\001\022)\n%DELETE_NAMESPACE_DELETE_FROM_" +
- "NS_TABLE\020\002\022#\n\037DELETE_NAMESPACE_REMOVE_FR" +
- "OM_ZK\020\003\022\'\n#DELETE_NAMESPACE_DELETE_DIREC" +
- "TORIES\020\004\022+\n\'DELETE_NAMESPACE_REMOVE_NAME" +
- "SPACE_QUOTA\020\005*\331\001\n\024AddColumnFamilyState\022\035" +
- "\n\031ADD_COLUMN_FAMILY_PREPARE\020\001\022#\n\037ADD_COL",
- "UMN_FAMILY_PRE_OPERATION\020\002\022-\n)ADD_COLUMN" +
- "_FAMILY_UPDATE_TABLE_DESCRIPTOR\020\003\022$\n ADD" +
- "_COLUMN_FAMILY_POST_OPERATION\020\004\022(\n$ADD_C" +
- "OLUMN_FAMILY_REOPEN_ALL_REGIONS\020\005*\353\001\n\027Mo" +
- "difyColumnFamilyState\022 \n\034MODIFY_COLUMN_F" +
- "AMILY_PREPARE\020\001\022&\n\"MODIFY_COLUMN_FAMILY_" +
- "PRE_OPERATION\020\002\0220\n,MODIFY_COLUMN_FAMILY_" +
- "UPDATE_TABLE_DESCRIPTOR\020\003\022\'\n#MODIFY_COLU" +
- "MN_FAMILY_POST_OPERATION\020\004\022+\n\'MODIFY_COL" +
- "UMN_FAMILY_REOPEN_ALL_REGIONS\020\005*\226\002\n\027Dele",
- "teColumnFamilyState\022 \n\034DELETE_COLUMN_FAM" +
- "ILY_PREPARE\020\001\022&\n\"DELETE_COLUMN_FAMILY_PR" +
- "E_OPERATION\020\002\0220\n,DELETE_COLUMN_FAMILY_UP" +
- "DATE_TABLE_DESCRIPTOR\020\003\022)\n%DELETE_COLUMN" +
- "_FAMILY_DELETE_FS_LAYOUT\020\004\022\'\n#DELETE_COL" +
- "UMN_FAMILY_POST_OPERATION\020\005\022+\n\'DELETE_CO" +
- "LUMN_FAMILY_REOPEN_ALL_REGIONS\020\006*\350\001\n\020Ena" +
- "bleTableState\022\030\n\024ENABLE_TABLE_PREPARE\020\001\022" +
- "\036\n\032ENABLE_TABLE_PRE_OPERATION\020\002\022)\n%ENABL" +
- "E_TABLE_SET_ENABLING_TABLE_STATE\020\003\022$\n EN",
- "ABLE_TABLE_MARK_REGIONS_ONLINE\020\004\022(\n$ENAB" +
- "LE_TABLE_SET_EN
<TRUNCATED>
[5/6] hbase git commit: HBASE-16119 Procedure v2 - Reimplement Merge
region (Stephen Yuan Jiang)
Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
index 6f20fcc..576c739 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
@@ -9414,41 +9414,53 @@ public final class AdminProtos {
}
- public interface CloseRegionForSplitRequestOrBuilder extends
- // @@protoc_insertion_point(interface_extends:hbase.pb.CloseRegionForSplitRequest)
+ public interface CloseRegionForSplitOrMergeRequestOrBuilder extends
+ // @@protoc_insertion_point(interface_extends:hbase.pb.CloseRegionForSplitOrMergeRequest)
org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
- boolean hasRegion();
+ java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier>
+ getRegionList();
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
- org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion();
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index);
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
- org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder();
+ int getRegionCount();
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
+ getRegionOrBuilderList();
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
+ int index);
}
/**
* <pre>
**
- * Closes the specified region and create
- * child region.
+ * Closes the specified region(s) for
+ * split or merge
* </pre>
*
- * Protobuf type {@code hbase.pb.CloseRegionForSplitRequest}
+ * Protobuf type {@code hbase.pb.CloseRegionForSplitOrMergeRequest}
*/
- public static final class CloseRegionForSplitRequest extends
+ public static final class CloseRegionForSplitOrMergeRequest extends
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
- // @@protoc_insertion_point(message_implements:hbase.pb.CloseRegionForSplitRequest)
- CloseRegionForSplitRequestOrBuilder {
- // Use CloseRegionForSplitRequest.newBuilder() to construct.
- private CloseRegionForSplitRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+ // @@protoc_insertion_point(message_implements:hbase.pb.CloseRegionForSplitOrMergeRequest)
+ CloseRegionForSplitOrMergeRequestOrBuilder {
+ // Use CloseRegionForSplitOrMergeRequest.newBuilder() to construct.
+ private CloseRegionForSplitOrMergeRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
super(builder);
}
- private CloseRegionForSplitRequest() {
+ private CloseRegionForSplitOrMergeRequest() {
+ region_ = java.util.Collections.emptyList();
}
@java.lang.Override
@@ -9456,7 +9468,7 @@ public final class AdminProtos {
getUnknownFields() {
return this.unknownFields;
}
- private CloseRegionForSplitRequest(
+ private CloseRegionForSplitOrMergeRequest(
org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
@@ -9480,16 +9492,12 @@ public final class AdminProtos {
break;
}
case 10: {
- org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder subBuilder = null;
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- subBuilder = region_.toBuilder();
- }
- region_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.PARSER, extensionRegistry);
- if (subBuilder != null) {
- subBuilder.mergeFrom(region_);
- region_ = subBuilder.buildPartial();
+ if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+ region_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier>();
+ mutable_bitField0_ |= 0x00000001;
}
- bitField0_ |= 0x00000001;
+ region_.add(
+ input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.PARSER, extensionRegistry));
break;
}
}
@@ -9500,42 +9508,58 @@ public final class AdminProtos {
throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
e).setUnfinishedMessage(this);
} finally {
+ if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+ region_ = java.util.Collections.unmodifiableList(region_);
+ }
this.unknownFields = unknownFields.build();
makeExtensionsImmutable();
}
}
public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitRequest_descriptor;
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_descriptor;
}
protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
internalGetFieldAccessorTable() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitRequest_fieldAccessorTable
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_fieldAccessorTable
.ensureFieldAccessorsInitialized(
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest.Builder.class);
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.Builder.class);
}
- private int bitField0_;
public static final int REGION_FIELD_NUMBER = 1;
- private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_;
+ private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> region_;
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
- public boolean hasRegion() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
+ public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> getRegionList() {
+ return region_;
}
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
- public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() {
- return region_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : region_;
+ public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
+ getRegionOrBuilderList() {
+ return region_;
}
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
- public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() {
- return region_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : region_;
+ public int getRegionCount() {
+ return region_.size();
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index) {
+ return region_.get(index);
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
+ int index) {
+ return region_.get(index);
}
private byte memoizedIsInitialized = -1;
@@ -9544,13 +9568,11 @@ public final class AdminProtos {
if (isInitialized == 1) return true;
if (isInitialized == 0) return false;
- if (!hasRegion()) {
- memoizedIsInitialized = 0;
- return false;
- }
- if (!getRegion().isInitialized()) {
- memoizedIsInitialized = 0;
- return false;
+ for (int i = 0; i < getRegionCount(); i++) {
+ if (!getRegion(i).isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
}
memoizedIsInitialized = 1;
return true;
@@ -9558,8 +9580,8 @@ public final class AdminProtos {
public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- output.writeMessage(1, getRegion());
+ for (int i = 0; i < region_.size(); i++) {
+ output.writeMessage(1, region_.get(i));
}
unknownFields.writeTo(output);
}
@@ -9569,9 +9591,9 @@ public final class AdminProtos {
if (size != -1) return size;
size = 0;
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ for (int i = 0; i < region_.size(); i++) {
size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
- .computeMessageSize(1, getRegion());
+ .computeMessageSize(1, region_.get(i));
}
size += unknownFields.getSerializedSize();
memoizedSize = size;
@@ -9584,17 +9606,14 @@ public final class AdminProtos {
if (obj == this) {
return true;
}
- if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest)) {
+ if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest)) {
return super.equals(obj);
}
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest) obj;
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest) obj;
boolean result = true;
- result = result && (hasRegion() == other.hasRegion());
- if (hasRegion()) {
- result = result && getRegion()
- .equals(other.getRegion());
- }
+ result = result && getRegionList()
+ .equals(other.getRegionList());
result = result && unknownFields.equals(other.unknownFields);
return result;
}
@@ -9606,67 +9625,67 @@ public final class AdminProtos {
}
int hash = 41;
hash = (19 * hash) + getDescriptorForType().hashCode();
- if (hasRegion()) {
+ if (getRegionCount() > 0) {
hash = (37 * hash) + REGION_FIELD_NUMBER;
- hash = (53 * hash) + getRegion().hashCode();
+ hash = (53 * hash) + getRegionList().hashCode();
}
hash = (29 * hash) + unknownFields.hashCode();
memoizedHashCode = hash;
return hash;
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest parseFrom(
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(
org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest parseFrom(
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(
org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data, extensionRegistry);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest parseFrom(byte[] data)
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(byte[] data)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest parseFrom(
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(
byte[] data,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data, extensionRegistry);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest parseFrom(java.io.InputStream input)
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(java.io.InputStream input)
throws java.io.IOException {
return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
.parseWithIOException(PARSER, input);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest parseFrom(
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(
java.io.InputStream input,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
.parseWithIOException(PARSER, input, extensionRegistry);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest parseDelimitedFrom(java.io.InputStream input)
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseDelimitedFrom(java.io.InputStream input)
throws java.io.IOException {
return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
.parseDelimitedWithIOException(PARSER, input);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest parseDelimitedFrom(
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseDelimitedFrom(
java.io.InputStream input,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
.parseDelimitedWithIOException(PARSER, input, extensionRegistry);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest parseFrom(
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(
org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
throws java.io.IOException {
return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
.parseWithIOException(PARSER, input);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest parseFrom(
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(
org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
@@ -9678,7 +9697,7 @@ public final class AdminProtos {
public static Builder newBuilder() {
return DEFAULT_INSTANCE.toBuilder();
}
- public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest prototype) {
+ public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest prototype) {
return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
}
public Builder toBuilder() {
@@ -9695,29 +9714,29 @@ public final class AdminProtos {
/**
* <pre>
**
- * Closes the specified region and create
- * child region.
+ * Closes the specified region(s) for
+ * split or merge
* </pre>
*
- * Protobuf type {@code hbase.pb.CloseRegionForSplitRequest}
+ * Protobuf type {@code hbase.pb.CloseRegionForSplitOrMergeRequest}
*/
public static final class Builder extends
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
- // @@protoc_insertion_point(builder_implements:hbase.pb.CloseRegionForSplitRequest)
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequestOrBuilder {
+ // @@protoc_insertion_point(builder_implements:hbase.pb.CloseRegionForSplitOrMergeRequest)
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequestOrBuilder {
public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitRequest_descriptor;
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_descriptor;
}
protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
internalGetFieldAccessorTable() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitRequest_fieldAccessorTable
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_fieldAccessorTable
.ensureFieldAccessorsInitialized(
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest.Builder.class);
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.Builder.class);
}
- // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest.newBuilder()
+ // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.newBuilder()
private Builder() {
maybeForceBuilderInitialization();
}
@@ -9736,44 +9755,43 @@ public final class AdminProtos {
public Builder clear() {
super.clear();
if (regionBuilder_ == null) {
- region_ = null;
+ region_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000001);
} else {
regionBuilder_.clear();
}
- bitField0_ = (bitField0_ & ~0x00000001);
return this;
}
public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitRequest_descriptor;
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_descriptor;
}
- public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest getDefaultInstanceForType() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest.getDefaultInstance();
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest getDefaultInstanceForType() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.getDefaultInstance();
}
- public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest build() {
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest result = buildPartial();
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest build() {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(result);
}
return result;
}
- public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest buildPartial() {
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest(this);
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest buildPartial() {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest(this);
int from_bitField0_ = bitField0_;
- int to_bitField0_ = 0;
- if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
- to_bitField0_ |= 0x00000001;
- }
if (regionBuilder_ == null) {
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ region_ = java.util.Collections.unmodifiableList(region_);
+ bitField0_ = (bitField0_ & ~0x00000001);
+ }
result.region_ = region_;
} else {
result.region_ = regionBuilder_.build();
}
- result.bitField0_ = to_bitField0_;
onBuilt();
return result;
}
@@ -9805,18 +9823,41 @@ public final class AdminProtos {
return (Builder) super.addRepeatedField(field, value);
}
public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
- if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest) {
- return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest)other);
+ if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest) {
+ return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest)other);
} else {
super.mergeFrom(other);
return this;
}
}
- public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest other) {
- if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest.getDefaultInstance()) return this;
- if (other.hasRegion()) {
- mergeRegion(other.getRegion());
+ public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest other) {
+ if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.getDefaultInstance()) return this;
+ if (regionBuilder_ == null) {
+ if (!other.region_.isEmpty()) {
+ if (region_.isEmpty()) {
+ region_ = other.region_;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ } else {
+ ensureRegionIsMutable();
+ region_.addAll(other.region_);
+ }
+ onChanged();
+ }
+ } else {
+ if (!other.region_.isEmpty()) {
+ if (regionBuilder_.isEmpty()) {
+ regionBuilder_.dispose();
+ regionBuilder_ = null;
+ region_ = other.region_;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ regionBuilder_ =
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+ getRegionFieldBuilder() : null;
+ } else {
+ regionBuilder_.addAllMessages(other.region_);
+ }
+ }
}
this.mergeUnknownFields(other.unknownFields);
onChanged();
@@ -9824,11 +9865,10 @@ public final class AdminProtos {
}
public final boolean isInitialized() {
- if (!hasRegion()) {
- return false;
- }
- if (!getRegion().isInitialized()) {
- return false;
+ for (int i = 0; i < getRegionCount(); i++) {
+ if (!getRegion(i).isInitialized()) {
+ return false;
+ }
}
return true;
}
@@ -9837,11 +9877,11 @@ public final class AdminProtos {
org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest parsedMessage = null;
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parsedMessage = null;
try {
parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
} catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
- parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest) e.getUnfinishedMessage();
+ parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest) e.getUnfinishedMessage();
throw e.unwrapIOException();
} finally {
if (parsedMessage != null) {
@@ -9852,117 +9892,239 @@ public final class AdminProtos {
}
private int bitField0_;
- private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
- private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+ private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> region_ =
+ java.util.Collections.emptyList();
+ private void ensureRegionIsMutable() {
+ if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+ region_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier>(region_);
+ bitField0_ |= 0x00000001;
+ }
+ }
+
+ private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
+
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
- public boolean hasRegion() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
+ public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> getRegionList() {
+ if (regionBuilder_ == null) {
+ return java.util.Collections.unmodifiableList(region_);
+ } else {
+ return regionBuilder_.getMessageList();
+ }
}
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
- public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() {
+ public int getRegionCount() {
if (regionBuilder_ == null) {
- return region_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : region_;
+ return region_.size();
} else {
- return regionBuilder_.getMessage();
+ return regionBuilder_.getCount();
}
}
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
- public Builder setRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index) {
+ if (regionBuilder_ == null) {
+ return region_.get(index);
+ } else {
+ return regionBuilder_.getMessage(index);
+ }
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public Builder setRegion(
+ int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
if (regionBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
}
- region_ = value;
+ ensureRegionIsMutable();
+ region_.set(index, value);
onChanged();
} else {
- regionBuilder_.setMessage(value);
+ regionBuilder_.setMessage(index, value);
}
- bitField0_ |= 0x00000001;
return this;
}
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
public Builder setRegion(
- org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
+ int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
if (regionBuilder_ == null) {
- region_ = builderForValue.build();
+ ensureRegionIsMutable();
+ region_.set(index, builderForValue.build());
onChanged();
} else {
- regionBuilder_.setMessage(builderForValue.build());
+ regionBuilder_.setMessage(index, builderForValue.build());
}
- bitField0_ |= 0x00000001;
return this;
}
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
- public Builder mergeRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+ public Builder addRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
if (regionBuilder_ == null) {
- if (((bitField0_ & 0x00000001) == 0x00000001) &&
- region_ != null &&
- region_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance()) {
- region_ =
- org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.newBuilder(region_).mergeFrom(value).buildPartial();
- } else {
- region_ = value;
+ if (value == null) {
+ throw new NullPointerException();
}
+ ensureRegionIsMutable();
+ region_.add(value);
onChanged();
} else {
- regionBuilder_.mergeFrom(value);
+ regionBuilder_.addMessage(value);
}
- bitField0_ |= 0x00000001;
return this;
}
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public Builder addRegion(
+ int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+ if (regionBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureRegionIsMutable();
+ region_.add(index, value);
+ onChanged();
+ } else {
+ regionBuilder_.addMessage(index, value);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public Builder addRegion(
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
+ if (regionBuilder_ == null) {
+ ensureRegionIsMutable();
+ region_.add(builderForValue.build());
+ onChanged();
+ } else {
+ regionBuilder_.addMessage(builderForValue.build());
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public Builder addRegion(
+ int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
+ if (regionBuilder_ == null) {
+ ensureRegionIsMutable();
+ region_.add(index, builderForValue.build());
+ onChanged();
+ } else {
+ regionBuilder_.addMessage(index, builderForValue.build());
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public Builder addAllRegion(
+ java.lang.Iterable<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> values) {
+ if (regionBuilder_ == null) {
+ ensureRegionIsMutable();
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
+ values, region_);
+ onChanged();
+ } else {
+ regionBuilder_.addAllMessages(values);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
public Builder clearRegion() {
if (regionBuilder_ == null) {
- region_ = null;
+ region_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000001);
onChanged();
} else {
regionBuilder_.clear();
}
- bitField0_ = (bitField0_ & ~0x00000001);
return this;
}
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
- public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBuilder() {
- bitField0_ |= 0x00000001;
- onChanged();
- return getRegionFieldBuilder().getBuilder();
+ public Builder removeRegion(int index) {
+ if (regionBuilder_ == null) {
+ ensureRegionIsMutable();
+ region_.remove(index);
+ onChanged();
+ } else {
+ regionBuilder_.remove(index);
+ }
+ return this;
}
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
- public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() {
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBuilder(
+ int index) {
+ return getRegionFieldBuilder().getBuilder(index);
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
+ int index) {
+ if (regionBuilder_ == null) {
+ return region_.get(index); } else {
+ return regionBuilder_.getMessageOrBuilder(index);
+ }
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
+ getRegionOrBuilderList() {
if (regionBuilder_ != null) {
- return regionBuilder_.getMessageOrBuilder();
+ return regionBuilder_.getMessageOrBuilderList();
} else {
- return region_ == null ?
- org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : region_;
+ return java.util.Collections.unmodifiableList(region_);
}
}
/**
- * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
*/
- private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder addRegionBuilder() {
+ return getRegionFieldBuilder().addBuilder(
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance());
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder addRegionBuilder(
+ int index) {
+ return getRegionFieldBuilder().addBuilder(
+ index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance());
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder>
+ getRegionBuilderList() {
+ return getRegionFieldBuilder().getBuilderList();
+ }
+ private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
getRegionFieldBuilder() {
if (regionBuilder_ == null) {
- regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+ regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
- getRegion(),
+ region_,
+ ((bitField0_ & 0x00000001) == 0x00000001),
getParentForChildren(),
isClean());
region_ = null;
@@ -9980,46 +10142,46 @@ public final class AdminProtos {
}
- // @@protoc_insertion_point(builder_scope:hbase.pb.CloseRegionForSplitRequest)
+ // @@protoc_insertion_point(builder_scope:hbase.pb.CloseRegionForSplitOrMergeRequest)
}
- // @@protoc_insertion_point(class_scope:hbase.pb.CloseRegionForSplitRequest)
- private static final org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest DEFAULT_INSTANCE;
+ // @@protoc_insertion_point(class_scope:hbase.pb.CloseRegionForSplitOrMergeRequest)
+ private static final org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest DEFAULT_INSTANCE;
static {
- DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest();
+ DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest();
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest getDefaultInstance() {
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest getDefaultInstance() {
return DEFAULT_INSTANCE;
}
- @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitRequest>
- PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<CloseRegionForSplitRequest>() {
- public CloseRegionForSplitRequest parsePartialFrom(
+ @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitOrMergeRequest>
+ PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<CloseRegionForSplitOrMergeRequest>() {
+ public CloseRegionForSplitOrMergeRequest parsePartialFrom(
org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
- return new CloseRegionForSplitRequest(input, extensionRegistry);
+ return new CloseRegionForSplitOrMergeRequest(input, extensionRegistry);
}
};
- public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitRequest> parser() {
+ public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitOrMergeRequest> parser() {
return PARSER;
}
@java.lang.Override
- public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitRequest> getParserForType() {
+ public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitOrMergeRequest> getParserForType() {
return PARSER;
}
- public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest getDefaultInstanceForType() {
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest getDefaultInstanceForType() {
return DEFAULT_INSTANCE;
}
}
- public interface CloseRegionForSplitResponseOrBuilder extends
- // @@protoc_insertion_point(interface_extends:hbase.pb.CloseRegionForSplitResponse)
+ public interface CloseRegionForSplitOrMergeResponseOrBuilder extends
+ // @@protoc_insertion_point(interface_extends:hbase.pb.CloseRegionForSplitOrMergeResponse)
org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
/**
@@ -10032,17 +10194,17 @@ public final class AdminProtos {
boolean getClosed();
}
/**
- * Protobuf type {@code hbase.pb.CloseRegionForSplitResponse}
+ * Protobuf type {@code hbase.pb.CloseRegionForSplitOrMergeResponse}
*/
- public static final class CloseRegionForSplitResponse extends
+ public static final class CloseRegionForSplitOrMergeResponse extends
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
- // @@protoc_insertion_point(message_implements:hbase.pb.CloseRegionForSplitResponse)
- CloseRegionForSplitResponseOrBuilder {
- // Use CloseRegionForSplitResponse.newBuilder() to construct.
- private CloseRegionForSplitResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+ // @@protoc_insertion_point(message_implements:hbase.pb.CloseRegionForSplitOrMergeResponse)
+ CloseRegionForSplitOrMergeResponseOrBuilder {
+ // Use CloseRegionForSplitOrMergeResponse.newBuilder() to construct.
+ private CloseRegionForSplitOrMergeResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
super(builder);
}
- private CloseRegionForSplitResponse() {
+ private CloseRegionForSplitOrMergeResponse() {
closed_ = false;
}
@@ -10051,7 +10213,7 @@ public final class AdminProtos {
getUnknownFields() {
return this.unknownFields;
}
- private CloseRegionForSplitResponse(
+ private CloseRegionForSplitOrMergeResponse(
org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
@@ -10093,14 +10255,14 @@ public final class AdminProtos {
}
public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitResponse_descriptor;
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_descriptor;
}
protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
internalGetFieldAccessorTable() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitResponse_fieldAccessorTable
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_fieldAccessorTable
.ensureFieldAccessorsInitialized(
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse.Builder.class);
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.Builder.class);
}
private int bitField0_;
@@ -10161,10 +10323,10 @@ public final class AdminProtos {
if (obj == this) {
return true;
}
- if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse)) {
+ if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse)) {
return super.equals(obj);
}
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse) obj;
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse) obj;
boolean result = true;
result = result && (hasClosed() == other.hasClosed());
@@ -10193,58 +10355,58 @@ public final class AdminProtos {
return hash;
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse parseFrom(
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(
org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse parseFrom(
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(
org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data, extensionRegistry);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse parseFrom(byte[] data)
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(byte[] data)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse parseFrom(
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(
byte[] data,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data, extensionRegistry);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse parseFrom(java.io.InputStream input)
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(java.io.InputStream input)
throws java.io.IOException {
return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
.parseWithIOException(PARSER, input);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse parseFrom(
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(
java.io.InputStream input,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
.parseWithIOException(PARSER, input, extensionRegistry);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse parseDelimitedFrom(java.io.InputStream input)
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseDelimitedFrom(java.io.InputStream input)
throws java.io.IOException {
return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
.parseDelimitedWithIOException(PARSER, input);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse parseDelimitedFrom(
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseDelimitedFrom(
java.io.InputStream input,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
.parseDelimitedWithIOException(PARSER, input, extensionRegistry);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse parseFrom(
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(
org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
throws java.io.IOException {
return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
.parseWithIOException(PARSER, input);
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse parseFrom(
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(
org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
@@ -10256,7 +10418,7 @@ public final class AdminProtos {
public static Builder newBuilder() {
return DEFAULT_INSTANCE.toBuilder();
}
- public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse prototype) {
+ public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse prototype) {
return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
}
public Builder toBuilder() {
@@ -10271,25 +10433,25 @@ public final class AdminProtos {
return builder;
}
/**
- * Protobuf type {@code hbase.pb.CloseRegionForSplitResponse}
+ * Protobuf type {@code hbase.pb.CloseRegionForSplitOrMergeResponse}
*/
public static final class Builder extends
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
- // @@protoc_insertion_point(builder_implements:hbase.pb.CloseRegionForSplitResponse)
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponseOrBuilder {
+ // @@protoc_insertion_point(builder_implements:hbase.pb.CloseRegionForSplitOrMergeResponse)
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponseOrBuilder {
public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitResponse_descriptor;
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_descriptor;
}
protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
internalGetFieldAccessorTable() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitResponse_fieldAccessorTable
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_fieldAccessorTable
.ensureFieldAccessorsInitialized(
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse.Builder.class);
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.Builder.class);
}
- // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse.newBuilder()
+ // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.newBuilder()
private Builder() {
maybeForceBuilderInitialization();
}
@@ -10313,23 +10475,23 @@ public final class AdminProtos {
public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitResponse_descriptor;
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_descriptor;
}
- public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse getDefaultInstanceForType() {
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse.getDefaultInstance();
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse getDefaultInstanceForType() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.getDefaultInstance();
}
- public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse build() {
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse result = buildPartial();
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse build() {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(result);
}
return result;
}
- public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse buildPartial() {
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse(this);
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse buildPartial() {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse(this);
int from_bitField0_ = bitField0_;
int to_bitField0_ = 0;
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -10368,16 +10530,16 @@ public final class AdminProtos {
return (Builder) super.addRepeatedField(field, value);
}
public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
- if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse) {
- return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse)other);
+ if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse) {
+ return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse)other);
} else {
super.mergeFrom(other);
return this;
}
}
- public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse other) {
- if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse.getDefaultInstance()) return this;
+ public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse other) {
+ if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.getDefaultInstance()) return this;
if (other.hasClosed()) {
setClosed(other.getClosed());
}
@@ -10397,11 +10559,11 @@ public final class AdminProtos {
org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse parsedMessage = null;
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parsedMessage = null;
try {
parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
} catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
- parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse) e.getUnfinishedMessage();
+ parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse) e.getUnfinishedMessage();
throw e.unwrapIOException();
} finally {
if (parsedMessage != null) {
@@ -10454,39 +10616,39 @@ public final class AdminProtos {
}
- // @@protoc_insertion_point(builder_scope:hbase.pb.CloseRegionForSplitResponse)
+ // @@protoc_insertion_point(builder_scope:hbase.pb.CloseRegionForSplitOrMergeResponse)
}
- // @@protoc_insertion_point(class_scope:hbase.pb.CloseRegionForSplitResponse)
- private static final org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse DEFAULT_INSTANCE;
+ // @@protoc_insertion_point(class_scope:hbase.pb.CloseRegionForSplitOrMergeResponse)
+ private static final org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse DEFAULT_INSTANCE;
static {
- DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse();
+ DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse();
}
- public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse getDefaultInstance() {
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse getDefaultInstance() {
return DEFAULT_INSTANCE;
}
- @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitResponse>
- PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<CloseRegionForSplitResponse>() {
- public CloseRegionForSplitResponse parsePartialFrom(
+ @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitOrMergeResponse>
+ PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<CloseRegionForSplitOrMergeResponse>() {
+ public CloseRegionForSplitOrMergeResponse parsePartialFrom(
org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
- return new CloseRegionForSplitResponse(input, extensionRegistry);
+ return new CloseRegionForSplitOrMergeResponse(input, extensionRegistry);
}
};
- public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitResponse> parser() {
+ public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitOrMergeResponse> parser() {
return PARSER;
}
@java.lang.Override
- public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitResponse> getParserForType() {
+ public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitOrMergeResponse> getParserForType() {
return PARSER;
}
- public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse getDefaultInstanceForType() {
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse getDefaultInstanceForType() {
return DEFAULT_INSTANCE;
}
@@ -25173,7 +25335,7 @@ public final class AdminProtos {
* <code>optional .hbase.pb.TableName table_name = 1;</code>
*/
private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
- org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>
getTableNameFieldBuilder() {
if (tableNameBuilder_ == null) {
tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -25241,7 +25403,7 @@ public final class AdminProtos {
/**
* <code>repeated .hbase.pb.RegionLoad region_loads = 1;</code>
*/
- java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad>
+ java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad>
getRegionLoadsList();
/**
* <code>repeated .hbase.pb.RegionLoad region_loads = 1;</code>
@@ -25254,7 +25416,7 @@ public final class AdminProtos {
/**
* <code>repeated .hbase.pb.RegionLoad region_loads = 1;</code>
*/
- java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder>
+ java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder>
getRegionLoadsOrBuilderList();
/**
* <code>repeated .hbase.pb.RegionLoad region_loads = 1;</code>
@@ -25352,7 +25514,7 @@ public final class AdminProtos {
/**
* <code>repeated .hbase.pb.RegionLoad region_loads = 1;</code>
*/
- public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder>
+ public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder>
getRegionLoadsOrBuilderList() {
return regionLoads_;
}
@@ -25659,7 +25821,7 @@ public final class AdminProtos {
regionLoadsBuilder_ = null;
regionLoads_ = other.regionLoads_;
bitField0_ = (bitField0_ & ~0x00000001);
- regionLoadsBuilder_ =
+ regionLoadsBuilder_ =
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
getRegionLoadsFieldBuilder() : null;
} else {
@@ -25895,7 +26057,7 @@ public final class AdminProtos {
/**
* <code>repeated .hbase.pb.RegionLoad region_loads = 1;</code>
*/
- public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder>
+ public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder>
getRegionLoadsOrBuilderList() {
if (regionLoadsBuilder_ != null) {
return regionLoadsBuilder_.getMessageOrBuilderList();
@@ -25921,12 +26083,12 @@ public final class AdminProtos {
/**
* <code>repeated .hbase.pb.RegionLoad region_loads = 1;</code>
*/
- public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad.Builder>
+ public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad.Builder>
getRegionLoadsBuilderList() {
return getRegionLoadsFieldBuilder().getBuilderList();
}
private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
- org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder>
+ org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder>
getRegionLoadsFieldBuilder() {
if (regionLoadsBuilder_ == null) {
regionLoadsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -26045,12 +26207,12 @@ public final class AdminProtos {
org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse> done);
/**
- * <code>rpc CloseRegionForSplit(.hbase.pb.CloseRegionForSplitRequest) returns (.hbase.pb.CloseRegionForSplitResponse);</code>
+ * <code>rpc CloseRegionForSplitOrMerge(.hbase.pb.CloseRegionForSplitOrMergeRequest) returns (.hbase.pb.CloseRegionForSplitOrMergeResponse);</code>
*/
- public abstract void closeRegionForSplit(
+ public abstract void closeRegionForSplitOrMerge(
org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest request,
- org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse> done);
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest request,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse> done);
/**
* <code>rpc FlushRegion(.hbase.pb.FlushRegionRequest) returns (.hbase.pb.FlushRegionResponse);</code>
@@ -26202,11 +26364,11 @@ public final class AdminProtos {
}
@java.lang.Override
- public void closeRegionForSplit(
+ public void closeRegionForSplitOrMerge(
org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest request,
- org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse> done) {
- impl.closeRegionForSplit(controller, request, done);
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest request,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse> done) {
+ impl.closeRegionForSplitOrMerge(controller, request, done);
}
@java.lang.Override
@@ -26340,7 +26502,7 @@ public final class AdminProtos {
case 5:
return impl.closeRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest)request);
case 6:
- return impl.closeRegionForSplit(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest)request);
+ return impl.closeRegionForSplitOrMerge(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest)request);
case 7:
return impl.flushRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest)request);
case 8:
@@ -26392,7 +26554,7 @@ public final class AdminProtos {
case 5:
return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest.getDefaultInstance();
case 6:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.getDefaultInstance();
case 7:
return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest.getDefaultInstance();
case 8:
@@ -26444,7 +26606,7 @@ public final class AdminProtos {
case 5:
return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse.getDefaultInstance();
case 6:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.getDefaultInstance();
case 7:
return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.getDefaultInstance();
case 8:
@@ -26526,12 +26688,12 @@ public final class AdminProtos {
org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse> done);
/**
- * <code>rpc CloseRegionForSplit(.hbase.pb.CloseRegionForSplitRequest) returns (.hbase.pb.CloseRegionForSplitResponse);</code>
+ * <code>rpc CloseRegionForSplitOrMerge(.hbase.pb.CloseRegionForSplitOrMergeRequest) returns (.hbase.pb.CloseRegionForSplitOrMergeResponse);</code>
*/
- public abstract void closeRegionForSplit(
+ public abstract void closeRegionForSplitOrMerge(
org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest request,
- org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse> done);
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest request,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse> done);
/**
* <code>rpc FlushRegion(.hbase.pb.FlushRegionRequest) returns (.hbase.pb.FlushRegionResponse);</code>
@@ -26682,8 +26844,8 @@ public final class AdminProtos {
done));
return;
case 6:
- this.closeRegionForSplit(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest)request,
- org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse>specializeCallback(
+ this.closeRegionForSplitOrMerge(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest)request,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse>specializeCallback(
done));
return;
case 7:
@@ -26773,7 +26935,7 @@ public final class AdminProtos {
case 5:
return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest.getDefaultInstance();
case 6:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.getDefaultInstance();
case 7:
return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest.getDefaultInstance();
case 8:
@@ -26825,7 +26987,7 @@ public final class AdminProtos {
case 5:
return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse.getDefaultInstance();
case 6:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.getDefaultInstance();
case 7:
return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.getDefaultInstance();
case 8:
@@ -26961,19 +27123,19 @@ public final class AdminProtos {
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse.getDefaultInstance()));
}
- public void closeRegionForSplit(
+ public void closeRegionForSplitOrMerge(
org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest request,
- org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse> done) {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest request,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse> done) {
channel.callMethod(
getDescriptor().getMethods().get(6),
controller,
request,
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse.getDefaultInstance(),
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.getDefaultInstance(),
org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
done,
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse.class,
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse.getDefaultInstance()));
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.class,
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.getDefaultInstance()));
}
public void flushRegion(
@@ -27193,9 +27355,9 @@ public final class AdminProtos {
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
- public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse closeRegionForSplit(
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse closeRegionForSplitOrMerge(
org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest request)
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest request)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse flushRegion(
@@ -27338,15 +27500,15 @@ public final class AdminProtos {
}
- public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse closeRegionForSplit(
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse closeRegionForSplitOrMerge(
org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest request)
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest request)
throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
- return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse) channel.callBlockingMethod(
+ return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse) channel.callBlockingMethod(
getDescriptor().getMethods().get(6),
controller,
request,
- org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse.getDefaultInstance());
+ org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.getDefaultInstance());
}
@@ -27564,15 +27726,15 @@ public final class AdminProtos {
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
internal_static_hbase_pb_CloseRegionResponse_fieldAccessorTable;
private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
- internal_static_hbase_pb_CloseRegionForSplitRequest_descriptor;
+ internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_descriptor;
private static final
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
- internal_static_hbase_pb_CloseRegionForSplitRequest_fieldAccessorTable;
+ internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_fieldAccessorTable;
private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
- internal_static_hbase_pb_CloseRegionForSplitResponse_descriptor;
+ internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_descriptor;
private static final
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
- internal_static_hbase_pb_CloseRegionForSplitResponse_fieldAccessorTable;
+ internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_fieldAccessorTable;
private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
internal_static_hbase_pb_FlushRegionRequest_descriptor;
private static final
@@ -27690,12 +27852,12 @@ public final class AdminProtos {
internal_static_hbase_pb_UpdateConfigurationResponse_fieldAccessorTable;
private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
internal_static_hbase_pb_GetRegionLoadRequest_descriptor;
- private static final
+ private static final
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
internal_static_hbase_pb_GetRegionLoadRequest_fieldAccessorTable;
private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
internal_static_hbase_pb_GetRegionLoadResponse_descriptor;
- private static final
+ private static final
org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
internal_static_hbase_pb_GetRegionLoadResponse_fieldAccessorTable;
@@ -27742,94 +27904,95 @@ public final class AdminProtos {
"de\030\002 \001(\r\022\036\n\020transition_in_ZK\030\003 \001(\010:\004true" +
"\0220\n\022destination_server\030\004 \001(\0132\024.hbase.pb." +
"ServerName\022\027\n\017serverStartCode\030\005 \001(\004\"%\n\023C" +
- "loseRegionResponse\022\016\n\006closed\030\001 \002(\010\"G\n\032Cl" +
- "oseRegionForSplitRequest\022)\n\006region\030\001 \002(\013" +
- "2\031.hbase.pb.RegionSpecifier\"-\n\033CloseRegi" +
- "onForSplitResponse\022\016\n\006closed\030\001 \002(\010\"y\n\022Fl" +
- "ushRegionRequest\022)\n\006region\030\001 \002(\0132\031.hbase",
- ".pb.RegionSpecifier\022\030\n\020if_older_than_ts\030" +
- "\002 \001(\004\022\036\n\026write_flush_wal_marker\030\003 \001(\010\"_\n" +
- "\023FlushRegionResponse\022\027\n\017last_flush_time\030" +
- "\001 \002(\004\022\017\n\007flushed\030\002 \001(\010\022\036\n\026wrote_flush_wa" +
- "l_marker\030\003 \001(\010\"T\n\022SplitRegionRequest\022)\n\006" +
- "region\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022" +
- "\023\n\013split_point\030\002 \001(\014\"\025\n\023SplitRegionRespo" +
- "nse\"`\n\024CompactRegionRequest\022)\n\006region\030\001 " +
- "\002(\0132\031.hbase.pb.RegionSpecifier\022\r\n\005major\030" +
- "\002 \001(\010\022\016\n\006family\030\003 \001(\014\"\027\n\025CompactRegionRe",
- "sponse\"\315\001\n\031UpdateFavoredNodesRequest\022I\n\013" +
- "update_info\030\001 \003(\01324.hbase.pb.UpdateFavor" +
- "edNodesRequest.RegionUpdateInfo\032e\n\020Regio" +
- "nUpdateInfo\022$\n\006region\030\001 \002(\0132\024.hbase.pb.R" +
- "egionInfo\022+\n\rfavored_nodes\030\002 \003(\0132\024.hbase" +
- ".pb.ServerName\".\n\032UpdateFavoredNodesResp" +
- "onse\022\020\n\010response\030\001 \001(\r\"\244\001\n\023MergeRegionsR" +
- "equest\022+\n\010region_a\030\001 \002(\0132\031.hbase.pb.Regi" +
- "onSpecifier\022+\n\010region_b\030\002 \002(\0132\031.hbase.pb" +
- ".RegionSpecifier\022\027\n\010forcible\030\003 \001(\010:\005fals",
- "e\022\032\n\022master_system_time\030\004 \001(\004\"\026\n\024MergeRe" +
- "gionsResponse\"a\n\010WALEntry\022\035\n\003key\030\001 \002(\0132\020" +
- ".hbase.pb.WALKey\022\027\n\017key_value_bytes\030\002 \003(" +
- "\014\022\035\n\025associated_cell_count\030\003 \001(\005\"\242\001\n\030Rep" +
- "licateWALEntryRequest\022!\n\005entry\030\001 \003(\0132\022.h" +
- "base.pb.WALEntry\022\034\n\024replicationClusterId" +
- "\030\002 \001(\t\022\"\n\032sourceBaseNamespaceDirPath\030\003 \001" +
- "(\t\022!\n\031sourceHFileArchiveDirPath\030\004 \001(\t\"\033\n" +
- "\031ReplicateWALEntryResponse\"\026\n\024RollWALWri" +
- "terRequest\"0\n\025RollWALWriterResponse\022\027\n\017r",
- "egion_to_flush\030\001 \003(\014\"#\n\021StopServerReques" +
- "t\022\016\n\006reason\030\001 \002(\t\"\024\n\022StopServerResponse\"" +
- "\026\n\024GetServerInfoRequest\"K\n\nServe
<TRUNCATED>
[6/6] hbase git commit: HBASE-16119 Procedure v2 - Reimplement Merge
region (Stephen Yuan Jiang)
Posted by sy...@apache.org.
HBASE-16119 Procedure v2 - Reimplement Merge region (Stephen Yuan Jiang)
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/0a240778
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/0a240778
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/0a240778
Branch: refs/heads/master
Commit: 0a2407784146fbbe24d9ef0101599591a5aa9c21
Parents: 00b3024
Author: Stephen Yuan Jiang <sy...@gmail.com>
Authored: Thu Dec 1 22:41:15 2016 -0800
Committer: Stephen Yuan Jiang <sy...@gmail.com>
Committed: Thu Dec 1 22:41:15 2016 -0800
----------------------------------------------------------------------
.../org/apache/hadoop/hbase/client/Admin.java | 13 +
.../hbase/client/ConnectionImplementation.java | 7 +
.../apache/hadoop/hbase/client/HBaseAdmin.java | 96 +-
.../hbase/shaded/protobuf/ProtobufUtil.java | 39 +-
.../hbase/shaded/protobuf/RequestConverter.java | 17 +
.../shaded/protobuf/ResponseConverter.java | 4 +-
.../procedure2/ProcedureTestingUtility.java | 3 +-
.../shaded/protobuf/generated/AdminProtos.java | 891 +++--
.../generated/MasterProcedureProtos.java | 1896 +++++++++--
.../shaded/protobuf/generated/MasterProtos.java | 3144 +++++++++++++-----
.../src/main/protobuf/Admin.proto | 14 +-
.../src/main/protobuf/Master.proto | 18 +
.../src/main/protobuf/MasterProcedure.proto | 23 +-
.../hbase/rsgroup/RSGroupAdminEndpoint.java | 47 +-
.../BaseMasterAndRegionObserver.java | 48 +-
.../hbase/coprocessor/BaseMasterObserver.java | 48 +-
.../hbase/coprocessor/MasterObserver.java | 87 +-
.../hadoop/hbase/master/AssignmentManager.java | 33 +
.../org/apache/hadoop/hbase/master/HMaster.java | 45 +
.../hbase/master/MasterCoprocessorHost.java | 123 +-
.../hadoop/hbase/master/MasterRpcServices.java | 40 +
.../hadoop/hbase/master/MasterServices.java | 17 +-
.../hadoop/hbase/master/RegionStates.java | 8 +
.../hadoop/hbase/master/ServerManager.java | 13 +-
.../procedure/MergeTableRegionsProcedure.java | 907 +++++
.../procedure/SplitTableRegionProcedure.java | 20 +-
.../hbase/regionserver/HRegionFileSystem.java | 10 +-
.../hbase/regionserver/HRegionServer.java | 85 +-
.../hbase/regionserver/RSRpcServices.java | 31 +-
.../apache/hadoop/hbase/client/TestAdmin1.java | 9 +-
.../hbase/coprocessor/TestMasterObserver.java | 59 +-
.../coprocessor/TestRegionServerObserver.java | 2 +
.../hbase/master/MockNoopMasterServices.java | 9 +
.../hadoop/hbase/master/MockRegionServer.java | 8 +-
.../TestMergeTableRegionsProcedure.java | 263 ++
.../TestSplitTableRegionProcedure.java | 5 +-
.../hbase/namespace/TestNamespaceAuditor.java | 51 +-
.../replication/TestSerialReplication.java | 4 +-
38 files changed, 6600 insertions(+), 1537 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 52b935f..5b53a7e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -907,6 +907,7 @@ public interface Admin extends Abortable, Closeable {
void mergeRegions(final byte[] nameOfRegionA, final byte[] nameOfRegionB,
final boolean forcible) throws IOException;
+
/**
* Merge two regions. Asynchronous operation.
*
@@ -922,6 +923,18 @@ public interface Admin extends Abortable, Closeable {
final boolean forcible) throws IOException;
/**
+ * Merge regions. Asynchronous operation.
+ *
+ * @param nameofRegionsToMerge encoded or full name of daughter regions
+ * @param forcible true if do a compulsory merge, otherwise we will only merge
+ * adjacent regions
+ * @throws IOException
+ */
+ Future<Void> mergeRegionsAsync(
+ final byte[][] nameofRegionsToMerge,
+ final boolean forcible) throws IOException;
+
+ /**
* Split a table. Asynchronous operation.
*
* @param tableName table to split
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index aa984b1..e75d9a5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -1274,6 +1274,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
@Override
+ public MasterProtos.MergeTableRegionsResponse mergeTableRegions(
+ RpcController controller, MasterProtos.MergeTableRegionsRequest request)
+ throws ServiceException {
+ return stub.mergeTableRegions(controller, request);
+ }
+
+ @Override
public MasterProtos.AssignRegionResponse assignRegion(RpcController controller,
MasterProtos.AssignRegionRequest request) throws ServiceException {
return stub.assignRegion(controller, request);
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 00463f38..9bfe276 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -119,8 +119,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTabl
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
@@ -147,6 +145,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableD
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
@@ -1513,68 +1513,80 @@ public class HBaseAdmin implements Admin {
final byte[] nameOfRegionA,
final byte[] nameOfRegionB,
final boolean forcible) throws IOException {
+ byte[][] nameofRegionsToMerge = new byte[2][];
+ nameofRegionsToMerge[0] = nameOfRegionA;
+ nameofRegionsToMerge[1] = nameOfRegionB;
+ return mergeRegionsAsync(nameofRegionsToMerge, forcible);
+ }
- final byte[] encodedNameOfRegionA = isEncodedRegionName(nameOfRegionA) ?
- nameOfRegionA : HRegionInfo.encodeRegionName(nameOfRegionA).getBytes();
- final byte[] encodedNameOfRegionB = isEncodedRegionName(nameOfRegionB) ?
- nameOfRegionB : HRegionInfo.encodeRegionName(nameOfRegionB).getBytes();
-
- TableName tableName;
- Pair<HRegionInfo, ServerName> pair = getRegion(nameOfRegionA);
-
- if (pair != null) {
- if (pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
- throw new IllegalArgumentException ("Can't invoke merge on non-default regions directly");
- }
- tableName = pair.getFirst().getTable();
- } else {
- throw new UnknownRegionException (
- "Can't invoke merge on unknown region " + Bytes.toStringBinary(encodedNameOfRegionA));
+ /**
+ * Merge two regions. Asynchronous operation.
+ * @param nameofRegionsToMerge encoded or full name of daughter regions
+ * @param forcible true if do a compulsory merge, otherwise we will only merge
+ * adjacent regions
+ * @throws IOException
+ */
+ @Override
+ public Future<Void> mergeRegionsAsync(
+ final byte[][] nameofRegionsToMerge,
+ final boolean forcible) throws IOException {
+ assert(nameofRegionsToMerge.length >= 2);
+ byte[][] encodedNameofRegionsToMerge = new byte[nameofRegionsToMerge.length][];
+ for(int i = 0; i < nameofRegionsToMerge.length; i++) {
+ encodedNameofRegionsToMerge[i] = isEncodedRegionName(nameofRegionsToMerge[i]) ?
+ nameofRegionsToMerge[i] : HRegionInfo.encodeRegionName(nameofRegionsToMerge[i]).getBytes();
}
- pair = getRegion(nameOfRegionB);
- if (pair != null) {
- if (pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
- throw new IllegalArgumentException ("Can't invoke merge on non-default regions directly");
- }
+ TableName tableName = null;
+ Pair<HRegionInfo, ServerName> pair;
- if (!tableName.equals(pair.getFirst().getTable())) {
- throw new IllegalArgumentException ("Cannot merge regions from two different tables " +
- tableName + " and " + pair.getFirst().getTable());
+ for(int i = 0; i < nameofRegionsToMerge.length; i++) {
+ pair = getRegion(nameofRegionsToMerge[i]);
+
+ if (pair != null) {
+ if (pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+ throw new IllegalArgumentException ("Can't invoke merge on non-default regions directly");
+ }
+ if (tableName == null) {
+ tableName = pair.getFirst().getTable();
+ } else if (!tableName.equals(pair.getFirst().getTable())) {
+ throw new IllegalArgumentException ("Cannot merge regions from two different tables " +
+ tableName + " and " + pair.getFirst().getTable());
+ }
+ } else {
+ throw new UnknownRegionException (
+ "Can't invoke merge on unknown region "
+ + Bytes.toStringBinary(encodedNameofRegionsToMerge[i]));
}
- } else {
- throw new UnknownRegionException (
- "Can't invoke merge on unknown region " + Bytes.toStringBinary(encodedNameOfRegionB));
}
- DispatchMergingRegionsResponse response =
- executeCallable(new MasterCallable<DispatchMergingRegionsResponse>(getConnection(),
+ MergeTableRegionsResponse response =
+ executeCallable(new MasterCallable<MergeTableRegionsResponse>(getConnection(),
getRpcControllerFactory()) {
@Override
- protected DispatchMergingRegionsResponse rpcCall() throws Exception {
- DispatchMergingRegionsRequest request = RequestConverter
- .buildDispatchMergingRegionsRequest(
- encodedNameOfRegionA,
- encodedNameOfRegionB,
+ protected MergeTableRegionsResponse rpcCall() throws Exception {
+ MergeTableRegionsRequest request = RequestConverter
+ .buildMergeTableRegionsRequest(
+ encodedNameofRegionsToMerge,
forcible,
ng.getNonceGroup(),
ng.newNonce());
- return master.dispatchMergingRegions(getRpcController(), request);
+ return master.mergeTableRegions(getRpcController(), request);
}
});
- return new DispatchMergingRegionsFuture(this, tableName, response);
+ return new MergeTableRegionsFuture(this, tableName, response);
}
- private static class DispatchMergingRegionsFuture extends TableFuture<Void> {
- public DispatchMergingRegionsFuture(
+ private static class MergeTableRegionsFuture extends TableFuture<Void> {
+ public MergeTableRegionsFuture(
final HBaseAdmin admin,
final TableName tableName,
- final DispatchMergingRegionsResponse response) {
+ final MergeTableRegionsResponse response) {
super(admin, tableName,
(response != null && response.hasProcId()) ? response.getProcId() : null);
}
- public DispatchMergingRegionsFuture(
+ public MergeTableRegionsFuture(
final HBaseAdmin admin,
final TableName tableName,
final Long procId) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 5876fae..c6d647e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -105,8 +105,8 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse;
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.GetOnlineRegionRequest;
@@ -1756,26 +1756,26 @@ public final class ProtobufUtil {
}
/**
- * A helper to close a region for split
+ * A helper to close a region for split or merge
* using admin protocol.
*
* @param controller RPC controller
* @param admin Admin service
* @param server the RS that hosts the target region
- * @param parentRegionInfo the target region info
+ * @param regionInfo the target region info
* @return true if the region is closed
* @throws IOException
*/
- public static boolean closeRegionForSplit(
+ public static boolean closeRegionForSplitOrMerge(
final RpcController controller,
final AdminService.BlockingInterface admin,
final ServerName server,
- final HRegionInfo parentRegionInfo) throws IOException {
- CloseRegionForSplitRequest closeRegionForSplitRequest =
- ProtobufUtil.buildCloseRegionForSplitRequest(server, parentRegionInfo);
+ final HRegionInfo... regionInfo) throws IOException {
+ CloseRegionForSplitOrMergeRequest closeRegionForRequest =
+ ProtobufUtil.buildCloseRegionForSplitOrMergeRequest(server, regionInfo);
try {
- CloseRegionForSplitResponse response =
- admin.closeRegionForSplit(controller, closeRegionForSplitRequest);
+ CloseRegionForSplitOrMergeResponse response =
+ admin.closeRegionForSplitOrMerge(controller, closeRegionForRequest);
return ResponseConverter.isClosed(response);
} catch (ServiceException se) {
throw getRemoteException(se);
@@ -3130,19 +3130,22 @@ public final class ProtobufUtil {
}
/**
- * Create a CloseRegionForSplitRequest for a given region
+ * Create a CloseRegionForSplitOrMergeRequest for given regions
*
* @param server the RS server that hosts the region
- * @param parentRegionInfo the info of the region to close
+ * @param regionsToClose the info of the regions to close
* @return a CloseRegionForSplitRequest
*/
- public static CloseRegionForSplitRequest buildCloseRegionForSplitRequest(
+ public static CloseRegionForSplitOrMergeRequest buildCloseRegionForSplitOrMergeRequest(
final ServerName server,
- final HRegionInfo parentRegionInfo) {
- CloseRegionForSplitRequest.Builder builder = CloseRegionForSplitRequest.newBuilder();
- RegionSpecifier parentRegion = RequestConverter.buildRegionSpecifier(
- RegionSpecifierType.REGION_NAME, parentRegionInfo.getRegionName());
- builder.setRegion(parentRegion);
+ final HRegionInfo... regionsToClose) {
+ CloseRegionForSplitOrMergeRequest.Builder builder =
+ CloseRegionForSplitOrMergeRequest.newBuilder();
+ for(int i = 0; i < regionsToClose.length; i++) {
+ RegionSpecifier regionToClose = RequestConverter.buildRegionSpecifier(
+ RegionSpecifierType.REGION_NAME, regionsToClose[i].getRegionName());
+ builder.addRegion(regionToClose);
+ }
return builder.build();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 306c237..f938fd0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -96,6 +96,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJ
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
@@ -1096,6 +1097,22 @@ public final class RequestConverter {
return builder.build();
}
+ public static MergeTableRegionsRequest buildMergeTableRegionsRequest(
+ final byte[][] encodedNameOfdaughaterRegions,
+ final boolean forcible,
+ final long nonceGroup,
+ final long nonce) throws DeserializationException {
+ MergeTableRegionsRequest.Builder builder = MergeTableRegionsRequest.newBuilder();
+ for (int i = 0; i< encodedNameOfdaughaterRegions.length; i++) {
+ builder.addRegion(buildRegionSpecifier(
+ RegionSpecifierType.ENCODED_REGION_NAME, encodedNameOfdaughaterRegions[i]));
+ }
+ builder.setForcible(forcible);
+ builder.setNonceGroup(nonceGroup);
+ builder.setNonce(nonce);
+ return builder.build();
+ }
+
public static SplitTableRegionRequest buildSplitTableRegionRequest(
final HRegionInfo regionInfo,
final byte[] splitPoint,
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
index 11fc931..760f630 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.SingleResponse;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
@@ -259,7 +259,7 @@ public final class ResponseConverter {
* @return the region close state
*/
public static boolean isClosed
- (final CloseRegionForSplitResponse proto) {
+ (final CloseRegionForSplitOrMergeResponse proto) {
if (proto == null || !proto.hasClosed()) return false;
return proto.getClosed();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
index 8c9cea2..93f3460 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
@@ -149,7 +149,8 @@ public class ProcedureTestingUtility {
assertSingleExecutorForKillTests(procExecutor);
}
- private static <TEnv> void assertSingleExecutorForKillTests(final ProcedureExecutor<TEnv> procExecutor) {
+ private static <TEnv> void assertSingleExecutorForKillTests(
+ final ProcedureExecutor<TEnv> procExecutor) {
if (procExecutor.testing == null) return;
if (procExecutor.testing.killBeforeStoreUpdate ||
procExecutor.testing.toggleKillBeforeStoreUpdate) {
[3/6] hbase git commit: HBASE-16119 Procedure v2 - Reimplement Merge
region (Stephen Yuan Jiang)
Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0a240778/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index 03ef208..56442d1 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -7053,6 +7053,1504 @@ public final class MasterProtos {
}
+ public interface MergeTableRegionsRequestOrBuilder extends
+ // @@protoc_insertion_point(interface_extends:hbase.pb.MergeTableRegionsRequest)
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier>
+ getRegionList();
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index);
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ int getRegionCount();
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
+ getRegionOrBuilderList();
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
+ int index);
+
+ /**
+ * <code>optional bool forcible = 3 [default = false];</code>
+ */
+ boolean hasForcible();
+ /**
+ * <code>optional bool forcible = 3 [default = false];</code>
+ */
+ boolean getForcible();
+
+ /**
+ * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+ */
+ boolean hasNonceGroup();
+ /**
+ * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+ */
+ long getNonceGroup();
+
+ /**
+ * <code>optional uint64 nonce = 5 [default = 0];</code>
+ */
+ boolean hasNonce();
+ /**
+ * <code>optional uint64 nonce = 5 [default = 0];</code>
+ */
+ long getNonce();
+ }
+ /**
+ * <pre>
+ **
+ * Merging the specified regions in a table.
+ * </pre>
+ *
+ * Protobuf type {@code hbase.pb.MergeTableRegionsRequest}
+ */
+ public static final class MergeTableRegionsRequest extends
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+ // @@protoc_insertion_point(message_implements:hbase.pb.MergeTableRegionsRequest)
+ MergeTableRegionsRequestOrBuilder {
+ // Use MergeTableRegionsRequest.newBuilder() to construct.
+ private MergeTableRegionsRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+ super(builder);
+ }
+ private MergeTableRegionsRequest() {
+ region_ = java.util.Collections.emptyList();
+ forcible_ = false;
+ nonceGroup_ = 0L;
+ nonce_ = 0L;
+ }
+
+ @java.lang.Override
+ public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private MergeTableRegionsRequest(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ this();
+ int mutable_bitField0_ = 0;
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ case 10: {
+ if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+ region_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier>();
+ mutable_bitField0_ |= 0x00000001;
+ }
+ region_.add(
+ input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.PARSER, extensionRegistry));
+ break;
+ }
+ case 24: {
+ bitField0_ |= 0x00000001;
+ forcible_ = input.readBool();
+ break;
+ }
+ case 32: {
+ bitField0_ |= 0x00000002;
+ nonceGroup_ = input.readUInt64();
+ break;
+ }
+ case 40: {
+ bitField0_ |= 0x00000004;
+ nonce_ = input.readUInt64();
+ break;
+ }
+ }
+ }
+ } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+ e).setUnfinishedMessage(this);
+ } finally {
+ if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+ region_ = java.util.Collections.unmodifiableList(region_);
+ }
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_descriptor;
+ }
+
+ protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.Builder.class);
+ }
+
+ private int bitField0_;
+ public static final int REGION_FIELD_NUMBER = 1;
+ private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> region_;
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> getRegionList() {
+ return region_;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
+ getRegionOrBuilderList() {
+ return region_;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public int getRegionCount() {
+ return region_.size();
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index) {
+ return region_.get(index);
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
+ int index) {
+ return region_.get(index);
+ }
+
+ public static final int FORCIBLE_FIELD_NUMBER = 3;
+ private boolean forcible_;
+ /**
+ * <code>optional bool forcible = 3 [default = false];</code>
+ */
+ public boolean hasForcible() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional bool forcible = 3 [default = false];</code>
+ */
+ public boolean getForcible() {
+ return forcible_;
+ }
+
+ public static final int NONCE_GROUP_FIELD_NUMBER = 4;
+ private long nonceGroup_;
+ /**
+ * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+ */
+ public boolean hasNonceGroup() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+ */
+ public long getNonceGroup() {
+ return nonceGroup_;
+ }
+
+ public static final int NONCE_FIELD_NUMBER = 5;
+ private long nonce_;
+ /**
+ * <code>optional uint64 nonce = 5 [default = 0];</code>
+ */
+ public boolean hasNonce() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional uint64 nonce = 5 [default = 0];</code>
+ */
+ public long getNonce() {
+ return nonce_;
+ }
+
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized == 1) return true;
+ if (isInitialized == 0) return false;
+
+ for (int i = 0; i < getRegionCount(); i++) {
+ if (!getRegion(i).isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ for (int i = 0; i < region_.size(); i++) {
+ output.writeMessage(1, region_.get(i));
+ }
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeBool(3, forcible_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeUInt64(4, nonceGroup_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeUInt64(5, nonce_);
+ }
+ unknownFields.writeTo(output);
+ }
+
+ public int getSerializedSize() {
+ int size = memoizedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ for (int i = 0; i < region_.size(); i++) {
+ size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+ .computeMessageSize(1, region_.get(i));
+ }
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+ .computeBoolSize(3, forcible_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(4, nonceGroup_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(5, nonce_);
+ }
+ size += unknownFields.getSerializedSize();
+ memoizedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ public boolean equals(final java.lang.Object obj) {
+ if (obj == this) {
+ return true;
+ }
+ if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest)) {
+ return super.equals(obj);
+ }
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest) obj;
+
+ boolean result = true;
+ result = result && getRegionList()
+ .equals(other.getRegionList());
+ result = result && (hasForcible() == other.hasForcible());
+ if (hasForcible()) {
+ result = result && (getForcible()
+ == other.getForcible());
+ }
+ result = result && (hasNonceGroup() == other.hasNonceGroup());
+ if (hasNonceGroup()) {
+ result = result && (getNonceGroup()
+ == other.getNonceGroup());
+ }
+ result = result && (hasNonce() == other.hasNonce());
+ if (hasNonce()) {
+ result = result && (getNonce()
+ == other.getNonce());
+ }
+ result = result && unknownFields.equals(other.unknownFields);
+ return result;
+ }
+
+ @java.lang.Override
+ public int hashCode() {
+ if (memoizedHashCode != 0) {
+ return memoizedHashCode;
+ }
+ int hash = 41;
+ hash = (19 * hash) + getDescriptorForType().hashCode();
+ if (getRegionCount() > 0) {
+ hash = (37 * hash) + REGION_FIELD_NUMBER;
+ hash = (53 * hash) + getRegionList().hashCode();
+ }
+ if (hasForcible()) {
+ hash = (37 * hash) + FORCIBLE_FIELD_NUMBER;
+ hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
+ getForcible());
+ }
+ if (hasNonceGroup()) {
+ hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
+ hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
+ getNonceGroup());
+ }
+ if (hasNonce()) {
+ hash = (37 * hash) + NONCE_FIELD_NUMBER;
+ hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
+ getNonce());
+ }
+ hash = (29 * hash) + unknownFields.hashCode();
+ memoizedHashCode = hash;
+ return hash;
+ }
+
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(byte[] data)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(
+ byte[] data,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(
+ java.io.InputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseDelimitedWithIOException(PARSER, input);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseDelimitedFrom(
+ java.io.InputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input, extensionRegistry);
+ }
+
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder() {
+ return DEFAULT_INSTANCE.toBuilder();
+ }
+ public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest prototype) {
+ return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() {
+ return this == DEFAULT_INSTANCE
+ ? new Builder() : new Builder().mergeFrom(this);
+ }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * <pre>
+ **
+ * Merging the specified regions in a table.
+ * </pre>
+ *
+ * Protobuf type {@code hbase.pb.MergeTableRegionsRequest}
+ */
+ public static final class Builder extends
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+ // @@protoc_insertion_point(builder_implements:hbase.pb.MergeTableRegionsRequest)
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequestOrBuilder {
+ public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_descriptor;
+ }
+
+ protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.Builder.class);
+ }
+
+ // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .alwaysUseFieldBuilders) {
+ getRegionFieldBuilder();
+ }
+ }
+ public Builder clear() {
+ super.clear();
+ if (regionBuilder_ == null) {
+ region_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000001);
+ } else {
+ regionBuilder_.clear();
+ }
+ forcible_ = false;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ nonceGroup_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000004);
+ nonce_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000008);
+ return this;
+ }
+
+ public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_descriptor;
+ }
+
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest getDefaultInstanceForType() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.getDefaultInstance();
+ }
+
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest build() {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest buildPartial() {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (regionBuilder_ == null) {
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ region_ = java.util.Collections.unmodifiableList(region_);
+ bitField0_ = (bitField0_ & ~0x00000001);
+ }
+ result.region_ = region_;
+ } else {
+ result.region_ = regionBuilder_.build();
+ }
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.forcible_ = forcible_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.nonceGroup_ = nonceGroup_;
+ if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.nonce_ = nonce_;
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder clone() {
+ return (Builder) super.clone();
+ }
+ public Builder setField(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+ Object value) {
+ return (Builder) super.setField(field, value);
+ }
+ public Builder clearField(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+ return (Builder) super.clearField(field);
+ }
+ public Builder clearOneof(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+ return (Builder) super.clearOneof(oneof);
+ }
+ public Builder setRepeatedField(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+ int index, Object value) {
+ return (Builder) super.setRepeatedField(field, index, value);
+ }
+ public Builder addRepeatedField(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+ Object value) {
+ return (Builder) super.addRepeatedField(field, value);
+ }
+ public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+ if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest) {
+ return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest other) {
+ if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.getDefaultInstance()) return this;
+ if (regionBuilder_ == null) {
+ if (!other.region_.isEmpty()) {
+ if (region_.isEmpty()) {
+ region_ = other.region_;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ } else {
+ ensureRegionIsMutable();
+ region_.addAll(other.region_);
+ }
+ onChanged();
+ }
+ } else {
+ if (!other.region_.isEmpty()) {
+ if (regionBuilder_.isEmpty()) {
+ regionBuilder_.dispose();
+ regionBuilder_ = null;
+ region_ = other.region_;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ regionBuilder_ =
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+ getRegionFieldBuilder() : null;
+ } else {
+ regionBuilder_.addAllMessages(other.region_);
+ }
+ }
+ }
+ if (other.hasForcible()) {
+ setForcible(other.getForcible());
+ }
+ if (other.hasNonceGroup()) {
+ setNonceGroup(other.getNonceGroup());
+ }
+ if (other.hasNonce()) {
+ setNonce(other.getNonce());
+ }
+ this.mergeUnknownFields(other.unknownFields);
+ onChanged();
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ for (int i = 0; i < getRegionCount(); i++) {
+ if (!getRegion(i).isInitialized()) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest) e.getUnfinishedMessage();
+ throw e.unwrapIOException();
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> region_ =
+ java.util.Collections.emptyList();
+ private void ensureRegionIsMutable() {
+ if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+ region_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier>(region_);
+ bitField0_ |= 0x00000001;
+ }
+ }
+
+ private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
+
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> getRegionList() {
+ if (regionBuilder_ == null) {
+ return java.util.Collections.unmodifiableList(region_);
+ } else {
+ return regionBuilder_.getMessageList();
+ }
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public int getRegionCount() {
+ if (regionBuilder_ == null) {
+ return region_.size();
+ } else {
+ return regionBuilder_.getCount();
+ }
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index) {
+ if (regionBuilder_ == null) {
+ return region_.get(index);
+ } else {
+ return regionBuilder_.getMessage(index);
+ }
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public Builder setRegion(
+ int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+ if (regionBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureRegionIsMutable();
+ region_.set(index, value);
+ onChanged();
+ } else {
+ regionBuilder_.setMessage(index, value);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public Builder setRegion(
+ int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
+ if (regionBuilder_ == null) {
+ ensureRegionIsMutable();
+ region_.set(index, builderForValue.build());
+ onChanged();
+ } else {
+ regionBuilder_.setMessage(index, builderForValue.build());
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public Builder addRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+ if (regionBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureRegionIsMutable();
+ region_.add(value);
+ onChanged();
+ } else {
+ regionBuilder_.addMessage(value);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public Builder addRegion(
+ int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+ if (regionBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureRegionIsMutable();
+ region_.add(index, value);
+ onChanged();
+ } else {
+ regionBuilder_.addMessage(index, value);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public Builder addRegion(
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
+ if (regionBuilder_ == null) {
+ ensureRegionIsMutable();
+ region_.add(builderForValue.build());
+ onChanged();
+ } else {
+ regionBuilder_.addMessage(builderForValue.build());
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public Builder addRegion(
+ int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
+ if (regionBuilder_ == null) {
+ ensureRegionIsMutable();
+ region_.add(index, builderForValue.build());
+ onChanged();
+ } else {
+ regionBuilder_.addMessage(index, builderForValue.build());
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public Builder addAllRegion(
+ java.lang.Iterable<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> values) {
+ if (regionBuilder_ == null) {
+ ensureRegionIsMutable();
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
+ values, region_);
+ onChanged();
+ } else {
+ regionBuilder_.addAllMessages(values);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public Builder clearRegion() {
+ if (regionBuilder_ == null) {
+ region_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000001);
+ onChanged();
+ } else {
+ regionBuilder_.clear();
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public Builder removeRegion(int index) {
+ if (regionBuilder_ == null) {
+ ensureRegionIsMutable();
+ region_.remove(index);
+ onChanged();
+ } else {
+ regionBuilder_.remove(index);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBuilder(
+ int index) {
+ return getRegionFieldBuilder().getBuilder(index);
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
+ int index) {
+ if (regionBuilder_ == null) {
+ return region_.get(index); } else {
+ return regionBuilder_.getMessageOrBuilder(index);
+ }
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
+ getRegionOrBuilderList() {
+ if (regionBuilder_ != null) {
+ return regionBuilder_.getMessageOrBuilderList();
+ } else {
+ return java.util.Collections.unmodifiableList(region_);
+ }
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder addRegionBuilder() {
+ return getRegionFieldBuilder().addBuilder(
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance());
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder addRegionBuilder(
+ int index) {
+ return getRegionFieldBuilder().addBuilder(
+ index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance());
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+ */
+ public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder>
+ getRegionBuilderList() {
+ return getRegionFieldBuilder().getBuilderList();
+ }
+ private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
+ getRegionFieldBuilder() {
+ if (regionBuilder_ == null) {
+ regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
+ region_,
+ ((bitField0_ & 0x00000001) == 0x00000001),
+ getParentForChildren(),
+ isClean());
+ region_ = null;
+ }
+ return regionBuilder_;
+ }
+
+ private boolean forcible_ ;
+ /**
+ * <code>optional bool forcible = 3 [default = false];</code>
+ */
+ public boolean hasForcible() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional bool forcible = 3 [default = false];</code>
+ */
+ public boolean getForcible() {
+ return forcible_;
+ }
+ /**
+ * <code>optional bool forcible = 3 [default = false];</code>
+ */
+ public Builder setForcible(boolean value) {
+ bitField0_ |= 0x00000002;
+ forcible_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional bool forcible = 3 [default = false];</code>
+ */
+ public Builder clearForcible() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ forcible_ = false;
+ onChanged();
+ return this;
+ }
+
+ private long nonceGroup_ ;
+ /**
+ * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+ */
+ public boolean hasNonceGroup() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+ */
+ public long getNonceGroup() {
+ return nonceGroup_;
+ }
+ /**
+ * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+ */
+ public Builder setNonceGroup(long value) {
+ bitField0_ |= 0x00000004;
+ nonceGroup_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+ */
+ public Builder clearNonceGroup() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ nonceGroup_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ private long nonce_ ;
+ /**
+ * <code>optional uint64 nonce = 5 [default = 0];</code>
+ */
+ public boolean hasNonce() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional uint64 nonce = 5 [default = 0];</code>
+ */
+ public long getNonce() {
+ return nonce_;
+ }
+ /**
+ * <code>optional uint64 nonce = 5 [default = 0];</code>
+ */
+ public Builder setNonce(long value) {
+ bitField0_ |= 0x00000008;
+ nonce_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional uint64 nonce = 5 [default = 0];</code>
+ */
+ public Builder clearNonce() {
+ bitField0_ = (bitField0_ & ~0x00000008);
+ nonce_ = 0L;
+ onChanged();
+ return this;
+ }
+ public final Builder setUnknownFields(
+ final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+ return super.setUnknownFields(unknownFields);
+ }
+
+ public final Builder mergeUnknownFields(
+ final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+ return super.mergeUnknownFields(unknownFields);
+ }
+
+
+ // @@protoc_insertion_point(builder_scope:hbase.pb.MergeTableRegionsRequest)
+ }
+
+ // @@protoc_insertion_point(class_scope:hbase.pb.MergeTableRegionsRequest)
+ private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest DEFAULT_INSTANCE;
+ static {
+ DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest();
+ }
+
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest getDefaultInstance() {
+ return DEFAULT_INSTANCE;
+ }
+
+ @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<MergeTableRegionsRequest>
+ PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<MergeTableRegionsRequest>() {
+ public MergeTableRegionsRequest parsePartialFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return new MergeTableRegionsRequest(input, extensionRegistry);
+ }
+ };
+
+ public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<MergeTableRegionsRequest> parser() {
+ return PARSER;
+ }
+
+ @java.lang.Override
+ public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<MergeTableRegionsRequest> getParserForType() {
+ return PARSER;
+ }
+
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest getDefaultInstanceForType() {
+ return DEFAULT_INSTANCE;
+ }
+
+ }
+
+ public interface MergeTableRegionsResponseOrBuilder extends
+ // @@protoc_insertion_point(interface_extends:hbase.pb.MergeTableRegionsResponse)
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+ /**
+ * <code>optional uint64 proc_id = 1;</code>
+ */
+ boolean hasProcId();
+ /**
+ * <code>optional uint64 proc_id = 1;</code>
+ */
+ long getProcId();
+ }
+ /**
+ * Protobuf type {@code hbase.pb.MergeTableRegionsResponse}
+ */
+ public static final class MergeTableRegionsResponse extends
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+ // @@protoc_insertion_point(message_implements:hbase.pb.MergeTableRegionsResponse)
+ MergeTableRegionsResponseOrBuilder {
+ // Use MergeTableRegionsResponse.newBuilder() to construct.
+ private MergeTableRegionsResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+ super(builder);
+ }
+ private MergeTableRegionsResponse() {
+ procId_ = 0L;
+ }
+
+ @java.lang.Override
+ public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private MergeTableRegionsResponse(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ this();
+ int mutable_bitField0_ = 0;
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ case 8: {
+ bitField0_ |= 0x00000001;
+ procId_ = input.readUInt64();
+ break;
+ }
+ }
+ }
+ } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+ e).setUnfinishedMessage(this);
+ } finally {
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_descriptor;
+ }
+
+ protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.Builder.class);
+ }
+
+ private int bitField0_;
+ public static final int PROC_ID_FIELD_NUMBER = 1;
+ private long procId_;
+ /**
+ * <code>optional uint64 proc_id = 1;</code>
+ */
+ public boolean hasProcId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional uint64 proc_id = 1;</code>
+ */
+ public long getProcId() {
+ return procId_;
+ }
+
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized == 1) return true;
+ if (isInitialized == 0) return false;
+
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeUInt64(1, procId_);
+ }
+ unknownFields.writeTo(output);
+ }
+
+ public int getSerializedSize() {
+ int size = memoizedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(1, procId_);
+ }
+ size += unknownFields.getSerializedSize();
+ memoizedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ public boolean equals(final java.lang.Object obj) {
+ if (obj == this) {
+ return true;
+ }
+ if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse)) {
+ return super.equals(obj);
+ }
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse) obj;
+
+ boolean result = true;
+ result = result && (hasProcId() == other.hasProcId());
+ if (hasProcId()) {
+ result = result && (getProcId()
+ == other.getProcId());
+ }
+ result = result && unknownFields.equals(other.unknownFields);
+ return result;
+ }
+
+ @java.lang.Override
+ public int hashCode() {
+ if (memoizedHashCode != 0) {
+ return memoizedHashCode;
+ }
+ int hash = 41;
+ hash = (19 * hash) + getDescriptorForType().hashCode();
+ if (hasProcId()) {
+ hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
+ hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
+ getProcId());
+ }
+ hash = (29 * hash) + unknownFields.hashCode();
+ memoizedHashCode = hash;
+ return hash;
+ }
+
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(byte[] data)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(
+ byte[] data,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(
+ java.io.InputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseDelimitedWithIOException(PARSER, input);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseDelimitedFrom(
+ java.io.InputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input);
+ }
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input, extensionRegistry);
+ }
+
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder() {
+ return DEFAULT_INSTANCE.toBuilder();
+ }
+ public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse prototype) {
+ return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() {
+ return this == DEFAULT_INSTANCE
+ ? new Builder() : new Builder().mergeFrom(this);
+ }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * Protobuf type {@code hbase.pb.MergeTableRegionsResponse}
+ */
+ public static final class Builder extends
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+ // @@protoc_insertion_point(builder_implements:hbase.pb.MergeTableRegionsResponse)
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponseOrBuilder {
+ public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_descriptor;
+ }
+
+ protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.Builder.class);
+ }
+
+ // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+ .alwaysUseFieldBuilders) {
+ }
+ }
+ public Builder clear() {
+ super.clear();
+ procId_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ return this;
+ }
+
+ public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_descriptor;
+ }
+
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse getDefaultInstanceForType() {
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.getDefaultInstance();
+ }
+
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse build() {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse buildPartial() {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.procId_ = procId_;
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder clone() {
+ return (Builder) super.clone();
+ }
+ public Builder setField(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+ Object value) {
+ return (Builder) super.setField(field, value);
+ }
+ public Builder clearField(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+ return (Builder) super.clearField(field);
+ }
+ public Builder clearOneof(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+ return (Builder) super.clearOneof(oneof);
+ }
+ public Builder setRepeatedField(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+ int index, Object value) {
+ return (Builder) super.setRepeatedField(field, index, value);
+ }
+ public Builder addRepeatedField(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+ Object value) {
+ return (Builder) super.addRepeatedField(field, value);
+ }
+ public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+ if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse) {
+ return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse other) {
+ if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.getDefaultInstance()) return this;
+ if (other.hasProcId()) {
+ setProcId(other.getProcId());
+ }
+ this.mergeUnknownFields(other.unknownFields);
+ onChanged();
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ return true;
+ }
+
+ public Builder mergeFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse) e.getUnfinishedMessage();
+ throw e.unwrapIOException();
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ private long procId_ ;
+ /**
+ * <code>optional uint64 proc_id = 1;</code>
+ */
+ public boolean hasProcId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional uint64 proc_id = 1;</code>
+ */
+ public long getProcId() {
+ return procId_;
+ }
+ /**
+ * <code>optional uint64 proc_id = 1;</code>
+ */
+ public Builder setProcId(long value) {
+ bitField0_ |= 0x00000001;
+ procId_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional uint64 proc_id = 1;</code>
+ */
+ public Builder clearProcId() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ procId_ = 0L;
+ onChanged();
+ return this;
+ }
+ public final Builder setUnknownFields(
+ final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+ return super.setUnknownFields(unknownFields);
+ }
+
+ public final Builder mergeUnknownFields(
+ final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+ return super.mergeUnknownFields(unknownFields);
+ }
+
+
+ // @@protoc_insertion_point(builder_scope:hbase.pb.MergeTableRegionsResponse)
+ }
+
+ // @@protoc_insertion_point(class_scope:hbase.pb.MergeTableRegionsResponse)
+ private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse DEFAULT_INSTANCE;
+ static {
+ DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse();
+ }
+
+ public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse getDefaultInstance() {
+ return DEFAULT_INSTANCE;
+ }
+
+ @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<MergeTableRegionsResponse>
+ PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<MergeTableRegionsResponse>() {
+ public MergeTableRegionsResponse parsePartialFrom(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+ return new MergeTableRegionsResponse(input, extensionRegistry);
+ }
+ };
+
+ public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<MergeTableRegionsResponse> parser() {
+ return PARSER;
+ }
+
+ @java.lang.Override
+ public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<MergeTableRegionsResponse> getParserForType() {
+ return PARSER;
+ }
+
+ public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse getDefaultInstanceForType() {
+ return DEFAULT_INSTANCE;
+ }
+
+ }
+
public interface AssignRegionRequestOrBuilder extends
// @@protoc_insertion_point(interface_extends:hbase.pb.AssignRegionRequest)
org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
@@ -64243,6 +65741,18 @@ public final class MasterProtos {
/**
* <pre>
+ ** Master merge the regions
+ * </pre>
+ *
+ * <code>rpc MergeTableRegions(.hbase.pb.MergeTableRegionsRequest) returns (.hbase.pb.MergeTableRegionsResponse);</code>
+ */
+ public abstract void mergeTableRegions(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest request,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse> done);
+
+ /**
+ * <pre>
** Assign a region to a server chosen at random.
* </pre>
*
@@ -64920,6 +66430,14 @@ public final class MasterProtos {
}
@java.lang.Override
+ public void mergeTableRegions(
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+ org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest request,
+ org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse> done) {
+ impl.mergeTableRegions(controller, request, done);
+ }
+
+ @java.lang.Override
public void assignRegion(
org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest request,
@@ -65338,98 +66856,100 @@ public final class MasterProtos {
case 9:
return impl.dispatchMergingRegions(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest)request);
case 10:
- return impl.assignRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest)request);
+ return impl.mergeTableRegions(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest)request);
case 11:
- return impl.unassignRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest)request);
+ return impl.assignRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest)request);
case 12:
- return impl.offlineRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest)request);
+ return impl.unassignRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest)request);
case 13:
- return impl.deleteTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest)request);
+ return impl.offlineRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest)request);
case 14:
- return impl.truncateTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest)request);
+ return impl.deleteTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest)request);
case 15:
- return impl.enableTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest)request);
+ return impl.truncateTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest)request);
case 16:
- return impl.disableTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest)request);
+ return impl.enableTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest)request);
case 17:
- return impl.modifyTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest)request);
+ return impl.disableTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest)request);
case 18:
- return impl.createTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest)request);
+ return impl.modifyTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest)request);
case 19:
- return impl.shutdown(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest)request);
+ return impl.createTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest)request);
case 20:
- return impl.stopMaster(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest)request);
+ return impl.shutdown(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest)request);
case 21:
- return impl.isMasterInMaintenanceMode(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest)request);
+ return impl.stopMaster(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest)request);
case 22:
- return impl.balance(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest)request);
+ return impl.isMasterInMaintenanceMode(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest)request);
case 23:
- return impl.setBalancerRunning(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest)request);
+ return impl.balance(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest)request);
case 24:
- return impl.isBalancerEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest)request);
+ return impl.setBalancerRunning(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest)request);
case 25:
- return impl.setSplitOrMergeEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest)request);
+ return impl.isBalancerEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest)request);
case 26:
- return impl.isSplitOrMergeEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest)request);
+ return impl.setSplitOrMergeEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest)request);
case 27:
- return impl.normalize(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest)request);
+ return impl.isSplitOrMergeEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest)request);
case 28:
- return impl.setNormalizerRunning(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest)request);
+ return impl.normalize(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest)request);
case 29:
- return impl.isNormalizerEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest)request);
+ return impl.setNormalizerRunning(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest)request);
case 30:
- return impl.runCatalogScan(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest)request);
+ return impl.isNormalizerEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest)request);
case 31:
- return impl.enableCatalogJanitor(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest)request);
+ return impl.runCatalogScan(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest)request);
case 32:
- return impl.isCatalogJanitorEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest)request);
+ return impl.enableCatalogJanitor(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest)request);
case 33:
- return impl.execMasterService(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest)request);
+ return impl.isCatalogJanitorEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest)request);
case 34:
- return impl.snapshot(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest)request);
+ return impl.execMasterService(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest)request);
case 35:
- return impl.getCompletedSnapshots(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest)request);
+ return impl.snapshot(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest)request);
case 36:
- return impl.deleteSnapshot(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest)request);
+ return impl.getCompletedSnapshots(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest)request);
case 37:
- return impl.isSnapshotDone(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest)request);
+ return impl.deleteSnapshot(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest)request);
case 38:
- return impl.restoreSnapshot(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest)request);
+ return impl.isSnapshotDone(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest)request);
case 39:
- return impl.execProcedure(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest)request);
+ return impl.restoreSnapshot(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest)request);
case 40:
- return impl.execProcedureWithRet(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest)request);
+ return impl.execProcedure(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest)request);
case 41:
- return impl.isProcedureDone(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest)request);
+ return impl.execProcedureWithRet(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest)request);
case 42:
- return impl.modifyNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest)request);
+ return impl.isProcedureDone(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest)request);
case 43:
- return impl.createNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest)request);
+ return impl.modifyNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest)request);
case 44:
- return impl.deleteNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest)request);
+ return impl.createNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest)request);
case 45:
- return impl.getNamespaceDescriptor(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest)request);
+ return impl.deleteNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest)request);
case 46:
- return impl.listNamespaceDescriptors(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest)request);
+ return impl.getNamespaceDescriptor(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest)request);
case 47:
- return impl.listTableDescriptorsByNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest)request);
+ return impl.listNamespaceDescriptors(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest)request);
case 48:
- return impl.listTableNamesByNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest)request);
+ return impl.listTableDescriptorsByNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest)request);
case 49:
- return impl.getTableState(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest)request);
+ return impl.listTableNamesByNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest)request);
case 50:
- return impl.setQuota(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest)request);
+ return impl.getTableState(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest)request);
case 51:
- return impl.getLastMajorCompactionTimestamp(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest)request);
+ return impl.setQuota(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest)request);
case 52:
- return impl.getLastMajorCompactionTimestampForRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest)request);
+ return impl.getLastMajorCompactionTimestamp(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest)request);
case 53:
- return impl.getProcedureResult(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest)request);
+ return impl.getLastMajorCompactionTimestampForRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest)request);
case 54:
- return impl.getSecurityCapabilities(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest)request);
+ return impl.getProcedureResult(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest)request);
case 55:
- return impl.abortProcedure(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest)request);
+ return impl.getSecurityCapabilities(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest)request);
case 56:
+ return impl.abortProcedure(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest)request);
+ case 57:
return impl.listProcedures(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest)request);
default:
throw new java.lang.AssertionError("Can't get here.");
@@ -65466,98 +66986,100 @@ public final class MasterProtos {
case 9:
return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest.getDefaultInstance();
case 10:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.getDefaultInstance();
case 11:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest.getDefaultInstance();
case 12:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest.getDefaultInstance();
case 13:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest.getDefaultInstance();
case 14:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest.getDefaultInstance();
case 15:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest.getDefaultInstance();
case 16:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest.getDefaultInstance();
case 17:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest.getDefaultInstance();
case 18:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest.getDefaultInstance();
case 19:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest.getDefaultInstance();
case 20:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest.getDefaultInstance();
case 21:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest.getDefaultInstance();
case 22:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest.getDefaultInstance();
case 23:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest.getDefaultInstance();
case 24:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest.getDefaultInstance();
case 25:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest.getDefaultInstance();
case 26:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.getDefaultInstance();
case 27:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.getDefaultInstance();
case 28:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest.getDefaultInstance();
case 29:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.getDefaultInstance();
case 30:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest.getDefaultInstance();
case 31:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest.getDefaultInstance();
case 32:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest.getDefaultInstance();
case 33:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest.getDefaultInstance();
case 34:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest.getDefaultInstance();
+ return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest.getDefaultInstance();
case 35:
- return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest.getDefaul
<TRUNCATED>