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/07/15 14:07:08 UTC
[1/4] hbase git commit: HBASE-14552 Procedure V2: Reimplement
DispatchMergingRegionHandler (Stephen Yuan Jiang)
Repository: hbase
Updated Branches:
refs/heads/master 28d8609e5 -> f04eeecff
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
new file mode 100644
index 0000000..0dfb711
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
@@ -0,0 +1,584 @@
+/**
+ * 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.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+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.exceptions.MergeRegionException;
+import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.CatalogJanitor;
+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.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsState;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * The procedure to Merge a region in a table.
+ */
+@InterfaceAudience.Private
+public class DispatchMergingRegionsProcedure
+extends StateMachineProcedure<MasterProcedureEnv, DispatchMergingRegionsState>
+implements TableProcedureInterface {
+ private static final Log LOG = LogFactory.getLog(DispatchMergingRegionsProcedure.class);
+
+ private final AtomicBoolean aborted = new AtomicBoolean(false);
+ private Boolean traceEnabled;
+ private AssignmentManager assignmentManager;
+ private int timeout;
+ private ServerName regionLocation;
+ private String regionsToMergeListFullName;
+ private String regionsToMergeListEncodedName;
+
+ private UserGroupInformation user;
+ private TableName tableName;
+ private HRegionInfo [] regionsToMerge;
+ private boolean forcible;
+
+ public DispatchMergingRegionsProcedure() {
+ this.traceEnabled = isTraceEnabled();
+ this.assignmentManager = null;
+ this.timeout = -1;
+ this.regionLocation = null;
+ this.regionsToMergeListFullName = null;
+ this.regionsToMergeListEncodedName = null;
+ }
+
+ public DispatchMergingRegionsProcedure(
+ final MasterProcedureEnv env,
+ final TableName tableName,
+ final HRegionInfo [] regionsToMerge,
+ final boolean forcible) throws IOException {
+ this.traceEnabled = isTraceEnabled();
+ this.assignmentManager = getAssignmentManager(env);
+ this.tableName = tableName;
+ // For now, we only merge 2 regions. It could be extended to more than 2 regions in
+ // the future.
+ assert(regionsToMerge.length == 2);
+ this.regionsToMerge = regionsToMerge;
+ this.forcible = forcible;
+
+ this.user = env.getRequestUser().getUGI();
+ this.setOwner(this.user.getShortUserName());
+
+ this.timeout = -1;
+ this.regionsToMergeListFullName = getRegionsToMergeListFullNameString();
+ this.regionsToMergeListEncodedName = getRegionsToMergeListEncodedNameString();
+ }
+
+ @Override
+ protected Flow executeFromState(
+ final MasterProcedureEnv env,
+ final DispatchMergingRegionsState state) throws InterruptedException {
+ if (isTraceEnabled()) {
+ LOG.trace(this + " execute state=" + state);
+ }
+
+ try {
+ switch (state) {
+ case DISPATCH_MERGING_REGIONS_PREPARE:
+ prepareMergeRegion(env);
+ setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_PRE_OPERATION);
+ break;
+ case DISPATCH_MERGING_REGIONS_PRE_OPERATION:
+ //Unused for now - reserve to add preMerge coprocessor in the future
+ setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS);
+ break;
+ case DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS:
+ if (MoveRegionsToSameRS(env)) {
+ setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS);
+ } else {
+ LOG.info("Cancel merging regions " + getRegionsToMergeListFullNameString()
+ + ", because can't move them to the same RS");
+ setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_POST_OPERATION);
+ }
+ break;
+ case DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS:
+ doMergeInRS(env);
+ setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_POST_OPERATION);
+ break;
+ case DISPATCH_MERGING_REGIONS_POST_OPERATION:
+ //Unused for now - reserve to add postCompletedMerge coprocessor in the future
+ 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 " + tableName + " (in state=" + state + ")", e);
+
+ setFailure("master-merge-regions", e);
+ }
+ return Flow.HAS_MORE_STATE;
+ }
+
+ @Override
+ protected void rollbackState(
+ final MasterProcedureEnv env,
+ final DispatchMergingRegionsState state) throws IOException, InterruptedException {
+ if (isTraceEnabled()) {
+ LOG.trace(this + " rollback state=" + state);
+ }
+
+ try {
+ switch (state) {
+ case DISPATCH_MERGING_REGIONS_POST_OPERATION:
+ break; // nothing to rollback
+ case DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS:
+ 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);
+ break;
+ case DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS:
+ break; // nothing to rollback
+ case DISPATCH_MERGING_REGIONS_PRE_OPERATION:
+ break; // nothing to rollback
+ case DISPATCH_MERGING_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 " + tableName, e);
+ throw e;
+ }
+ }
+
+ @Override
+ protected DispatchMergingRegionsState getState(final int stateId) {
+ return DispatchMergingRegionsState.valueOf(stateId);
+ }
+
+ @Override
+ protected int getStateId(final DispatchMergingRegionsState state) {
+ return state.getNumber();
+ }
+
+ @Override
+ protected DispatchMergingRegionsState getInitialState() {
+ return DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_PREPARE;
+ }
+
+ @Override
+ protected void setNextState(DispatchMergingRegionsState state) {
+ if (aborted.get()) {
+ setAbortFailure("merge-table-regions", "abort requested");
+ } else {
+ super.setNextState(state);
+ }
+ }
+
+ @Override
+ public boolean abort(final MasterProcedureEnv env) {
+ aborted.set(true);
+ return true;
+ }
+
+ @Override
+ public void serializeStateData(final OutputStream stream) throws IOException {
+ super.serializeStateData(stream);
+
+ MasterProcedureProtos.DispatchMergingRegionsStateData.Builder dispatchMergingRegionsMsg =
+ MasterProcedureProtos.DispatchMergingRegionsStateData.newBuilder()
+ .setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
+ .setTableName(ProtobufUtil.toProtoTableName(tableName))
+ .setForcible(forcible);
+ for (HRegionInfo hri: regionsToMerge) {
+ dispatchMergingRegionsMsg.addRegionInfo(HRegionInfo.convert(hri));
+ }
+ dispatchMergingRegionsMsg.build().writeDelimitedTo(stream);
+ }
+
+ @Override
+ public void deserializeStateData(final InputStream stream) throws IOException {
+ super.deserializeStateData(stream);
+
+ MasterProcedureProtos.DispatchMergingRegionsStateData dispatchMergingRegionsMsg =
+ MasterProcedureProtos.DispatchMergingRegionsStateData.parseDelimitedFrom(stream);
+ user = MasterProcedureUtil.toUserInfo(dispatchMergingRegionsMsg.getUserInfo());
+ tableName = ProtobufUtil.toTableName(dispatchMergingRegionsMsg.getTableName());
+
+ assert(dispatchMergingRegionsMsg.getRegionInfoCount() == 2);
+ regionsToMerge = new HRegionInfo[dispatchMergingRegionsMsg.getRegionInfoCount()];
+ for (int i = 0; i < regionsToMerge.length; i++) {
+ regionsToMerge[i] = HRegionInfo.convert(dispatchMergingRegionsMsg.getRegionInfo(i));
+ }
+ }
+
+ @Override
+ public void toStringClassDetails(StringBuilder sb) {
+ sb.append(getClass().getSimpleName());
+ sb.append(" (table=");
+ sb.append(tableName);
+ sb.append(" regions=");
+ sb.append(getRegionsToMergeListFullNameString());
+ sb.append(" forcible=");
+ sb.append(forcible);
+ sb.append(")");
+ }
+
+ @Override
+ protected boolean acquireLock(final MasterProcedureEnv env) {
+ 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 tableName;
+ }
+
+ @Override
+ public TableOperationType getTableOperationType() {
+ return TableOperationType.MERGE;
+ }
+
+ /**
+ * 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 " + regionsToMerge[0].getRegionNameAsString()
+ + ", " + regionsToMerge[1].getRegionNameAsString() + ", because region "
+ + (regionAHasMergeQualifier ? regionsToMerge[0].getEncodedName() : regionsToMerge[1]
+ .getEncodedName()) + " has merge qualifier";
+ LOG.info(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);
+ }
+
+ 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 (!forcible && !HRegionInfo.areAdjacent(regionsToMerge[0], regionsToMerge[1])) {
+ throw new MergeRegionException(
+ "Unable to merge not adjacent regions "
+ + regionsToMerge[0].getRegionNameAsString() + ", "
+ + regionsToMerge[1].getRegionNameAsString()
+ + " where forcible = " + forcible);
+ }
+ }
+
+ /**
+ * 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;
+ }
+
+ /**
+ * Do the real merge operation in the region server that hosts regions
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void doMergeInRS(final MasterProcedureEnv env) throws IOException {
+ long duration = 0;
+ long startTime = EnvironmentEdgeManager.currentTime();
+ do {
+ try {
+ if (getServerName(env) == null) {
+ // The merge probably already happen. Check
+ RegionState regionState = getAssignmentManager(env).getRegionStates().getRegionState(
+ regionsToMerge[0].getEncodedName());
+ if (regionState.isMerging() || regionState.isMerged()) {
+ LOG.info("Merge regions " + getRegionsToMergeListEncodedNameString() +
+ " is in progress or completed. No need to send a new request.");
+ } else {
+ LOG.warn("Cannot sending merge to hosting server of the regions " +
+ getRegionsToMergeListEncodedNameString() + " as the server is unknown");
+ }
+ return;
+ }
+ // TODO: the following RPC call is not idempotent. Multiple calls (eg. after master
+ // failover, re-execute this step) could result in some exception thrown that does not
+ // paint the correct picture. This behavior is on-par with old releases. Improvement
+ // could happen in the future.
+ env.getMasterServices().getServerManager().sendRegionsMerge(
+ getServerName(env),
+ regionsToMerge[0],
+ regionsToMerge[1],
+ forcible,
+ user);
+ LOG.info("Sent merge to server " + getServerName(env) + " for region " +
+ getRegionsToMergeListEncodedNameString() + ", focible=" + forcible);
+ return;
+ } catch (RegionOpeningException roe) {
+ // Do a retry since region should be online on RS immediately
+ LOG.warn("Failed mergering regions in " + getServerName(env) + ", retrying...", roe);
+ } catch (Exception ie) {
+ LOG.warn("Failed sending merge to " + getServerName(env) + " for regions " +
+ getRegionsToMergeListEncodedNameString() + ", focible=" + forcible, ie);
+ return;
+ }
+ } while ((duration = EnvironmentEdgeManager.currentTime() - startTime) <= getTimeout(env));
+
+ // If we reaches here, it means that we get timed out.
+ String msg = "Failed sending merge to " + getServerName(env) + " after " + duration + "ms";
+ LOG.warn(msg);
+ throw new IOException(msg);
+ }
+
+ 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/f04eeecf/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 fd55f66..ec64eac 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
@@ -1180,7 +1180,7 @@ public class TestAdmin1 {
gotException = false;
// Try merging a replica with another. Should fail.
try {
- TEST_UTIL.getAdmin().mergeRegions(regions.get(1).getFirst().getEncodedNameAsBytes(),
+ TEST_UTIL.getAdmin().mergeRegionsAsync(regions.get(1).getFirst().getEncodedNameAsBytes(),
regions.get(2).getFirst().getEncodedNameAsBytes(), true);
} catch (IllegalArgumentException m) {
gotException = true;
@@ -1376,7 +1376,7 @@ public class TestAdmin1 {
assertEquals(3, admin.getTableRegions(tableName).size());
regionA = tableRegions.get(0);
regionB = tableRegions.get(1);
- admin.mergeRegions(regionA.getRegionName(), regionB.getRegionName(), false);
+ admin.mergeRegionsAsync(regionA.getRegionName(), regionB.getRegionName(), false);
Thread.sleep(1000);
assertEquals(2, admin.getTableRegions(tableName).size());
@@ -1384,7 +1384,8 @@ public class TestAdmin1 {
tableRegions = admin.getTableRegions(tableName);
regionA = tableRegions.get(0);
regionB = tableRegions.get(1);
- admin.mergeRegions(regionA.getEncodedNameAsBytes(), regionB.getEncodedNameAsBytes(), false);
+ admin.mergeRegionsAsync(
+ regionA.getEncodedNameAsBytes(), regionB.getEncodedNameAsBytes(), false);
Thread.sleep(1000);
assertEquals(1, admin.getTableRegions(tableName).size());
} finally {
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
index 45093bb..3948d18 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
@@ -272,19 +271,6 @@ public class TestHBaseAdminNoCluster {
(IsCatalogJanitorEnabledRequest)Mockito.any());
}
});
- // Admin.mergeRegions()
- testMasterOperationIsRetried(new MethodCaller() {
- @Override
- public void call(Admin admin) throws Exception {
- admin.mergeRegions(new byte[0], new byte[0], true);
- }
- @Override
- public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
- Mockito.verify(masterAdmin, Mockito.atLeast(count))
- .dispatchMergingRegions((RpcController)Mockito.any(),
- (DispatchMergingRegionsRequest)Mockito.any());
- }
- });
}
private static interface MethodCaller {
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
index c5ca0b5..e10cde5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
@@ -116,7 +116,7 @@ public class TestSplitOrMergeStatus {
assertTrue(results[0]);
List<HRegionInfo> regions = admin.getTableRegions(t.getName());
assertTrue(regions.size() > 1);
- admin.mergeRegions(regions.get(0).getEncodedNameAsBytes(),
+ admin.mergeRegionsAsync(regions.get(0).getEncodedNameAsBytes(),
regions.get(1).getEncodedNameAsBytes(), true);
int count = waitOnSplitOrMerge(t).size();
assertTrue(orignalCount == count);
@@ -125,7 +125,7 @@ public class TestSplitOrMergeStatus {
results = admin.setSplitOrMergeEnabled(true, false, true, MasterSwitchType.MERGE);
assertEquals(results.length, 1);
assertFalse(results[0]);
- admin.mergeRegions(regions.get(0).getEncodedNameAsBytes(),
+ admin.mergeRegionsAsync(regions.get(0).getEncodedNameAsBytes(),
regions.get(1).getEncodedNameAsBytes(), true);
count = waitOnSplitOrMerge(t).size();
assertTrue(orignalCount>count);
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/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 b4e93bd..7a7d448 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
@@ -1541,7 +1541,7 @@ public class TestMasterObserver {
RegionLocator regionLocator = connection.getRegionLocator(htd.getTableName());
List<HRegionLocation> regions = regionLocator.getAllRegionLocations();
- admin.mergeRegions(regions.get(0).getRegionInfo().getEncodedNameAsBytes(),
+ admin.mergeRegionsAsync(regions.get(0).getRegionInfo().getEncodedNameAsBytes(),
regions.get(1).getRegionInfo().getEncodedNameAsBytes(), true);
assertTrue("Coprocessor should have been called on region merge",
cp.wasDispatchMergeCalled());
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/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 9fcfd43..ecf9da1 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
@@ -92,8 +92,10 @@ public class TestRegionServerObserver {
admin.createTable(desc, new byte[][] { Bytes.toBytes("row") });
assertFalse(regionServerObserver.wasRegionMergeCalled());
List<Region> regions = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME));
- admin.mergeRegions(regions.get(0).getRegionInfo().getEncodedNameAsBytes(), regions.get(1)
- .getRegionInfo().getEncodedNameAsBytes(), true);
+ admin.mergeRegionsAsync(
+ regions.get(0).getRegionInfo().getEncodedNameAsBytes(),
+ regions.get(1).getRegionInfo().getEncodedNameAsBytes(),
+ true);
int regionsCount = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME)).size();
while (regionsCount != 1) {
regionsCount = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME)).size();
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/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 60b62e4..c7a42d9 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
@@ -93,6 +93,11 @@ public class MockNoopMasterServices implements MasterServices, Server {
}
@Override
+ public CatalogJanitor getCatalogJanitor() {
+ return null;
+ }
+
+ @Override
public MasterFileSystem getMasterFileSystem() {
return null;
}
@@ -282,8 +287,13 @@ public class MockNoopMasterServices implements MasterServices, Server {
}
@Override
- public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b,
- boolean forcible, User user) throws IOException {
+ public long dispatchMergingRegions(
+ final HRegionInfo region_a,
+ final HRegionInfo region_b,
+ final boolean forcible,
+ final long nonceGroup,
+ final long nonce) throws IOException {
+ return -1;
}
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
index 5b3abea..6ad2c5d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
@@ -242,7 +242,7 @@ public class TestAssignmentListener {
listener.reset();
List<HRegionInfo> regions = admin.getTableRegions(TABLE_NAME);
assertEquals(2, regions.size());
- admin.mergeRegions(regions.get(0).getEncodedNameAsBytes(),
+ admin.mergeRegionsAsync(regions.get(0).getEncodedNameAsBytes(),
regions.get(1).getEncodedNameAsBytes(), true);
listener.awaitModifications(3);
assertEquals(1, admin.getTableRegions(TABLE_NAME).size());
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDispatchMergingRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDispatchMergingRegionsProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDispatchMergingRegionsProcedure.java
new file mode 100644
index 0000000..17d1e757c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDispatchMergingRegionsProcedure.java
@@ -0,0 +1,296 @@
+/**
+ * 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.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CloneSnapshotState;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsState;
+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 TestDispatchMergingRegionsProcedure {
+ private static final Log LOG = LogFactory.getLog(TestDispatchMergingRegionsProcedure.class);
+
+ protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+ private static long nonceGroup = HConstants.NO_NONCE;
+ private static long nonce = HConstants.NO_NONCE;
+
+ 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, 3);
+ }
+
+ @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();
+
+ HTableDescriptor desc = new HTableDescriptor(tableName);
+ desc.addFamily(new HColumnDescriptor(FAMILY));
+ byte[][] splitRows = new byte[2][];
+ splitRows[0] = new byte[]{(byte)'3'};
+ splitRows[1] = new byte[]{(byte)'6'};
+ admin.createTable(desc, splitRows);
+
+ List<HRegionInfo> tableRegions;
+ HRegionInfo [] regionsToMerge = new HRegionInfo[2];
+
+ tableRegions = admin.getTableRegions(tableName);
+ assertEquals(3, admin.getTableRegions(tableName).size());
+ regionsToMerge[0] = tableRegions.get(0);
+ regionsToMerge[1] = tableRegions.get(1);
+
+ long procId = procExec.submitProcedure(new DispatchMergingRegionsProcedure(
+ procExec.getEnvironment(), tableName, regionsToMerge, true));
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+
+ assertEquals(2, admin.getTableRegions(tableName).size());
+ }
+
+ /**
+ * This tests two concurrent region merges
+ */
+ @Test(timeout=90000)
+ public void testMergeRegionsConcurrently() throws Exception {
+ final TableName tableName = TableName.valueOf("testMergeTwoRegions");
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ HTableDescriptor desc = new HTableDescriptor(tableName);
+ desc.addFamily(new HColumnDescriptor(FAMILY));
+ byte[][] splitRows = new byte[3][];
+ splitRows[0] = new byte[]{(byte)'2'};
+ splitRows[1] = new byte[]{(byte)'4'};
+ splitRows[2] = new byte[]{(byte)'6'};
+ admin.createTable(desc, splitRows);
+
+ List<HRegionInfo> tableRegions;
+ HRegionInfo [] regionsToMerge1 = new HRegionInfo[2];
+ HRegionInfo [] regionsToMerge2 = new HRegionInfo[2];
+
+ tableRegions = admin.getTableRegions(tableName);
+ assertEquals(4, admin.getTableRegions(tableName).size());
+ 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 DispatchMergingRegionsProcedure(
+ procExec.getEnvironment(), tableName, regionsToMerge1, true));
+ long procId2 = procExec.submitProcedure(new DispatchMergingRegionsProcedure(
+ procExec.getEnvironment(), tableName, regionsToMerge2, true));
+ ProcedureTestingUtility.waitProcedure(procExec, procId1);
+ ProcedureTestingUtility.waitProcedure(procExec, procId2);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
+
+ assertEquals(2, admin.getTableRegions(tableName).size());
+ }
+
+ @Test(timeout=60000)
+ public void testMergeRegionsTwiceWithSameNonce() throws Exception {
+ final TableName tableName = TableName.valueOf("testMergeRegionsTwiceWithSameNonce");
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ HTableDescriptor desc = new HTableDescriptor(tableName);
+ desc.addFamily(new HColumnDescriptor(FAMILY));
+ byte[][] splitRows = new byte[2][];
+ splitRows[0] = new byte[]{(byte)'3'};
+ splitRows[1] = new byte[]{(byte)'6'};
+ admin.createTable(desc, splitRows);
+
+ List<HRegionInfo> tableRegions;
+ HRegionInfo [] regionsToMerge = new HRegionInfo[2];
+
+ tableRegions = admin.getTableRegions(tableName);
+ assertEquals(3, admin.getTableRegions(tableName).size());
+ regionsToMerge[0] = tableRegions.get(0);
+ regionsToMerge[1] = tableRegions.get(1);
+
+ long procId1 = procExec.submitProcedure(new DispatchMergingRegionsProcedure(
+ procExec.getEnvironment(), tableName, regionsToMerge, true), nonceGroup, nonce);
+ long procId2 = procExec.submitProcedure(new DispatchMergingRegionsProcedure(
+ procExec.getEnvironment(), tableName, regionsToMerge, true), nonceGroup, nonce);
+ 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);
+ assertTrue(procId1 == procId2);
+ assertEquals(2, admin.getTableRegions(tableName).size());
+ }
+
+ @Test(timeout=60000)
+ public void testRecoveryAndDoubleExecution() throws Exception {
+ final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ HTableDescriptor desc = new HTableDescriptor(tableName);
+ desc.addFamily(new HColumnDescriptor(FAMILY));
+ byte[][] splitRows = new byte[2][];
+ splitRows[0] = new byte[]{(byte)'3'};
+ splitRows[1] = new byte[]{(byte)'6'};
+ admin.createTable(desc, splitRows);
+
+ ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+ List<HRegionInfo> tableRegions;
+ HRegionInfo [] regionsToMerge = new HRegionInfo[2];
+
+ tableRegions = admin.getTableRegions(tableName);
+ assertEquals(3, admin.getTableRegions(tableName).size());
+ regionsToMerge[0] = tableRegions.get(0);
+ regionsToMerge[1] = tableRegions.get(1);
+
+ long procId = procExec.submitProcedure(
+ new DispatchMergingRegionsProcedure(
+ procExec.getEnvironment(), tableName, regionsToMerge, true));
+
+ // Restart the executor and execute the step twice
+ int numberOfSteps = DispatchMergingRegionsState.values().length;
+ MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
+ procExec,
+ procId,
+ numberOfSteps,
+ DispatchMergingRegionsState.values());
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+
+ assertEquals(2, admin.getTableRegions(tableName).size());
+ }
+
+ @Test(timeout = 60000)
+ public void testRollbackAndDoubleExecution() throws Exception {
+ final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ HTableDescriptor desc = new HTableDescriptor(tableName);
+ desc.addFamily(new HColumnDescriptor(FAMILY));
+ byte[][] splitRows = new byte[2][];
+ splitRows[0] = new byte[]{(byte)'3'};
+ splitRows[1] = new byte[]{(byte)'6'};
+ admin.createTable(desc, splitRows);
+
+ ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+ List<HRegionInfo> tableRegions;
+ HRegionInfo [] regionsToMerge = new HRegionInfo[2];
+
+ tableRegions = admin.getTableRegions(tableName);
+ assertEquals(3, admin.getTableRegions(tableName).size());
+ regionsToMerge[0] = tableRegions.get(0);
+ regionsToMerge[1] = tableRegions.get(1);
+
+ long procId = procExec.submitProcedure(
+ new DispatchMergingRegionsProcedure(
+ procExec.getEnvironment(), tableName, regionsToMerge, true));
+
+ int numberOfSteps = DispatchMergingRegionsState.values().length - 3;
+ MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
+ procExec,
+ procId,
+ numberOfSteps,
+ DispatchMergingRegionsState.values());
+ }
+
+ private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+ return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
index 4d347f2..9dd5065 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.TestMobSnapshotCloneIndependence;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
@@ -56,7 +55,6 @@ import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.ClassRule;
-import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestRule;
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/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 d8b1b2e..8c9db88 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
@@ -334,7 +334,9 @@ public class TestNamespaceAuditor {
// merge the two regions
final Set<String> encodedRegionNamesToMerge =
Sets.newHashSet(hris.get(0).getEncodedName(), hris.get(1).getEncodedName());
- ADMIN.mergeRegions(hris.get(0).getEncodedNameAsBytes(), hris.get(1).getEncodedNameAsBytes(),
+ ADMIN.mergeRegionsAsync(
+ hris.get(0).getEncodedNameAsBytes(),
+ hris.get(1).getEncodedNameAsBytes(),
false);
UTIL.waitFor(10000, 100, new Waiter.ExplainingPredicate<Exception>() {
@@ -420,7 +422,9 @@ public class TestNamespaceAuditor {
regionServerObserver.failMerge(true);
regionServerObserver.triggered = false;
- ADMIN.mergeRegions(hris.get(1).getEncodedNameAsBytes(), hris.get(2).getEncodedNameAsBytes(),
+ ADMIN.mergeRegionsAsync(
+ hris.get(1).getEncodedNameAsBytes(),
+ hris.get(2).getEncodedNameAsBytes(),
false);
regionServerObserver.waitUtilTriggered();
hris = ADMIN.getTableRegions(tableTwo);
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
index 3307b73..f88c7dd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
@@ -320,6 +321,7 @@ public class TestRegionMergeTransactionOnCluster {
public void testMerge() throws Exception {
LOG.info("Starting testMerge");
final TableName tableName = TableName.valueOf("testMerge");
+ final HBaseAdmin hbaseAdmin = TEST_UTIL.getHBaseAdmin();
try {
// Create table and load data.
@@ -332,7 +334,7 @@ public class TestRegionMergeTransactionOnCluster {
regionStates.regionOffline(a);
try {
// Merge offline region. Region a is offline here
- ADMIN.mergeRegions(a.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), false);
+ hbaseAdmin.mergeRegionsSync(a.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), false);
fail("Offline regions should not be able to merge");
} catch (IOException ie) {
System.out.println(ie);
@@ -340,18 +342,20 @@ public class TestRegionMergeTransactionOnCluster {
StringUtils.stringifyException(ie).contains("regions not online")
&& ie instanceof MergeRegionException);
}
+
try {
// Merge the same region: b and b.
- ADMIN.mergeRegions(b.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), true);
+ hbaseAdmin.mergeRegionsSync(b.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), true);
fail("A region should not be able to merge with itself, even forcifully");
} catch (IOException ie) {
assertTrue("Exception should mention regions not online",
StringUtils.stringifyException(ie).contains("region to itself")
&& ie instanceof MergeRegionException);
}
+
try {
// Merge unknown regions
- ADMIN.mergeRegions(Bytes.toBytes("-f1"), Bytes.toBytes("-f2"), true);
+ hbaseAdmin.mergeRegionsSync(Bytes.toBytes("-f1"), Bytes.toBytes("-f2"), true);
fail("Unknown region could not be merged");
} catch (IOException ie) {
assertTrue("UnknownRegionException should be thrown",
@@ -419,7 +423,7 @@ public class TestRegionMergeTransactionOnCluster {
TEST_UTIL.getConnection(), tablename);
HRegionInfo regionA = tableRegions.get(regionAnum).getFirst();
HRegionInfo regionB = tableRegions.get(regionBnum).getFirst();
- ADMIN.mergeRegions(
+ ADMIN.mergeRegionsAsync(
regionA.getEncodedNameAsBytes(),
regionB.getEncodedNameAsBytes(), false);
return new PairOfSameType<HRegionInfo>(regionA, regionB);
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
index 4c2d69a..8efc2d7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
@@ -42,9 +42,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.TestMobSnapshotCloneIndependence;
import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.procedure.TestMasterFailoverWithProcedures;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@@ -58,7 +56,6 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
-import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestRule;
@@ -326,9 +323,9 @@ public class TestFlushSnapshotFromClient {
int numRegions = admin.getTableRegions(TABLE_NAME).size();
int numRegionsAfterMerge = numRegions - 2;
- admin.mergeRegions(regions.get(1).getEncodedNameAsBytes(),
+ admin.mergeRegionsAsync(regions.get(1).getEncodedNameAsBytes(),
regions.get(2).getEncodedNameAsBytes(), true);
- admin.mergeRegions(regions.get(5).getEncodedNameAsBytes(),
+ admin.mergeRegionsAsync(regions.get(5).getEncodedNameAsBytes(),
regions.get(6).getEncodedNameAsBytes(), true);
// Verify that there's one region less
@@ -367,9 +364,9 @@ public class TestFlushSnapshotFromClient {
int numRegions = admin.getTableRegions(TABLE_NAME).size();
int numRegionsAfterMerge = numRegions - 2;
- admin.mergeRegions(regions.get(1).getEncodedNameAsBytes(),
+ admin.mergeRegionsAsync(regions.get(1).getEncodedNameAsBytes(),
regions.get(2).getEncodedNameAsBytes(), true);
- admin.mergeRegions(regions.get(5).getEncodedNameAsBytes(),
+ admin.mergeRegionsAsync(regions.get(5).getEncodedNameAsBytes(),
regions.get(6).getEncodedNameAsBytes(), true);
waitRegionsAfterMerge(numRegionsAfterMerge);
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java
index 1fa681a..e194bf7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java
@@ -26,13 +26,11 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.master.procedure.TestMasterFailoverWithProcedures;
import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.junit.BeforeClass;
import org.junit.ClassRule;
-import org.junit.Rule;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestRule;
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
index 84ef6da..ecfe521 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
@@ -631,7 +631,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
assertNotEquals(region1, region2);
// do a region merge
- admin.mergeRegions(region1.getEncodedNameAsBytes(), region2.getEncodedNameAsBytes(), false);
+ admin.mergeRegionsAsync(
+ region1.getEncodedNameAsBytes(), region2.getEncodedNameAsBytes(), false);
// wait until region merged
long timeout = System.currentTimeMillis() + 30 * 1000;
[4/4] hbase git commit: HBASE-14552 Procedure V2: Reimplement
DispatchMergingRegionHandler (Stephen Yuan Jiang)
Posted by sy...@apache.org.
HBASE-14552 Procedure V2: Reimplement DispatchMergingRegionHandler (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/f04eeecf
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f04eeecf
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f04eeecf
Branch: refs/heads/master
Commit: f04eeecffc4ec20a2cfc86ef55f6b724ecbc7e5e
Parents: 28d8609
Author: Stephen Yuan Jiang <sy...@gmail.com>
Authored: Fri Jul 15 07:06:53 2016 -0700
Committer: Stephen Yuan Jiang <sy...@gmail.com>
Committed: Fri Jul 15 07:06:53 2016 -0700
----------------------------------------------------------------------
.../org/apache/hadoop/hbase/client/Admin.java | 17 +
.../apache/hadoop/hbase/client/HBaseAdmin.java | 104 +-
.../hadoop/hbase/protobuf/ProtobufUtil.java | 8 +-
...MergeRandomAdjacentRegionsOfTableAction.java | 2 +-
.../generated/MasterProcedureProtos.java | 1823 +++++++++++++++---
.../hbase/protobuf/generated/MasterProtos.java | 893 ++++++---
hbase-protocol/src/main/protobuf/Master.proto | 3 +
.../src/main/protobuf/MasterProcedure.proto | 15 +
.../org/apache/hadoop/hbase/master/HMaster.java | 57 +-
.../hadoop/hbase/master/MasterRpcServices.java | 41 +-
.../hadoop/hbase/master/MasterServices.java | 17 +-
.../hadoop/hbase/master/RegionStates.java | 2 +-
.../hadoop/hbase/master/ServerManager.java | 5 +-
.../handler/DispatchMergingRegionHandler.java | 191 --
.../normalizer/MergeNormalizationPlan.java | 2 +-
.../DispatchMergingRegionsProcedure.java | 584 ++++++
.../apache/hadoop/hbase/client/TestAdmin1.java | 7 +-
.../hbase/client/TestHBaseAdminNoCluster.java | 14 -
.../hbase/client/TestSplitOrMergeStatus.java | 4 +-
.../hbase/coprocessor/TestMasterObserver.java | 2 +-
.../coprocessor/TestRegionServerObserver.java | 6 +-
.../hbase/master/MockNoopMasterServices.java | 14 +-
.../hbase/master/TestAssignmentListener.java | 2 +-
.../TestDispatchMergingRegionsProcedure.java | 296 +++
.../TestMasterFailoverWithProcedures.java | 2 -
.../hbase/namespace/TestNamespaceAuditor.java | 8 +-
.../TestRegionMergeTransactionOnCluster.java | 12 +-
.../snapshot/TestFlushSnapshotFromClient.java | 11 +-
.../TestMobFlushSnapshotFromClient.java | 2 -
.../hadoop/hbase/util/TestHBaseFsckOneRS.java | 3 +-
30 files changed, 3320 insertions(+), 827 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/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 51a26bc..34e0a89 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
@@ -869,11 +869,28 @@ public interface Admin extends Abortable, Closeable {
* @param forcible true if do a compulsory merge, otherwise we will only merge two adjacent
* regions
* @throws IOException
+ * @deprecated Since 2.0. Will be removed in 3.0. Use
+ * {@link #mergeRegionsAsync(byte[], byte[], boolean)} instead.
*/
+ @Deprecated
void mergeRegions(final byte[] nameOfRegionA, final byte[] nameOfRegionB,
final boolean forcible) throws IOException;
/**
+ * Merge two regions. Asynchronous operation.
+ *
+ * @param nameOfRegionA encoded or full name of region a
+ * @param nameOfRegionB encoded or full name of region b
+ * @param forcible true if do a compulsory merge, otherwise we will only merge
+ * two adjacent regions
+ * @throws IOException
+ */
+ public Future<Void> mergeRegionsAsync(
+ final byte[] nameOfRegionA,
+ final byte[] nameOfRegionB,
+ 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/f04eeecf/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 da0de51..d18b8b3 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
@@ -115,6 +115,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRespon
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
@@ -1527,31 +1528,95 @@ public class HBaseAdmin implements Admin {
}
/**
+ * Merge two regions. Synchronous operation.
+ * Note: It is not feasible to predict the length of merge.
+ * Therefore, this is for internal testing only.
+ * @param nameOfRegionA encoded or full name of region a
+ * @param nameOfRegionB encoded or full name of region b
+ * @param forcible true if do a compulsory merge, otherwise we will only merge
+ * two adjacent regions
+ * @throws IOException
+ */
+ @VisibleForTesting
+ public void mergeRegionsSync(
+ final byte[] nameOfRegionA,
+ final byte[] nameOfRegionB,
+ final boolean forcible) throws IOException {
+ get(
+ mergeRegionsAsync(nameOfRegionA, nameOfRegionB, forcible),
+ syncWaitTimeout,
+ TimeUnit.MILLISECONDS);
+ }
+
+ /**
* Merge two regions. Asynchronous operation.
* @param nameOfRegionA encoded or full name of region a
* @param nameOfRegionB encoded or full name of region b
* @param forcible true if do a compulsory merge, otherwise we will only merge
* two adjacent regions
* @throws IOException
+ * @deprecated Since 2.0. Will be removed in 3.0. Use
+ * {@link #mergeRegionsAsync(byte[], byte[], boolean)} instead.
*/
+ @Deprecated
@Override
public void mergeRegions(final byte[] nameOfRegionA,
final byte[] nameOfRegionB, final boolean forcible)
throws IOException {
+ mergeRegionsAsync(nameOfRegionA, nameOfRegionB, forcible);
+ }
+
+ /**
+ * Merge two regions. Asynchronous operation.
+ * @param nameOfRegionA encoded or full name of region a
+ * @param nameOfRegionB encoded or full name of region b
+ * @param forcible true if do a compulsory merge, otherwise we will only merge
+ * two adjacent regions
+ * @throws IOException
+ */
+ @Override
+ public Future<Void> mergeRegionsAsync(
+ final byte[] nameOfRegionA,
+ final byte[] nameOfRegionB,
+ final boolean forcible) throws IOException {
+
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 && pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID)
- throw new IllegalArgumentException("Can't invoke merge on non-default regions directly");
+
+ 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));
+ }
+
pair = getRegion(nameOfRegionB);
- if (pair != null && pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID)
- throw new IllegalArgumentException("Can't invoke merge on non-default regions directly");
- executeCallable(new MasterCallable<Void>(getConnection()) {
+ 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.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(encodedNameOfRegionB));
+ }
+
+ DispatchMergingRegionsResponse response =
+ executeCallable(new MasterCallable<DispatchMergingRegionsResponse>(getConnection()) {
@Override
- public Void call(int callTimeout) throws ServiceException {
+ public DispatchMergingRegionsResponse call(int callTimeout) throws ServiceException {
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
controller.setCallTimeout(callTimeout);
@@ -1559,13 +1624,36 @@ public class HBaseAdmin implements Admin {
DispatchMergingRegionsRequest request = RequestConverter
.buildDispatchMergingRegionsRequest(encodedNameOfRegionA,
encodedNameOfRegionB, forcible);
- master.dispatchMergingRegions(controller, request);
+ return master.dispatchMergingRegions(controller, request);
} catch (DeserializationException de) {
LOG.error("Could not parse destination server name: " + de);
+ throw new ServiceException(new DoNotRetryIOException(de));
}
- return null;
}
});
+ return new DispatchMergingRegionsFuture(this, tableName, response);
+ }
+
+ private static class DispatchMergingRegionsFuture extends TableFuture<Void> {
+ public DispatchMergingRegionsFuture(
+ final HBaseAdmin admin,
+ final TableName tableName,
+ final DispatchMergingRegionsResponse response) {
+ super(admin, tableName,
+ (response != null && response.hasProcId()) ? response.getProcId() : null);
+ }
+
+ public DispatchMergingRegionsFuture(
+ final HBaseAdmin admin,
+ final TableName tableName,
+ final Long procId) {
+ super(admin, tableName, procId);
+ }
+
+ @Override
+ public String getOperationType() {
+ return "MERGE_REGIONS";
+ }
}
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index fecc3c2..c477063 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -42,8 +42,10 @@ import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+
import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier
.RegionSpecifierType.REGION_NAME;
+
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
@@ -159,7 +161,6 @@ import org.apache.hadoop.hbase.quotas.ThrottleType;
import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
-import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.access.Permission;
import org.apache.hadoop.hbase.security.access.TablePermission;
import org.apache.hadoop.hbase.security.access.UserPermission;
@@ -175,6 +176,7 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import com.google.common.collect.ArrayListMultimap;
@@ -1898,12 +1900,12 @@ public final class ProtobufUtil {
public static void mergeRegions(final RpcController controller,
final AdminService.BlockingInterface admin,
final HRegionInfo region_a, final HRegionInfo region_b,
- final boolean forcible, final User user) throws IOException {
+ final boolean forcible, final UserGroupInformation user) throws IOException {
final MergeRegionsRequest request = RequestConverter.buildMergeRegionsRequest(
region_a.getRegionName(), region_b.getRegionName(),forcible);
if (user != null) {
try {
- user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
+ user.doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
admin.mergeRegions(controller, request);
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java
index 8645dc4..03d310e 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java
@@ -65,7 +65,7 @@ public class MergeRandomAdjacentRegionsOfTableAction extends Action {
}
try {
- admin.mergeRegions(a.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), false);
+ admin.mergeRegionsAsync(a.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), false);
} catch (Exception ex) {
LOG.warn("Merge failed, might be caused by other chaos: " + ex.getMessage());
}
[3/4] hbase git commit: HBASE-14552 Procedure V2: Reimplement
DispatchMergingRegionHandler (Stephen Yuan Jiang)
Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
index 9ed9d7a..faad7e1 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
@@ -1598,6 +1598,115 @@ public final class MasterProcedureProtos {
}
/**
+ * Protobuf enum {@code hbase.pb.DispatchMergingRegionsState}
+ */
+ public enum DispatchMergingRegionsState
+ implements com.google.protobuf.ProtocolMessageEnum {
+ /**
+ * <code>DISPATCH_MERGING_REGIONS_PREPARE = 1;</code>
+ */
+ DISPATCH_MERGING_REGIONS_PREPARE(0, 1),
+ /**
+ * <code>DISPATCH_MERGING_REGIONS_PRE_OPERATION = 2;</code>
+ */
+ DISPATCH_MERGING_REGIONS_PRE_OPERATION(1, 2),
+ /**
+ * <code>DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS = 3;</code>
+ */
+ DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS(2, 3),
+ /**
+ * <code>DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS = 4;</code>
+ */
+ DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS(3, 4),
+ /**
+ * <code>DISPATCH_MERGING_REGIONS_POST_OPERATION = 5;</code>
+ */
+ DISPATCH_MERGING_REGIONS_POST_OPERATION(4, 5),
+ ;
+
+ /**
+ * <code>DISPATCH_MERGING_REGIONS_PREPARE = 1;</code>
+ */
+ public static final int DISPATCH_MERGING_REGIONS_PREPARE_VALUE = 1;
+ /**
+ * <code>DISPATCH_MERGING_REGIONS_PRE_OPERATION = 2;</code>
+ */
+ public static final int DISPATCH_MERGING_REGIONS_PRE_OPERATION_VALUE = 2;
+ /**
+ * <code>DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS = 3;</code>
+ */
+ public static final int DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS_VALUE = 3;
+ /**
+ * <code>DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS = 4;</code>
+ */
+ public static final int DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS_VALUE = 4;
+ /**
+ * <code>DISPATCH_MERGING_REGIONS_POST_OPERATION = 5;</code>
+ */
+ public static final int DISPATCH_MERGING_REGIONS_POST_OPERATION_VALUE = 5;
+
+
+ public final int getNumber() { return value; }
+
+ public static DispatchMergingRegionsState valueOf(int value) {
+ switch (value) {
+ case 1: return DISPATCH_MERGING_REGIONS_PREPARE;
+ case 2: return DISPATCH_MERGING_REGIONS_PRE_OPERATION;
+ case 3: return DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS;
+ case 4: return DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS;
+ case 5: return DISPATCH_MERGING_REGIONS_POST_OPERATION;
+ default: return null;
+ }
+ }
+
+ public static com.google.protobuf.Internal.EnumLiteMap<DispatchMergingRegionsState>
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static com.google.protobuf.Internal.EnumLiteMap<DispatchMergingRegionsState>
+ internalValueMap =
+ new com.google.protobuf.Internal.EnumLiteMap<DispatchMergingRegionsState>() {
+ public DispatchMergingRegionsState findValueByNumber(int number) {
+ return DispatchMergingRegionsState.valueOf(number);
+ }
+ };
+
+ public final com.google.protobuf.Descriptors.EnumValueDescriptor
+ getValueDescriptor() {
+ return getDescriptor().getValues().get(index);
+ }
+ public final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptorForType() {
+ return getDescriptor();
+ }
+ public static final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(14);
+ }
+
+ private static final DispatchMergingRegionsState[] VALUES = values();
+
+ public static DispatchMergingRegionsState valueOf(
+ 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 index;
+ private final int value;
+
+ private DispatchMergingRegionsState(int index, int value) {
+ this.index = index;
+ this.value = value;
+ }
+
+ // @@protoc_insertion_point(enum_scope:hbase.pb.DispatchMergingRegionsState)
+ }
+
+ /**
* Protobuf enum {@code hbase.pb.ServerCrashState}
*/
public enum ServerCrashState
@@ -1725,7 +1834,7 @@ public final class MasterProcedureProtos {
}
public static final com.google.protobuf.Descriptors.EnumDescriptor
getDescriptor() {
- return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(14);
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(15);
}
private static final ServerCrashState[] VALUES = values();
@@ -19078,122 +19187,91 @@ public final class MasterProcedureProtos {
// @@protoc_insertion_point(class_scope:hbase.pb.RestoreSnapshotStateData)
}
- public interface ServerCrashStateDataOrBuilder
+ public interface DispatchMergingRegionsStateDataOrBuilder
extends com.google.protobuf.MessageOrBuilder {
- // required .hbase.pb.ServerName server_name = 1;
- /**
- * <code>required .hbase.pb.ServerName server_name = 1;</code>
- */
- boolean hasServerName();
- /**
- * <code>required .hbase.pb.ServerName server_name = 1;</code>
- */
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServerName();
+ // required .hbase.pb.UserInformation user_info = 1;
/**
- * <code>required .hbase.pb.ServerName server_name = 1;</code>
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
*/
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder();
-
- // optional bool distributed_log_replay = 2;
+ boolean hasUserInfo();
/**
- * <code>optional bool distributed_log_replay = 2;</code>
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
*/
- boolean hasDistributedLogReplay();
+ org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo();
/**
- * <code>optional bool distributed_log_replay = 2;</code>
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
*/
- boolean getDistributedLogReplay();
+ org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
- // repeated .hbase.pb.RegionInfo regions_on_crashed_server = 3;
- /**
- * <code>repeated .hbase.pb.RegionInfo regions_on_crashed_server = 3;</code>
- */
- java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>
- getRegionsOnCrashedServerList();
- /**
- * <code>repeated .hbase.pb.RegionInfo regions_on_crashed_server = 3;</code>
- */
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionsOnCrashedServer(int index);
+ // required .hbase.pb.TableName table_name = 2;
/**
- * <code>repeated .hbase.pb.RegionInfo regions_on_crashed_server = 3;</code>
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
*/
- int getRegionsOnCrashedServerCount();
+ boolean hasTableName();
/**
- * <code>repeated .hbase.pb.RegionInfo regions_on_crashed_server = 3;</code>
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
*/
- java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
- getRegionsOnCrashedServerOrBuilderList();
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
/**
- * <code>repeated .hbase.pb.RegionInfo regions_on_crashed_server = 3;</code>
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
*/
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionsOnCrashedServerOrBuilder(
- int index);
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
- // repeated .hbase.pb.RegionInfo regions_assigned = 4;
+ // repeated .hbase.pb.RegionInfo region_info = 3;
/**
- * <code>repeated .hbase.pb.RegionInfo regions_assigned = 4;</code>
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
*/
java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>
- getRegionsAssignedList();
+ getRegionInfoList();
/**
- * <code>repeated .hbase.pb.RegionInfo regions_assigned = 4;</code>
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
*/
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionsAssigned(int index);
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
/**
- * <code>repeated .hbase.pb.RegionInfo regions_assigned = 4;</code>
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
*/
- int getRegionsAssignedCount();
+ int getRegionInfoCount();
/**
- * <code>repeated .hbase.pb.RegionInfo regions_assigned = 4;</code>
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
*/
java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
- getRegionsAssignedOrBuilderList();
+ getRegionInfoOrBuilderList();
/**
- * <code>repeated .hbase.pb.RegionInfo regions_assigned = 4;</code>
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
*/
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionsAssignedOrBuilder(
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
int index);
- // optional bool carrying_meta = 5;
- /**
- * <code>optional bool carrying_meta = 5;</code>
- */
- boolean hasCarryingMeta();
- /**
- * <code>optional bool carrying_meta = 5;</code>
- */
- boolean getCarryingMeta();
-
- // optional bool should_split_wal = 6 [default = true];
+ // optional bool forcible = 4;
/**
- * <code>optional bool should_split_wal = 6 [default = true];</code>
+ * <code>optional bool forcible = 4;</code>
*/
- boolean hasShouldSplitWal();
+ boolean hasForcible();
/**
- * <code>optional bool should_split_wal = 6 [default = true];</code>
+ * <code>optional bool forcible = 4;</code>
*/
- boolean getShouldSplitWal();
+ boolean getForcible();
}
/**
- * Protobuf type {@code hbase.pb.ServerCrashStateData}
+ * Protobuf type {@code hbase.pb.DispatchMergingRegionsStateData}
*/
- public static final class ServerCrashStateData extends
+ public static final class DispatchMergingRegionsStateData extends
com.google.protobuf.GeneratedMessage
- implements ServerCrashStateDataOrBuilder {
- // Use ServerCrashStateData.newBuilder() to construct.
- private ServerCrashStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ implements DispatchMergingRegionsStateDataOrBuilder {
+ // Use DispatchMergingRegionsStateData.newBuilder() to construct.
+ private DispatchMergingRegionsStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
super(builder);
this.unknownFields = builder.getUnknownFields();
}
- private ServerCrashStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+ private DispatchMergingRegionsStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
- private static final ServerCrashStateData defaultInstance;
- public static ServerCrashStateData getDefaultInstance() {
+ private static final DispatchMergingRegionsStateData defaultInstance;
+ public static DispatchMergingRegionsStateData getDefaultInstance() {
return defaultInstance;
}
- public ServerCrashStateData getDefaultInstanceForType() {
+ public DispatchMergingRegionsStateData getDefaultInstanceForType() {
return defaultInstance;
}
@@ -19203,7 +19281,7 @@ public final class MasterProcedureProtos {
getUnknownFields() {
return this.unknownFields;
}
- private ServerCrashStateData(
+ private DispatchMergingRegionsStateData(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -19227,47 +19305,42 @@ public final class MasterProcedureProtos {
break;
}
case 10: {
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null;
+ org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder subBuilder = null;
if (((bitField0_ & 0x00000001) == 0x00000001)) {
- subBuilder = serverName_.toBuilder();
+ subBuilder = userInfo_.toBuilder();
}
- serverName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry);
+ userInfo_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.PARSER, extensionRegistry);
if (subBuilder != null) {
- subBuilder.mergeFrom(serverName_);
- serverName_ = subBuilder.buildPartial();
+ subBuilder.mergeFrom(userInfo_);
+ userInfo_ = subBuilder.buildPartial();
}
bitField0_ |= 0x00000001;
break;
}
- case 16: {
+ case 18: {
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ subBuilder = tableName_.toBuilder();
+ }
+ tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(tableName_);
+ tableName_ = subBuilder.buildPartial();
+ }
bitField0_ |= 0x00000002;
- distributedLogReplay_ = input.readBool();
break;
}
case 26: {
if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
- regionsOnCrashedServer_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>();
+ regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>();
mutable_bitField0_ |= 0x00000004;
}
- regionsOnCrashedServer_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
- break;
- }
- case 34: {
- if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
- regionsAssigned_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>();
- mutable_bitField0_ |= 0x00000008;
- }
- regionsAssigned_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
+ regionInfo_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
break;
}
- case 40: {
+ case 32: {
bitField0_ |= 0x00000004;
- carryingMeta_ = input.readBool();
- break;
- }
- case 48: {
- bitField0_ |= 0x00000008;
- shouldSplitWal_ = input.readBool();
+ forcible_ = input.readBool();
break;
}
}
@@ -19279,10 +19352,7 @@ public final class MasterProcedureProtos {
e.getMessage()).setUnfinishedMessage(this);
} finally {
if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
- regionsOnCrashedServer_ = java.util.Collections.unmodifiableList(regionsOnCrashedServer_);
- }
- if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
- regionsAssigned_ = java.util.Collections.unmodifiableList(regionsAssigned_);
+ regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
}
this.unknownFields = unknownFields.build();
makeExtensionsImmutable();
@@ -19290,61 +19360,1348 @@ public final class MasterProcedureProtos {
}
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
- return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ServerCrashStateData_descriptor;
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
- return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ServerCrashStateData_fieldAccessorTable
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_fieldAccessorTable
.ensureFieldAccessorsInitialized(
- org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashStateData.Builder.class);
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.Builder.class);
}
- public static com.google.protobuf.Parser<ServerCrashStateData> PARSER =
- new com.google.protobuf.AbstractParser<ServerCrashStateData>() {
- public ServerCrashStateData parsePartialFrom(
+ public static com.google.protobuf.Parser<DispatchMergingRegionsStateData> PARSER =
+ new com.google.protobuf.AbstractParser<DispatchMergingRegionsStateData>() {
+ public DispatchMergingRegionsStateData parsePartialFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
- return new ServerCrashStateData(input, extensionRegistry);
+ return new DispatchMergingRegionsStateData(input, extensionRegistry);
}
};
@java.lang.Override
- public com.google.protobuf.Parser<ServerCrashStateData> getParserForType() {
+ public com.google.protobuf.Parser<DispatchMergingRegionsStateData> getParserForType() {
return PARSER;
}
private int bitField0_;
- // required .hbase.pb.ServerName server_name = 1;
- public static final int SERVER_NAME_FIELD_NUMBER = 1;
- private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName serverName_;
+ // required .hbase.pb.UserInformation user_info = 1;
+ public static final int USER_INFO_FIELD_NUMBER = 1;
+ private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_;
/**
- * <code>required .hbase.pb.ServerName server_name = 1;</code>
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
*/
- public boolean hasServerName() {
+ public boolean hasUserInfo() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
/**
- * <code>required .hbase.pb.ServerName server_name = 1;</code>
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
*/
- public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServerName() {
- return serverName_;
+ public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+ return userInfo_;
}
/**
- * <code>required .hbase.pb.ServerName server_name = 1;</code>
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
*/
- public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder() {
- return serverName_;
+ public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+ return userInfo_;
}
- // optional bool distributed_log_replay = 2;
- public static final int DISTRIBUTED_LOG_REPLAY_FIELD_NUMBER = 2;
- private boolean distributedLogReplay_;
+ // required .hbase.pb.TableName table_name = 2;
+ public static final int TABLE_NAME_FIELD_NUMBER = 2;
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
/**
- * <code>optional bool distributed_log_replay = 2;</code>
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
*/
- public boolean hasDistributedLogReplay() {
+ public boolean hasTableName() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+ return tableName_;
+ }
+ /**
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+ return tableName_;
+ }
+
+ // repeated .hbase.pb.RegionInfo region_info = 3;
+ public static final int REGION_INFO_FIELD_NUMBER = 3;
+ private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_;
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+ */
+ public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
+ return regionInfo_;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+ */
+ public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
+ getRegionInfoOrBuilderList() {
+ return regionInfo_;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+ */
+ public int getRegionInfoCount() {
+ return regionInfo_.size();
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
+ return regionInfo_.get(index);
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+ int index) {
+ return regionInfo_.get(index);
+ }
+
+ // optional bool forcible = 4;
+ public static final int FORCIBLE_FIELD_NUMBER = 4;
+ private boolean forcible_;
+ /**
+ * <code>optional bool forcible = 4;</code>
+ */
+ public boolean hasForcible() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional bool forcible = 4;</code>
+ */
+ public boolean getForcible() {
+ return forcible_;
+ }
+
+ private void initFields() {
+ userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+ tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+ regionInfo_ = java.util.Collections.emptyList();
+ forcible_ = false;
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ if (!hasUserInfo()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasTableName()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!getUserInfo().isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!getTableName().isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ for (int i = 0; i < getRegionInfoCount(); i++) {
+ if (!getRegionInfo(i).isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeMessage(1, userInfo_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeMessage(2, tableName_);
+ }
+ for (int i = 0; i < regionInfo_.size(); i++) {
+ output.writeMessage(3, regionInfo_.get(i));
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeBool(4, forcible_);
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(1, userInfo_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(2, tableName_);
+ }
+ for (int i = 0; i < regionInfo_.size(); i++) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(3, regionInfo_.get(i));
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBoolSize(4, forcible_);
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ @java.lang.Override
+ public boolean equals(final java.lang.Object obj) {
+ if (obj == this) {
+ return true;
+ }
+ if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData)) {
+ return super.equals(obj);
+ }
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData) obj;
+
+ boolean result = true;
+ result = result && (hasUserInfo() == other.hasUserInfo());
+ if (hasUserInfo()) {
+ result = result && getUserInfo()
+ .equals(other.getUserInfo());
+ }
+ result = result && (hasTableName() == other.hasTableName());
+ if (hasTableName()) {
+ result = result && getTableName()
+ .equals(other.getTableName());
+ }
+ result = result && getRegionInfoList()
+ .equals(other.getRegionInfoList());
+ result = result && (hasForcible() == other.hasForcible());
+ if (hasForcible()) {
+ result = result && (getForcible()
+ == other.getForcible());
+ }
+ result = result &&
+ getUnknownFields().equals(other.getUnknownFields());
+ return result;
+ }
+
+ private int memoizedHashCode = 0;
+ @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 (hasTableName()) {
+ hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+ hash = (53 * hash) + getTableName().hashCode();
+ }
+ if (getRegionInfoCount() > 0) {
+ hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
+ hash = (53 * hash) + getRegionInfoList().hashCode();
+ }
+ if (hasForcible()) {
+ hash = (37 * hash) + FORCIBLE_FIELD_NUMBER;
+ hash = (53 * hash) + hashBoolean(getForcible());
+ }
+ hash = (29 * hash) + getUnknownFields().hashCode();
+ memoizedHashCode = hash;
+ return hash;
+ }
+
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * Protobuf type {@code hbase.pb.DispatchMergingRegionsStateData}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateDataOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.Builder.class);
+ }
+
+ // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ getUserInfoFieldBuilder();
+ getTableNameFieldBuilder();
+ getRegionInfoFieldBuilder();
+ }
+ }
+ private static Builder create() {
+ return new Builder();
+ }
+
+ public Builder clear() {
+ super.clear();
+ if (userInfoBuilder_ == null) {
+ userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+ } else {
+ userInfoBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ if (tableNameBuilder_ == null) {
+ tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+ } else {
+ tableNameBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000002);
+ if (regionInfoBuilder_ == null) {
+ regionInfo_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000004);
+ } else {
+ regionInfoBuilder_.clear();
+ }
+ forcible_ = false;
+ bitField0_ = (bitField0_ & ~0x00000008);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_descriptor;
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData getDefaultInstanceForType() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.getDefaultInstance();
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData build() {
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData buildPartial() {
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData(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 (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ if (tableNameBuilder_ == null) {
+ result.tableName_ = tableName_;
+ } else {
+ result.tableName_ = tableNameBuilder_.build();
+ }
+ if (regionInfoBuilder_ == null) {
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
+ bitField0_ = (bitField0_ & ~0x00000004);
+ }
+ result.regionInfo_ = regionInfo_;
+ } else {
+ result.regionInfo_ = regionInfoBuilder_.build();
+ }
+ if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.forcible_ = forcible_;
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData) {
+ return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData other) {
+ if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.getDefaultInstance()) return this;
+ if (other.hasUserInfo()) {
+ mergeUserInfo(other.getUserInfo());
+ }
+ if (other.hasTableName()) {
+ mergeTableName(other.getTableName());
+ }
+ if (regionInfoBuilder_ == null) {
+ if (!other.regionInfo_.isEmpty()) {
+ if (regionInfo_.isEmpty()) {
+ regionInfo_ = other.regionInfo_;
+ bitField0_ = (bitField0_ & ~0x00000004);
+ } else {
+ ensureRegionInfoIsMutable();
+ regionInfo_.addAll(other.regionInfo_);
+ }
+ onChanged();
+ }
+ } else {
+ if (!other.regionInfo_.isEmpty()) {
+ if (regionInfoBuilder_.isEmpty()) {
+ regionInfoBuilder_.dispose();
+ regionInfoBuilder_ = null;
+ regionInfo_ = other.regionInfo_;
+ bitField0_ = (bitField0_ & ~0x00000004);
+ regionInfoBuilder_ =
+ com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+ getRegionInfoFieldBuilder() : null;
+ } else {
+ regionInfoBuilder_.addAllMessages(other.regionInfo_);
+ }
+ }
+ }
+ if (other.hasForcible()) {
+ setForcible(other.getForcible());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ if (!hasUserInfo()) {
+
+ return false;
+ }
+ if (!hasTableName()) {
+
+ return false;
+ }
+ if (!getUserInfo().isInitialized()) {
+
+ return false;
+ }
+ if (!getTableName().isInitialized()) {
+
+ return false;
+ }
+ for (int i = 0; i < getRegionInfoCount(); i++) {
+ if (!getRegionInfo(i).isInitialized()) {
+
+ return false;
+ }
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ // required .hbase.pb.UserInformation user_info = 1;
+ private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.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.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+ if (userInfoBuilder_ == null) {
+ return userInfo_;
+ } else {
+ return userInfoBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ public Builder setUserInfo(org.apache.hadoop.hbase.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.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.protobuf.generated.RPCProtos.UserInformation value) {
+ if (userInfoBuilder_ == null) {
+ if (((bitField0_ & 0x00000001) == 0x00000001) &&
+ userInfo_ != org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance()) {
+ userInfo_ =
+ org.apache.hadoop.hbase.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_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+ onChanged();
+ } else {
+ userInfoBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ public org.apache.hadoop.hbase.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.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+ if (userInfoBuilder_ != null) {
+ return userInfoBuilder_.getMessageOrBuilder();
+ } else {
+ return userInfo_;
+ }
+ }
+ /**
+ * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder>
+ getUserInfoFieldBuilder() {
+ if (userInfoBuilder_ == null) {
+ userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
+ userInfo_,
+ getParentForChildren(),
+ isClean());
+ userInfo_ = null;
+ }
+ return userInfoBuilder_;
+ }
+
+ // required .hbase.pb.TableName table_name = 2;
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+ /**
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
+ */
+ public boolean hasTableName() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+ if (tableNameBuilder_ == null) {
+ return tableName_;
+ } else {
+ return tableNameBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
+ */
+ public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+ if (tableNameBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ tableName_ = value;
+ onChanged();
+ } else {
+ tableNameBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
+ */
+ public Builder setTableName(
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+ if (tableNameBuilder_ == null) {
+ tableName_ = builderForValue.build();
+ onChanged();
+ } else {
+ tableNameBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
+ */
+ public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+ if (tableNameBuilder_ == null) {
+ if (((bitField0_ & 0x00000002) == 0x00000002) &&
+ tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+ tableName_ =
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+ } else {
+ tableName_ = value;
+ }
+ onChanged();
+ } else {
+ tableNameBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
+ */
+ public Builder clearTableName() {
+ if (tableNameBuilder_ == null) {
+ tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+ onChanged();
+ } else {
+ tableNameBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000002);
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+ bitField0_ |= 0x00000002;
+ onChanged();
+ return getTableNameFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+ if (tableNameBuilder_ != null) {
+ return tableNameBuilder_.getMessageOrBuilder();
+ } else {
+ return tableName_;
+ }
+ }
+ /**
+ * <code>required .hbase.pb.TableName table_name = 2;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>
+ getTableNameFieldBuilder() {
+ if (tableNameBuilder_ == null) {
+ tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+ tableName_,
+ getParentForChildren(),
+ isClean());
+ tableName_ = null;
+ }
+ return tableNameBuilder_;
+ }
+
+ // repeated .hbase.pb.RegionInfo region_info = 3;
+ private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_ =
+ java.util.Collections.emptyList();
+ private void ensureRegionInfoIsMutable() {
+ if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+ regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>(regionInfo_);
+ bitField0_ |= 0x00000004;
+ }
+ }
+
+ private com.google.protobuf.RepeatedFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
+
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+ */
+ public java.util.List<org.apache.hadoop.hbase.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 = 3;</code>
+ */
+ public int getRegionInfoCount() {
+ if (regionInfoBuilder_ == null) {
+ return regionInfo_.size();
+ } else {
+ return regionInfoBuilder_.getCount();
+ }
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+ */
+ public org.apache.hadoop.hbase.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 = 3;</code>
+ */
+ public Builder setRegionInfo(
+ int index, org.apache.hadoop.hbase.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 = 3;</code>
+ */
+ public Builder setRegionInfo(
+ int index, org.apache.hadoop.hbase.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 = 3;</code>
+ */
+ public Builder addRegionInfo(org.apache.hadoop.hbase.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 = 3;</code>
+ */
+ public Builder addRegionInfo(
+ int index, org.apache.hadoop.hbase.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 = 3;</code>
+ */
+ public Builder addRegionInfo(
+ org.apache.hadoop.hbase.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 = 3;</code>
+ */
+ public Builder addRegionInfo(
+ int index, org.apache.hadoop.hbase.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 = 3;</code>
+ */
+ public Builder addAllRegionInfo(
+ java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> values) {
+ if (regionInfoBuilder_ == null) {
+ ensureRegionInfoIsMutable();
+ super.addAll(values, regionInfo_);
+ onChanged();
+ } else {
+ regionInfoBuilder_.addAllMessages(values);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+ */
+ public Builder clearRegionInfo() {
+ if (regionInfoBuilder_ == null) {
+ regionInfo_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000004);
+ onChanged();
+ } else {
+ regionInfoBuilder_.clear();
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</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 = 3;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder(
+ int index) {
+ return getRegionInfoFieldBuilder().getBuilder(index);
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+ */
+ public org.apache.hadoop.hbase.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 = 3;</code>
+ */
+ public java.util.List<? extends org.apache.hadoop.hbase.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 = 3;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder addRegionInfoBuilder() {
+ return getRegionInfoFieldBuilder().addBuilder(
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance());
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder addRegionInfoBuilder(
+ int index) {
+ return getRegionInfoFieldBuilder().addBuilder(
+ index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance());
+ }
+ /**
+ * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+ */
+ public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder>
+ getRegionInfoBuilderList() {
+ return getRegionInfoFieldBuilder().getBuilderList();
+ }
+ private com.google.protobuf.RepeatedFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
+ getRegionInfoFieldBuilder() {
+ if (regionInfoBuilder_ == null) {
+ regionInfoBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
+ regionInfo_,
+ ((bitField0_ & 0x00000004) == 0x00000004),
+ getParentForChildren(),
+ isClean());
+ regionInfo_ = null;
+ }
+ return regionInfoBuilder_;
+ }
+
+ // optional bool forcible = 4;
+ private boolean forcible_ ;
+ /**
+ * <code>optional bool forcible = 4;</code>
+ */
+ public boolean hasForcible() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional bool forcible = 4;</code>
+ */
+ public boolean getForcible() {
+ return forcible_;
+ }
+ /**
+ * <code>optional bool forcible = 4;</code>
+ */
+ public Builder setForcible(boolean value) {
+ bitField0_ |= 0x00000008;
+ forcible_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional bool forcible = 4;</code>
+ */
+ public Builder clearForcible() {
+ bitField0_ = (bitField0_ & ~0x00000008);
+ forcible_ = false;
+ onChanged();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:hbase.pb.DispatchMergingRegionsStateData)
+ }
+
+ static {
+ defaultInstance = new DispatchMergingRegionsStateData(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:hbase.pb.DispatchMergingRegionsStateData)
+ }
+
+ public interface ServerCrashStateDataOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // required .hbase.pb.ServerName server_name = 1;
+ /**
+ * <code>required .hbase.pb.ServerName server_name = 1;</code>
+ */
+ boolean hasServerName();
+ /**
+ * <code>required .hbase.pb.ServerName server_name = 1;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServerName();
+ /**
+ * <code>required .hbase.pb.ServerName server_name = 1;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder();
+
+ // optional bool distributed_log_replay = 2;
+ /**
+ * <code>optional bool distributed_log_replay = 2;</code>
+ */
+ boolean hasDistributedLogReplay();
+ /**
+ * <code>optional bool distributed_log_replay = 2;</code>
+ */
+ boolean getDistributedLogReplay();
+
+ // repeated .hbase.pb.RegionInfo regions_on_crashed_server = 3;
+ /**
+ * <code>repeated .hbase.pb.RegionInfo regions_on_crashed_server = 3;</code>
+ */
+ java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>
+ getRegionsOnCrashedServerList();
+ /**
+ * <code>repeated .hbase.pb.RegionInfo regions_on_crashed_server = 3;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionsOnCrashedServer(int index);
+ /**
+ * <code>repeated .hbase.pb.RegionInfo regions_on_crashed_server = 3;</code>
+ */
+ int getRegionsOnCrashedServerCount();
+ /**
+ * <code>repeated .hbase.pb.RegionInfo regions_on_crashed_server = 3;</code>
+ */
+ java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
+ getRegionsOnCrashedServerOrBuilderList();
+ /**
+ * <code>repeated .hbase.pb.RegionInfo regions_on_crashed_server = 3;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionsOnCrashedServerOrBuilder(
+ int index);
+
+ // repeated .hbase.pb.RegionInfo regions_assigned = 4;
+ /**
+ * <code>repeated .hbase.pb.RegionInfo regions_assigned = 4;</code>
+ */
+ java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>
+ getRegionsAssignedList();
+ /**
+ * <code>repeated .hbase.pb.RegionInfo regions_assigned = 4;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionsAssigned(int index);
+ /**
+ * <code>repeated .hbase.pb.RegionInfo regions_assigned = 4;</code>
+ */
+ int getRegionsAssignedCount();
+ /**
+ * <code>repeated .hbase.pb.RegionInfo regions_assigned = 4;</code>
+ */
+ java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>
+ getRegionsAssignedOrBuilderList();
+ /**
+ * <code>repeated .hbase.pb.RegionInfo regions_assigned = 4;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionsAssignedOrBuilder(
+ int index);
+
+ // optional bool carrying_meta = 5;
+ /**
+ * <code>optional bool carrying_meta = 5;</code>
+ */
+ boolean hasCarryingMeta();
+ /**
+ * <code>optional bool carrying_meta = 5;</code>
+ */
+ boolean getCarryingMeta();
+
+ // optional bool should_split_wal = 6 [default = true];
+ /**
+ * <code>optional bool should_split_wal = 6 [default = true];</code>
+ */
+ boolean hasShouldSplitWal();
+ /**
+ * <code>optional bool should_split_wal = 6 [default = true];</code>
+ */
+ boolean getShouldSplitWal();
+ }
+ /**
+ * Protobuf type {@code hbase.pb.ServerCrashStateData}
+ */
+ public static final class ServerCrashStateData extends
+ com.google.protobuf.GeneratedMessage
+ implements ServerCrashStateDataOrBuilder {
+ // Use ServerCrashStateData.newBuilder() to construct.
+ private ServerCrashStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private ServerCrashStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final ServerCrashStateData defaultInstance;
+ public static ServerCrashStateData getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public ServerCrashStateData getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private ServerCrashStateData(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ initFields();
+ int mutable_bitField0_ = 0;
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ 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.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ subBuilder = serverName_.toBuilder();
+ }
+ serverName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(serverName_);
+ serverName_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000001;
+ break;
+ }
+ case 16: {
+ bitField0_ |= 0x00000002;
+ distributedLogReplay_ = input.readBool();
+ break;
+ }
+ case 26: {
+ if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+ regionsOnCrashedServer_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>();
+ mutable_bitField0_ |= 0x00000004;
+ }
+ regionsOnCrashedServer_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
+ break;
+ }
+ case 34: {
+ if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+ regionsAssigned_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>();
+ mutable_bitField0_ |= 0x00000008;
+ }
+ regionsAssigned_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
+ break;
+ }
+ case 40: {
+ bitField0_ |= 0x00000004;
+ carryingMeta_ = input.readBool();
+ break;
+ }
+ case 48: {
+ bitField0_ |= 0x00000008;
+ shouldSplitWal_ = input.readBool();
+ break;
+ }
+ }
+ }
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e.getMessage()).setUnfinishedMessage(this);
+ } finally {
+ if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+ regionsOnCrashedServer_ = java.util.Collections.unmodifiableList(regionsOnCrashedServer_);
+ }
+ if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+ regionsAssigned_ = java.util.Collections.unmodifiableList(regionsAssigned_);
+ }
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ServerCrashStateData_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ServerCrashStateData_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashStateData.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser<ServerCrashStateData> PARSER =
+ new com.google.protobuf.AbstractParser<ServerCrashStateData>() {
+ public ServerCrashStateData parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new ServerCrashStateData(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser<ServerCrashStateData> getParserForType() {
+ return PARSER;
+ }
+
+ private int bitField0_;
+ // required .hbase.pb.ServerName server_name = 1;
+ public static final int SERVER_NAME_FIELD_NUMBER = 1;
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName serverName_;
+ /**
+ * <code>required .hbase.pb.ServerName server_name = 1;</code>
+ */
+ public boolean hasServerName() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>required .hbase.pb.ServerName server_name = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServerName() {
+ return serverName_;
+ }
+ /**
+ * <code>required .hbase.pb.ServerName server_name = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder() {
+ return serverName_;
+ }
+
+ // optional bool distributed_log_replay = 2;
+ public static final int DISTRIBUTED_LOG_REPLAY_FIELD_NUMBER = 2;
+ private boolean distributedLogReplay_;
+ /**
+ * <code>optional bool distributed_log_replay = 2;</code>
+ */
+ public boolean hasDistributedLogReplay() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
/**
@@ -20745,6 +22102,11 @@ public final class MasterProcedureProtos {
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_hbase_pb_RestoreSnapshotStateData_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_hbase_pb_DispatchMergingRegionsStateData_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_hbase_pb_DispatchMergingRegionsStateData_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
internal_static_hbase_pb_ServerCrashStateData_descriptor;
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -20832,102 +22194,113 @@ public final class MasterProcedureProtos {
"o\0221\n\023region_info_for_add\030\006 \003(\0132\024.hbase.p" +
"b.RegionInfo\022T\n!parent_to_child_regions_" +
"pair_list\030\007 \003(\0132).hbase.pb.RestoreParent" +
- "ToChildRegionsPair\"\201\002\n\024ServerCrashStateD" +
- "ata\022)\n\013server_name\030\001 \002(\0132\024.hbase.pb.Serv" +
- "erName\022\036\n\026distributed_log_replay\030\002 \001(\010\0227" +
- "\n\031regions_on_crashed_server\030\003 \003(\0132\024.hbas" +
- "e.pb.RegionInfo\022.\n\020regions_assigned\030\004 \003(" +
- "\0132\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_" +
- "OPERATION\020\001\022 \n\034CREATE_TABLE_WRITE_FS_LAY" +
- "OUT\020\002\022\034\n\030CREATE_TABLE_ADD_TO_META\020\003\022\037\n\033C" +
- "REATE_TABLE_ASSIGN_REGIONS\020\004\022\"\n\036CREATE_T" +
- "ABLE_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_" +
- "TABLE_DESCRIPTOR\020\003\022&\n\"MODIFY_TABLE_REMOV",
- "E_REPLICA_COLUMN\020\004\022!\n\035MODIFY_TABLE_DELET" +
- "E_FS_LAYOUT\020\005\022\037\n\033MODIFY_TABLE_POST_OPERA" +
- "TION\020\006\022#\n\037MODIFY_TABLE_REOPEN_ALL_REGION" +
- "S\020\007*\212\002\n\022TruncateTableState\022 \n\034TRUNCATE_T" +
- "ABLE_PRE_OPERATION\020\001\022#\n\037TRUNCATE_TABLE_R" +
- "EMOVE_FROM_META\020\002\022\"\n\036TRUNCATE_TABLE_CLEA" +
- "R_FS_LAYOUT\020\003\022#\n\037TRUNCATE_TABLE_CREATE_F" +
- "S_LAYOUT\020\004\022\036\n\032TRUNCATE_TABLE_ADD_TO_META" +
- "\020\005\022!\n\035TRUNCATE_TABLE_ASSIGN_REGIONS\020\006\022!\n" +
- "\035TRUNCATE_TABLE_POST_OPERATION\020\007*\337\001\n\020Del",
- "eteTableState\022\036\n\032DELETE_TABLE_PRE_OPERAT" +
- "ION\020\001\022!\n\035DELETE_TABLE_REMOVE_FROM_META\020\002" +
- "\022 \n\034DELETE_TABLE_CLEAR_FS_LAYOUT\020\003\022\"\n\036DE" +
- "LETE_TABLE_UPDATE_DESC_CACHE\020\004\022!\n\035DELETE" +
- "_TABLE_UNASSIGN_REGIONS\020\005\022\037\n\033DELETE_TABL" +
- "E_POST_OPERATION\020\006*\320\001\n\024CreateNamespaceSt" +
- "ate\022\034\n\030CREATE_NAMESPACE_PREPARE\020\001\022%\n!CRE" +
- "ATE_NAMESPACE_CREATE_DIRECTORY\020\002\022)\n%CREA" +
- "TE_NAMESPACE_INSERT_INTO_NS_TABLE\020\003\022\036\n\032C" +
- "REATE_NAMESPACE_UPDATE_ZK\020\004\022(\n$CREATE_NA",
- "MESPACE_SET_NAMESPACE_QUOTA\020\005*z\n\024ModifyN" +
- "amespaceState\022\034\n\030MODIFY_NAMESPACE_PREPAR" +
- "E\020\001\022$\n MODIFY_NAMESPACE_UPDATE_NS_TABLE\020" +
- "\002\022\036\n\032MODIFY_NAMESPACE_UPDATE_ZK\020\003*\332\001\n\024De" +
- "leteNamespaceState\022\034\n\030DELETE_NAMESPACE_P" +
- "REPARE\020\001\022)\n%DELETE_NAMESPACE_DELETE_FROM" +
- "_NS_TABLE\020\002\022#\n\037DELETE_NAMESPACE_REMOVE_F" +
- "ROM_ZK\020\003\022\'\n#DELETE_NAMESPACE_DELETE_DIRE" +
- "CTORIES\020\004\022+\n\'DELETE_NAMESPACE_REMOVE_NAM" +
- "ESPACE_QUOTA\020\005*\331\001\n\024AddColumnFamilyState\022",
- "\035\n\031ADD_COLUMN_FAMILY_PREPARE\020\001\022#\n\037ADD_CO" +
- "LUMN_FAMILY_PRE_OPERATION\020\002\022-\n)ADD_COLUM" +
- "N_FAMILY_UPDATE_TABLE_DESCRIPTOR\020\003\022$\n AD" +
- "D_COLUMN_FAMILY_POST_OPERATION\020\004\022(\n$ADD_" +
- "COLUMN_FAMILY_REOPEN_ALL_REGIONS\020\005*\353\001\n\027M" +
- "odifyColumnFamilyState\022 \n\034MODIFY_COLUMN_" +
- "FAMILY_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_COL" +
- "UMN_FAMILY_POST_OPERATION\020\004\022+\n\'MODIFY_CO",
- "LUMN_FAMILY_REOPEN_ALL_REGIONS\020\005*\226\002\n\027Del" +
- "eteColumnFamilyState\022 \n\034DELETE_COLUMN_FA" +
- "MILY_PREPARE\020\001\022&\n\"DELETE_COLUMN_FAMILY_P" +
- "RE_OPERATION\020\002\0220\n,DELETE_COLUMN_FAMILY_U" +
- "PDATE_TABLE_DESCRIPTOR\020\003\022)\n%DELETE_COLUM" +
- "N_FAMILY_DELETE_FS_LAYOUT\020\004\022\'\n#DELETE_CO" +
- "LUMN_FAMILY_POST_OPERATION\020\005\022+\n\'DELETE_C" +
- "OLUMN_FAMILY_REOPEN_ALL_REGIONS\020\006*\350\001\n\020En" +
- "ableTableState\022\030\n\024ENABLE_TABLE_PREPARE\020\001" +
- "\022\036\n\032ENABLE_TABLE_PRE_OPERATION\020\002\022)\n%ENAB",
- "LE_TABLE_SET_ENABLING_TABLE_STATE\020\003\022$\n E" +
- "NABLE_TABLE_MARK_REGIONS_ONLINE\020\004\022(\n$ENA" +
- "BLE_TABLE_SET_ENABLED_TABLE_STATE\020\005\022\037\n\033E" +
- "NABLE_TABLE_POST_OPERATION\020\006*\362\001\n\021Disable" +
- "TableState\022\031\n\025DISABLE_TABLE_PREPARE\020\001\022\037\n" +
- "\033DISABLE_TABLE_PRE_OPERATION\020\002\022+\n\'DISABL" +
- "E_TABLE_SET_DISABLING_TABLE_STATE\020\003\022&\n\"D" +
- "ISABLE_TABLE_MARK_REGIONS_OFFLINE\020\004\022*\n&D" +
- "ISABLE_TABLE_SET_DISABLED_TABLE_STATE\020\005\022" +
- " \n\034DISABLE_TABLE_POST_OPERATION\020\006*\346\001\n\022Cl",
- "oneSnapshotState\022 \n\034CLONE_SNAPSHOT_PRE_O" +
- "PERATION\020\001\022\"\n\036CLONE_SNAPSHOT_WRITE_FS_LA" +
- "YOUT\020\002\022\036\n\032CLONE_SNAPSHOT_ADD_TO_META\020\003\022!" +
- "\n\035CLONE_SNAPSHOT_ASSIGN_REGIONS\020\004\022$\n CLO" +
- "NE_SNAPSHOT_UPDATE_DESC_CACHE\020\005\022!\n\035CLONE" +
- "_SNAPSHOT_POST_OPERATION\020\006*\260\001\n\024RestoreSn" +
- "apshotState\022\"\n\036RESTORE_SNAPSHOT_PRE_OPER" +
- "ATION\020\001\022,\n(RESTORE_SNAPSHOT_UPDATE_TABLE" +
- "_DESCRIPTOR\020\002\022$\n RESTORE_SNAPSHOT_WRITE_" +
- "FS_LAYOUT\020\003\022 \n\034RESTORE_SNAPSHOT_UPDATE_M",
- "ETA\020\004*\234\002\n\020ServerCrashState\022\026\n\022SERVER_CRA" +
- "SH_START\020\001\022\035\n\031SERVER_CRASH_PROCESS_META\020" +
- "\002\022\034\n\030SERVER_CRASH_GET_REGIONS\020\003\022\036\n\032SERVE" +
- "R_CRASH_NO_SPLIT_LOGS\020\004\022\033\n\027SERVER_CRASH_" +
- "SPLIT_LOGS\020\005\022#\n\037SERVER_CRASH_PREPARE_LOG" +
- "_REPLAY\020\006\022\027\n\023SERVER_CRASH_ASSIGN\020\010\022\037\n\033SE" +
- "RVER_CRASH_WAIT_ON_ASSIGN\020\t\022\027\n\023SERVER_CR" +
- "ASH_FINISH\020dBK\n*org.apache.hadoop.hbase." +
- "protobuf.generatedB\025MasterProcedureProto" +
- "sH\001\210\001\001\240\001\001"
+ "ToChildRegionsPair\"\265\001\n\037DispatchMergingRe" +
+ "gionsStateData\022,\n\tuser_info\030\001 \002(\0132\031.hbas" +
+ "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\"\201\002\n\024ServerCrashStateData\022)\n\013server_na",
+ "me\030\001 \002(\0132\024.hbase.pb.ServerName\022\036\n\026distri" +
+ "buted_log_replay\030\002 \001(\010\0227\n\031regions_on_cra" +
+ "shed_server\030\003 \003(\0132\024.hbase.pb.RegionInfo\022" +
+ ".\n\020regions_assigned\030\004 \003(\0132\024.hbase.pb.Reg" +
+ "ionInfo\022\025\n\rcarrying_meta\030\005 \001(\010\022\036\n\020should" +
+ "_split_wal\030\006 \001(\010:\004true*\330\001\n\020CreateTableSt" +
+ "ate\022\036\n\032CREATE_TABLE_PRE_OPERATION\020\001\022 \n\034C" +
+ "REATE_TABLE_WRITE_FS_LAYOUT\020\002\022\034\n\030CREATE_" +
+ "TABLE_ADD_TO_META\020\003\022\037\n\033CREATE_TABLE_ASSI" +
+ "GN_REGIONS\020\004\022\"\n\036CREATE_TABLE_UPDATE_DESC",
+ "_CACHE\020\005\022\037\n\033CREATE_TABLE_POST_OPERATION\020" +
+ "\006*\207\002\n\020ModifyTableState\022\030\n\024MODIFY_TABLE_P" +
+ "REPARE\020\001\022\036\n\032MODIFY_TABLE_PRE_OPERATION\020\002" +
+ "\022(\n$MODIFY_TABLE_UPDATE_TABLE_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_OPERATION\020\006\022#\n\037MODIFY" +
+ "_TABLE_REOPEN_ALL_REGIONS\020\007*\212\002\n\022Truncate" +
+ "TableState\022 \n\034TRUNCATE_TABLE_PRE_OPERATI" +
+ "ON\020\001\022#\n\037TRUNCATE_TABLE_REMOVE_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\032TR" +
+ "UNCATE_TABLE_ADD_TO_META\020\005\022!\n\035TRUNCATE_T" +
+ "ABLE_ASSIGN_REGIONS\020\006\022!\n\035TRUNCATE_TABLE_" +
+ "POST_OPERATION\020\007*\337\001\n\020DeleteTableState\022\036\n" +
+ "\032DELETE_TABLE_PRE_OPERATION\020\001\022!\n\035DELETE_" +
+ "TABLE_REMOVE_FROM_META\020\002\022 \n\034DELETE_TABLE" +
+ "_CLEAR_FS_LAYOUT\020\003\022\"\n\036DELETE_TABLE_UPDAT" +
+ "E_DESC_CACHE\020\004\022!\n\035DELETE_TABLE_UNASSIGN_" +
+ "REGIONS\020\005\022\037\n\033DELETE_TABLE_POST_OPERATION",
+ "\020\006*\320\001\n\024CreateNamespaceState\022\034\n\030CREATE_NA" +
+ "MESPACE_PREPARE\020\001\022%\n!CREATE_NAMESPACE_CR" +
+ "EATE_DIRECTORY\020\002\022)\n%CREATE_NAMESPACE_INS" +
+ "ERT_INTO_NS_TABLE\020\003\022\036\n\032CREATE_NAMESPACE_" +
+ "UPDATE_ZK\020\004\022(\n$CREATE_NAMESPACE_SET_NAME" +
+ "SPACE_QUOTA\020\005*z\n\024ModifyNamespaceState\022\034\n" +
+ "\030MODIFY_NAMESPACE_PREPARE\020\001\022$\n MODIFY_NA" +
+ "MESPACE_UPDATE_NS_TABLE\020\002\022\036\n\032MODIFY_NAME" +
+ "SPACE_UPDATE_ZK\020\003*\332\001\n\024DeleteNamespaceSta" +
+ "te\022\034\n\030DELETE_NAMESPACE_PREPARE\020\001\022)\n%DELE",
+ "TE_NAMESPACE_DELETE_FROM_NS_TABLE\020\002\022#\n\037D" +
+ "ELETE_NAMESPACE_REMOVE_FROM_ZK\020\003\022\'\n#DELE" +
+ "TE_NAMESPACE_DELETE_DIRECTORIES\020\004\022+\n\'DEL" +
+ "ETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA\020\005*\331" +
+ "\001\n\024AddColumnFamilyState\022\035\n\031ADD_COLUMN_FA" +
+ "MILY_PREPARE\020\001\022#\n\037ADD_COLUMN_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_COLUMN_FAMILY_RE" +
+ "OPEN_ALL_REGIONS\020\005*\353\001\n\027ModifyColumnFamil",
+ "yState\022 \n\034MODIFY_COLUMN_FAMILY_PREPARE\020\001" +
+ "\022&\n\"MODIFY_COLUMN_FAMILY_PRE_OPERATION\020\002" +
+ "\0220\n,MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DE" +
+ "SCRIPTOR\020\003\022\'\n#MODIFY_COLUMN_FAMILY_POST_" +
+ "OPERATION\020\004\022+\n\'MODIFY_COLUMN_FAMILY_REOP" +
+ "EN_ALL_REGIONS\020\005*\226\002\n\027DeleteColumnFamilyS" +
+ "tate\022 \n\034DELETE_COLUMN_FAMILY_PREPARE\020\001\022&" +
+ "\n\"DELETE_COLUMN_FAMILY_PRE_OPERATION\020\002\0220" +
+ "\n,DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESC" +
+ "RIPTOR\020\003\022)\n%DELETE_COLUMN_FAMILY_DELETE_",
+ "FS_LAYOUT\020\004\022\'\n#DELETE_COLUMN_FAMILY_POST" +
+ "_OPERATION\020\005\022+\n\'DELETE_COLUMN_FAMILY_REO" +
+ "PEN_ALL_REGIONS\020\006*\350\001\n\020EnableTableState\022\030" +
+ "\n\024ENABLE_TABLE_PREPARE\020\001\022\036\n\032ENABLE_TABLE" +
+ "_PRE_OPERATION\020\002\022)\n%ENABLE_TABLE_SET_ENA" +
+ "BLING_TABLE_STATE\020\003\022$\n ENABLE_TABLE_MARK" +
+ "_REGIONS_ONLINE\020\004\022(\n$ENABLE_TABLE_SET_EN" +
+ "ABLED_TABLE_STATE\020\005\022\037\n\033ENABLE_TABLE_POST" +
+ "_OPERATION\020\006*\362\001\n\021DisableTableState\022\031\n\025DI" +
+ "SABLE_TABLE_PREPARE\020\001\022\037\n\033DISABLE_TABLE_P",
+ "RE_OPERATION\020\002\022+\n\'DISABLE_TABLE_SET_DISA" +
+ "BLING_TABLE_STATE\020\003\022&\n\"DISABLE_TABLE_MAR" +
+ "K_REGIONS_OFFLINE\020\004\022*\n&DISABLE_TABLE_SET" +
+ "_DISABLED_TABLE_STATE\020\005\022 \n\034DISABLE_TABLE" +
+ "_POST_OPERATION\020\006*\346\001\n\022CloneSnapshotState" +
+ "\022 \n\034CLONE_SNAPSHOT_PRE_OPERATION\020\001\022\"\n\036CL" +
+ "ONE_SNAPSHOT_WRITE_FS_LAYOUT\020\002\022\036\n\032CLONE_" +
+ "SNAPSHOT_ADD_TO_META\020\003\022!\n\035CLONE_SNAPSHOT" +
+ "_ASSIGN_REGIONS\020\004\022$\n CLONE_SNAPSHOT_UPDA" +
+ "TE_DESC_CACHE\020\005\022!\n\035CLONE_SNAPSHOT_POST_O",
+ "PERATION\020\006*\260\001\n\024RestoreSnapshotState\022\"\n\036R" +
+ "ESTORE_SNAPSHOT_PRE_OPERATION\020\001\022,\n(RESTO" +
+ "RE_SNAPSHOT_UPDATE_TABLE_DESCRIPTOR\020\002\022$\n" +
+ " RESTORE_SNAPSHOT_WRITE_FS_LAYOUT\020\003\022 \n\034R" +
+ "ESTORE_SNAPSHOT_UPDATE_META\020\004*\376\001\n\033Dispat" +
+ "chMergingRegionsState\022$\n DISPATCH_MERGIN" +
+ "G_REGIONS_PREPARE\020\001\022*\n&DISPATCH_MERGING_" +
+ "REGIONS_PRE_OPERATION\020\002\0223\n/DISPATCH_MERG" +
+ "ING_REGIONS_MOVE_REGION_TO_SAME_RS\020\003\022+\n\'" +
+ "DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS\020",
+ "\004\022+\n\'DISPATCH_MERGING_REGIONS_POST_OPERA" +
+ "TION\020\005*\234\002\n\020ServerCrashState\022\026\n\022SERVER_CR" +
+ "ASH_START\020\001\022\035\n\031SERVER_CRASH_PROCESS_META" +
+ "\020\002\022\034\n\030SERVER_CRASH_GET_REGIONS\020\003\022\036\n\032SERV" +
+ "ER_CRASH_NO_SPLIT_LOGS\020\004\022\033\n\027SERVER_CRASH" +
+ "_SPLIT_LOGS\020\005\022#\n\037SERVER_CRASH_PREPARE_LO" +
+ "G_REPLAY\020\006\022\027\n\023SERVER_CRASH_ASSIGN\020\010\022\037\n\033S" +
+ "ERVER_CRASH_WAIT_ON_ASSIGN\020\t\022\027\n\023SERVER_C" +
+ "RASH_FINISH\020dBK\n*org.apache.hadoop.hbase" +
+ ".protobuf.generatedB\025MasterProcedureProt",
+ "osH\001\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -21024,8 +22397,14 @@ public final class MasterProcedureProtos {
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal
<TRUNCATED>
[2/4] hbase git commit: HBASE-14552 Procedure V2: Reimplement
DispatchMergingRegionHandler (Stephen Yuan Jiang)
Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
index b4bd7af..4a92e14 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
@@ -5302,6 +5302,26 @@ public final class MasterProtos {
* <code>optional bool forcible = 3 [default = false];</code>
*/
boolean getForcible();
+
+ // optional uint64 nonce_group = 4 [default = 0];
+ /**
+ * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+ */
+ boolean hasNonceGroup();
+ /**
+ * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+ */
+ long getNonceGroup();
+
+ // optional uint64 nonce = 5 [default = 0];
+ /**
+ * <code>optional uint64 nonce = 5 [default = 0];</code>
+ */
+ boolean hasNonce();
+ /**
+ * <code>optional uint64 nonce = 5 [default = 0];</code>
+ */
+ long getNonce();
}
/**
* Protobuf type {@code hbase.pb.DispatchMergingRegionsRequest}
@@ -5390,6 +5410,16 @@ public final class MasterProtos {
forcible_ = input.readBool();
break;
}
+ case 32: {
+ bitField0_ |= 0x00000008;
+ nonceGroup_ = input.readUInt64();
+ break;
+ }
+ case 40: {
+ bitField0_ |= 0x00000010;
+ nonce_ = input.readUInt64();
+ break;
+ }
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -5490,10 +5520,44 @@ public final class MasterProtos {
return forcible_;
}
+ // optional uint64 nonce_group = 4 [default = 0];
+ 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_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+ */
+ public long getNonceGroup() {
+ return nonceGroup_;
+ }
+
+ // optional uint64 nonce = 5 [default = 0];
+ public static final int NONCE_FIELD_NUMBER = 5;
+ private long nonce_;
+ /**
+ * <code>optional uint64 nonce = 5 [default = 0];</code>
+ */
+ public boolean hasNonce() {
+ return ((bitField0_ & 0x00000010) == 0x00000010);
+ }
+ /**
+ * <code>optional uint64 nonce = 5 [default = 0];</code>
+ */
+ public long getNonce() {
+ return nonce_;
+ }
+
private void initFields() {
regionA_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance();
regionB_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance();
forcible_ = false;
+ nonceGroup_ = 0L;
+ nonce_ = 0L;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -5532,6 +5596,12 @@ public final class MasterProtos {
if (((bitField0_ & 0x00000004) == 0x00000004)) {
output.writeBool(3, forcible_);
}
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ output.writeUInt64(4, nonceGroup_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ output.writeUInt64(5, nonce_);
+ }
getUnknownFields().writeTo(output);
}
@@ -5553,6 +5623,14 @@ public final class MasterProtos {
size += com.google.protobuf.CodedOutputStream
.computeBoolSize(3, forcible_);
}
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(4, nonceGroup_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(5, nonce_);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@@ -5591,6 +5669,16 @@ public final class MasterProtos {
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 &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@@ -5616,6 +5704,14 @@ public final class MasterProtos {
hash = (37 * hash) + FORCIBLE_FIELD_NUMBER;
hash = (53 * hash) + hashBoolean(getForcible());
}
+ if (hasNonceGroup()) {
+ hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
+ hash = (53 * hash) + hashLong(getNonceGroup());
+ }
+ if (hasNonce()) {
+ hash = (37 * hash) + NONCE_FIELD_NUMBER;
+ hash = (53 * hash) + hashLong(getNonce());
+ }
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@@ -5746,6 +5842,10 @@ public final class MasterProtos {
bitField0_ = (bitField0_ & ~0x00000002);
forcible_ = false;
bitField0_ = (bitField0_ & ~0x00000004);
+ nonceGroup_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000008);
+ nonce_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000010);
return this;
}
@@ -5794,6 +5894,14 @@ public final class MasterProtos {
to_bitField0_ |= 0x00000004;
}
result.forcible_ = forcible_;
+ if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+ to_bitField0_ |= 0x00000008;
+ }
+ result.nonceGroup_ = nonceGroup_;
+ if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+ to_bitField0_ |= 0x00000010;
+ }
+ result.nonce_ = nonce_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@@ -5819,6 +5927,12 @@ public final class MasterProtos {
if (other.hasForcible()) {
setForcible(other.getForcible());
}
+ if (other.hasNonceGroup()) {
+ setNonceGroup(other.getNonceGroup());
+ }
+ if (other.hasNonce()) {
+ setNonce(other.getNonce());
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -6129,6 +6243,72 @@ public final class MasterProtos {
return this;
}
+ // optional uint64 nonce_group = 4 [default = 0];
+ private long nonceGroup_ ;
+ /**
+ * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+ */
+ public boolean hasNonceGroup() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <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_ |= 0x00000008;
+ nonceGroup_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+ */
+ public Builder clearNonceGroup() {
+ bitField0_ = (bitField0_ & ~0x00000008);
+ nonceGroup_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // optional uint64 nonce = 5 [default = 0];
+ private long nonce_ ;
+ /**
+ * <code>optional uint64 nonce = 5 [default = 0];</code>
+ */
+ public boolean hasNonce() {
+ return ((bitField0_ & 0x00000010) == 0x00000010);
+ }
+ /**
+ * <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_ |= 0x00000010;
+ nonce_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional uint64 nonce = 5 [default = 0];</code>
+ */
+ public Builder clearNonce() {
+ bitField0_ = (bitField0_ & ~0x00000010);
+ nonce_ = 0L;
+ onChanged();
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:hbase.pb.DispatchMergingRegionsRequest)
}
@@ -6142,6 +6322,16 @@ public final class MasterProtos {
public interface DispatchMergingRegionsResponseOrBuilder
extends com.google.protobuf.MessageOrBuilder {
+
+ // optional uint64 proc_id = 1;
+ /**
+ * <code>optional uint64 proc_id = 1;</code>
+ */
+ boolean hasProcId();
+ /**
+ * <code>optional uint64 proc_id = 1;</code>
+ */
+ long getProcId();
}
/**
* Protobuf type {@code hbase.pb.DispatchMergingRegionsResponse}
@@ -6176,6 +6366,7 @@ public final class MasterProtos {
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
initFields();
+ int mutable_bitField0_ = 0;
com.google.protobuf.UnknownFieldSet.Builder unknownFields =
com.google.protobuf.UnknownFieldSet.newBuilder();
try {
@@ -6193,6 +6384,11 @@ public final class MasterProtos {
}
break;
}
+ case 8: {
+ bitField0_ |= 0x00000001;
+ procId_ = input.readUInt64();
+ break;
+ }
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -6232,7 +6428,25 @@ public final class MasterProtos {
return PARSER;
}
+ private int bitField0_;
+ // optional uint64 proc_id = 1;
+ 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 void initFields() {
+ procId_ = 0L;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -6246,6 +6460,9 @@ public final class MasterProtos {
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeUInt64(1, procId_);
+ }
getUnknownFields().writeTo(output);
}
@@ -6255,6 +6472,10 @@ public final class MasterProtos {
if (size != -1) return size;
size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(1, procId_);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@@ -6278,6 +6499,11 @@ public final class MasterProtos {
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse) obj;
boolean result = true;
+ result = result && (hasProcId() == other.hasProcId());
+ if (hasProcId()) {
+ result = result && (getProcId()
+ == other.getProcId());
+ }
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@@ -6291,6 +6517,10 @@ public final class MasterProtos {
}
int hash = 41;
hash = (19 * hash) + getDescriptorForType().hashCode();
+ if (hasProcId()) {
+ hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
+ hash = (53 * hash) + hashLong(getProcId());
+ }
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@@ -6400,6 +6630,8 @@ public final class MasterProtos {
public Builder clear() {
super.clear();
+ procId_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000001);
return this;
}
@@ -6426,6 +6658,13 @@ public final class MasterProtos {
public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse buildPartial() {
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse(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;
}
@@ -6441,6 +6680,9 @@ public final class MasterProtos {
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse other) {
if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.getDefaultInstance()) return this;
+ if (other.hasProcId()) {
+ setProcId(other.getProcId());
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -6466,6 +6708,40 @@ public final class MasterProtos {
}
return this;
}
+ private int bitField0_;
+
+ // optional uint64 proc_id = 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_;
+ }
+ /**
+ * <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;
+ }
// @@protoc_insertion_point(builder_scope:hbase.pb.DispatchMergingRegionsResponse)
}
@@ -65571,316 +65847,317 @@ public final class MasterProtos {
"oc_id\030\001 \001(\004\"n\n\021MoveRegionRequest\022)\n\006regi" +
"on\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022.\n\020d" +
"est_server_name\030\002 \001(\0132\024.hbase.pb.ServerN",
- "ame\"\024\n\022MoveRegionResponse\"\222\001\n\035DispatchMe" +
+ "ame\"\024\n\022MoveRegionResponse\"\274\001\n\035DispatchMe" +
"rgingRegionsRequest\022+\n\010region_a\030\001 \002(\0132\031." +
"hbase.pb.RegionSpecifier\022+\n\010region_b\030\002 \002" +
"(\0132\031.hbase.pb.RegionSpecifier\022\027\n\010forcibl" +
- "e\030\003 \001(\010:\005false\" \n\036DispatchMergingRegions" +
- "Response\"@\n\023AssignRegionRequest\022)\n\006regio" +
- "n\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\"\026\n\024As" +
- "signRegionResponse\"X\n\025UnassignRegionRequ" +
- "est\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpe" +
- "cifier\022\024\n\005force\030\002 \001(\010:\005false\"\030\n\026Unassign",
- "RegionResponse\"A\n\024OfflineRegionRequest\022)" +
- "\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifie" +
- "r\"\027\n\025OfflineRegionResponse\"\177\n\022CreateTabl" +
- "eRequest\022+\n\014table_schema\030\001 \002(\0132\025.hbase.p" +
- "b.TableSchema\022\022\n\nsplit_keys\030\002 \003(\014\022\026\n\013non" +
- "ce_group\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"&\n\023C" +
- "reateTableResponse\022\017\n\007proc_id\030\001 \001(\004\"g\n\022D" +
- "eleteTableRequest\022\'\n\ntable_name\030\001 \002(\0132\023." +
- "hbase.pb.TableName\022\026\n\013nonce_group\030\002 \001(\004:" +
- "\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"&\n\023DeleteTableRespo",
- "nse\022\017\n\007proc_id\030\001 \001(\004\"\207\001\n\024TruncateTableRe" +
- "quest\022&\n\ttableName\030\001 \002(\0132\023.hbase.pb.Tabl" +
- "eName\022\035\n\016preserveSplits\030\002 \001(\010:\005false\022\026\n\013" +
- "nonce_group\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"(" +
- "\n\025TruncateTableResponse\022\017\n\007proc_id\030\001 \001(\004" +
- "\"g\n\022EnableTableRequest\022\'\n\ntable_name\030\001 \002" +
- "(\0132\023.hbase.pb.TableName\022\026\n\013nonce_group\030\002" +
- " \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"&\n\023EnableTable" +
- "Response\022\017\n\007proc_id\030\001 \001(\004\"h\n\023DisableTabl" +
- "eRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.",
- "TableName\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005non" +
- "ce\030\003 \001(\004:\0010\"\'\n\024DisableTableResponse\022\017\n\007p" +
- "roc_id\030\001 \001(\004\"\224\001\n\022ModifyTableRequest\022\'\n\nt" +
- "able_name\030\001 \002(\0132\023.hbase.pb.TableName\022+\n\014" +
- "table_schema\030\002 \002(\0132\025.hbase.pb.TableSchem" +
- "a\022\026\n\013nonce_group\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004" +
- ":\0010\"&\n\023ModifyTableResponse\022\017\n\007proc_id\030\001 " +
- "\001(\004\"~\n\026CreateNamespaceRequest\022:\n\023namespa" +
- "ceDescriptor\030\001 \002(\0132\035.hbase.pb.NamespaceD" +
- "escriptor\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005non",
- "ce\030\003 \001(\004:\0010\"*\n\027CreateNamespaceResponse\022\017" +
- "\n\007proc_id\030\001 \001(\004\"Y\n\026DeleteNamespaceReques" +
- "t\022\025\n\rnamespaceName\030\001 \002(\t\022\026\n\013nonce_group\030" +
- "\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"*\n\027DeleteName" +
- "spaceResponse\022\017\n\007proc_id\030\001 \001(\004\"~\n\026Modify" +
- "NamespaceRequest\022:\n\023namespaceDescriptor\030" +
- "\001 \002(\0132\035.hbase.pb.NamespaceDescriptor\022\026\n\013" +
- "nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"*" +
- "\n\027ModifyNamespaceResponse\022\017\n\007proc_id\030\001 \001" +
- "(\004\"6\n\035GetNamespaceDescriptorRequest\022\025\n\rn",
- "amespaceName\030\001 \002(\t\"\\\n\036GetNamespaceDescri" +
- "ptorResponse\022:\n\023namespaceDescriptor\030\001 \002(" +
- "\0132\035.hbase.pb.NamespaceDescriptor\"!\n\037List" +
- "NamespaceDescriptorsRequest\"^\n ListNames" +
- "paceDescriptorsResponse\022:\n\023namespaceDesc" +
- "riptor\030\001 \003(\0132\035.hbase.pb.NamespaceDescrip" +
- "tor\"?\n&ListTableDescriptorsByNamespaceRe" +
- "quest\022\025\n\rnamespaceName\030\001 \002(\t\"U\n\'ListTabl" +
- "eDescriptorsByNamespaceResponse\022*\n\013table" +
- "Schema\030\001 \003(\0132\025.hbase.pb.TableSchema\"9\n L",
- "istTableNamesByNamespaceRequest\022\025\n\rnames" +
- "paceName\030\001 \002(\t\"K\n!ListTableNamesByNamesp" +
- "aceResponse\022&\n\ttableName\030\001 \003(\0132\023.hbase.p" +
- "b.TableName\"\021\n\017ShutdownRequest\"\022\n\020Shutdo" +
- "wnResponse\"\023\n\021StopMasterRequest\"\024\n\022StopM" +
- "asterResponse\"\037\n\016BalanceRequest\022\r\n\005force" +
- "\030\001 \001(\010\"\'\n\017BalanceResponse\022\024\n\014balancer_ra" +
- "n\030\001 \002(\010\"<\n\031SetBalancerRunningRequest\022\n\n\002" +
- "on\030\001 \002(\010\022\023\n\013synchronous\030\002 \001(\010\"8\n\032SetBala" +
- "ncerRunningResponse\022\032\n\022prev_balance_valu",
- "e\030\001 \001(\010\"\032\n\030IsBalancerEnabledRequest\",\n\031I" +
- "sBalancerEnabledResponse\022\017\n\007enabled\030\001 \002(" +
- "\010\"\212\001\n\035SetSplitOrMergeEnabledRequest\022\017\n\007e" +
- "nabled\030\001 \002(\010\022\023\n\013synchronous\030\002 \001(\010\0220\n\014swi" +
- "tch_types\030\003 \003(\0162\032.hbase.pb.MasterSwitchT" +
- "ype\022\021\n\tskip_lock\030\004 \001(\010\"4\n\036SetSplitOrMerg" +
- "eEnabledResponse\022\022\n\nprev_value\030\001 \003(\010\"O\n\034" +
- "IsSplitOrMergeEnabledRequest\022/\n\013switch_t" +
- "ype\030\001 \002(\0162\032.hbase.pb.MasterSwitchType\"0\n" +
- "\035IsSplitOrMergeEnabledResponse\022\017\n\007enable",
- "d\030\001 \002(\010\"+\n)ReleaseSplitOrMergeLockAndRol" +
- "lbackRequest\",\n*ReleaseSplitOrMergeLockA" +
- "ndRollbackResponse\"\022\n\020NormalizeRequest\"+" +
- "\n\021NormalizeResponse\022\026\n\016normalizer_ran\030\001 " +
- "\002(\010\")\n\033SetNormalizerRunningRequest\022\n\n\002on" +
- "\030\001 \002(\010\"=\n\034SetNormalizerRunningResponse\022\035" +
- "\n\025prev_normalizer_value\030\001 \001(\010\"\034\n\032IsNorma" +
- "lizerEnabledRequest\".\n\033IsNormalizerEnabl" +
- "edResponse\022\017\n\007enabled\030\001 \002(\010\"\027\n\025RunCatalo" +
- "gScanRequest\"-\n\026RunCatalogScanResponse\022\023",
- "\n\013scan_result\030\001 \001(\005\"-\n\033EnableCatalogJani" +
- "torRequest\022\016\n\006enable\030\001 \002(\010\"2\n\034EnableCata" +
- "logJanitorResponse\022\022\n\nprev_value\030\001 \001(\010\" " +
- "\n\036IsCatalogJanitorEnabledRequest\"0\n\037IsCa" +
- "talogJanitorEnabledResponse\022\r\n\005value\030\001 \002" +
- "(\010\"B\n\017SnapshotRequest\022/\n\010snapshot\030\001 \002(\0132" +
- "\035.hbase.pb.SnapshotDescription\",\n\020Snapsh" +
- "otResponse\022\030\n\020expected_timeout\030\001 \002(\003\"\036\n\034" +
- "GetCompletedSnapshotsRequest\"Q\n\035GetCompl" +
- "etedSnapshotsResponse\0220\n\tsnapshots\030\001 \003(\013",
- "2\035.hbase.pb.SnapshotDescription\"H\n\025Delet" +
+ "e\030\003 \001(\010:\005false\022\026\n\013nonce_group\030\004 \001(\004:\0010\022\020" +
+ "\n\005nonce\030\005 \001(\004:\0010\"1\n\036DispatchMergingRegio" +
+ "nsResponse\022\017\n\007proc_id\030\001 \001(\004\"@\n\023AssignReg" +
+ "ionRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Re" +
+ "gionSpecifier\"\026\n\024AssignRegionResponse\"X\n" +
+ "\025UnassignRegionRequest\022)\n\006region\030\001 \002(\0132\031",
+ ".hbase.pb.RegionSpecifier\022\024\n\005force\030\002 \001(\010" +
+ ":\005false\"\030\n\026UnassignRegionResponse\"A\n\024Off" +
+ "lineRegionRequest\022)\n\006region\030\001 \002(\0132\031.hbas" +
+ "e.pb.RegionSpecifier\"\027\n\025OfflineRegionRes" +
+ "ponse\"\177\n\022CreateTableRequest\022+\n\014table_sch" +
+ "ema\030\001 \002(\0132\025.hbase.pb.TableSchema\022\022\n\nspli" +
+ "t_keys\030\002 \003(\014\022\026\n\013nonce_group\030\003 \001(\004:\0010\022\020\n\005" +
+ "nonce\030\004 \001(\004:\0010\"&\n\023CreateTableResponse\022\017\n" +
+ "\007proc_id\030\001 \001(\004\"g\n\022DeleteTableRequest\022\'\n\n" +
+ "table_name\030\001 \002(\0132\023.hbase.pb.TableName\022\026\n",
+ "\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"" +
+ "&\n\023DeleteTableResponse\022\017\n\007proc_id\030\001 \001(\004\"" +
+ "\207\001\n\024TruncateTableRequest\022&\n\ttableName\030\001 " +
+ "\002(\0132\023.hbase.pb.TableName\022\035\n\016preserveSpli" +
+ "ts\030\002 \001(\010:\005false\022\026\n\013nonce_group\030\003 \001(\004:\0010\022" +
+ "\020\n\005nonce\030\004 \001(\004:\0010\"(\n\025TruncateTableRespon" +
+ "se\022\017\n\007proc_id\030\001 \001(\004\"g\n\022EnableTableReques" +
+ "t\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.TableNa" +
+ "me\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(" +
+ "\004:\0010\"&\n\023EnableTableResponse\022\017\n\007proc_id\030\001",
+ " \001(\004\"h\n\023DisableTableRequest\022\'\n\ntable_nam" +
+ "e\030\001 \002(\0132\023.hbase.pb.TableName\022\026\n\013nonce_gr" +
+ "oup\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"\'\n\024Disabl" +
+ "eTableResponse\022\017\n\007proc_id\030\001 \001(\004\"\224\001\n\022Modi" +
+ "fyTableRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hba" +
+ "se.pb.TableName\022+\n\014table_schema\030\002 \002(\0132\025." +
+ "hbase.pb.TableSchema\022\026\n\013nonce_group\030\003 \001(" +
+ "\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"&\n\023ModifyTableRes" +
+ "ponse\022\017\n\007proc_id\030\001 \001(\004\"~\n\026CreateNamespac" +
+ "eRequest\022:\n\023namespaceDescriptor\030\001 \002(\0132\035.",
+ "hbase.pb.NamespaceDescriptor\022\026\n\013nonce_gr" +
+ "oup\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"*\n\027Create" +
+ "NamespaceResponse\022\017\n\007proc_id\030\001 \001(\004\"Y\n\026De" +
+ "leteNamespaceRequest\022\025\n\rnamespaceName\030\001 " +
+ "\002(\t\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001" +
+ "(\004:\0010\"*\n\027DeleteNamespaceResponse\022\017\n\007proc" +
+ "_id\030\001 \001(\004\"~\n\026ModifyNamespaceRequest\022:\n\023n" +
+ "amespaceDescriptor\030\001 \002(\0132\035.hbase.pb.Name" +
+ "spaceDescriptor\022\026\n\013nonce_group\030\002 \001(\004:\0010\022" +
+ "\020\n\005nonce\030\003 \001(\004:\0010\"*\n\027ModifyNamespaceResp",
+ "onse\022\017\n\007proc_id\030\001 \001(\004\"6\n\035GetNamespaceDes" +
+ "criptorRequest\022\025\n\rnamespaceName\030\001 \002(\t\"\\\n" +
+ "\036GetNamespaceDescriptorResponse\022:\n\023names" +
+ "paceDescriptor\030\001 \002(\0132\035.hbase.pb.Namespac" +
+ "eDescriptor\"!\n\037ListNamespaceDescriptorsR" +
+ "equest\"^\n ListNamespaceDescriptorsRespon" +
+ "se\022:\n\023namespaceDescriptor\030\001 \003(\0132\035.hbase." +
+ "pb.NamespaceDescriptor\"?\n&ListTableDescr" +
+ "iptorsByNamespaceRequest\022\025\n\rnamespaceNam" +
+ "e\030\001 \002(\t\"U\n\'ListTableDescriptorsByNamespa",
+ "ceResponse\022*\n\013tableSchema\030\001 \003(\0132\025.hbase." +
+ "pb.TableSchema\"9\n ListTableNamesByNamesp" +
+ "aceRequest\022\025\n\rnamespaceName\030\001 \002(\t\"K\n!Lis" +
+ "tTableNamesByNamespaceResponse\022&\n\ttableN" +
+ "ame\030\001 \003(\0132\023.hbase.pb.TableName\"\021\n\017Shutdo" +
+ "wnRequest\"\022\n\020ShutdownResponse\"\023\n\021StopMas" +
+ "terRequest\"\024\n\022StopMasterResponse\"\037\n\016Bala" +
+ "nceRequest\022\r\n\005force\030\001 \001(\010\"\'\n\017BalanceResp" +
+ "onse\022\024\n\014balancer_ran\030\001 \002(\010\"<\n\031SetBalance" +
+ "rRunningRequest\022\n\n\002on\030\001 \002(\010\022\023\n\013synchrono",
+ "us\030\002 \001(\010\"8\n\032SetBalancerRunningResponse\022\032" +
+ "\n\022prev_balance_value\030\001 \001(\010\"\032\n\030IsBalancer" +
+ "EnabledRequest\",\n\031IsBalancerEnabledRespo" +
+ "nse\022\017\n\007enabled\030\001 \002(\010\"\212\001\n\035SetSplitOrMerge" +
+ "EnabledRequest\022\017\n\007enabled\030\001 \002(\010\022\023\n\013synch" +
+ "ronous\030\002 \001(\010\0220\n\014switch_types\030\003 \003(\0162\032.hba" +
+ "se.pb.MasterSwitchType\022\021\n\tskip_lock\030\004 \001(" +
+ "\010\"4\n\036SetSplitOrMergeEnabledResponse\022\022\n\np" +
+ "rev_value\030\001 \003(\010\"O\n\034IsSplitOrMergeEnabled" +
+ "Request\022/\n\013switch_type\030\001 \002(\0162\032.hbase.pb.",
+ "MasterSwitchType\"0\n\035IsSplitOrMergeEnable" +
+ "dResponse\022\017\n\007enabled\030\001 \002(\010\"+\n)ReleaseSpl" +
+ "itOrMergeLockAndRollbackRequest\",\n*Relea" +
+ "seSplitOrMergeLockAndRollbackResponse\"\022\n" +
+ "\020NormalizeRequest\"+\n\021NormalizeResponse\022\026" +
+ "\n\016normalizer_ran\030\001 \002(\010\")\n\033SetNormalizerR" +
+ "unningRequest\022\n\n\002on\030\001 \002(\010\"=\n\034SetNormaliz" +
+ "erRunningResponse\022\035\n\025prev_normalizer_val" +
+ "ue\030\001 \001(\010\"\034\n\032IsNormalizerEnabledRequest\"." +
+ "\n\033IsNormalizerEnabledResponse\022\017\n\007enabled",
+ "\030\001 \002(\010\"\027\n\025RunCatalogScanRequest\"-\n\026RunCa" +
+ "talogScanResponse\022\023\n\013scan_result\030\001 \001(\005\"-" +
+ "\n\033EnableCatalogJanitorRequest\022\016\n\006enable\030" +
+ "\001 \002(\010\"2\n\034EnableCatalogJanitorResponse\022\022\n" +
+ "\nprev_value\030\001 \001(\010\" \n\036IsCatalogJanitorEna" +
+ "bledRequest\"0\n\037IsCatalogJanitorEnabledRe" +
+ "sponse\022\r\n\005value\030\001 \002(\010\"B\n\017SnapshotRequest" +
+ "\022/\n\010snapshot\030\001 \002(\0132\035.hbase.pb.SnapshotDe" +
+ "scription\",\n\020SnapshotResponse\022\030\n\020expecte" +
+ "d_timeout\030\001 \002(\003\"\036\n\034GetCompletedSnapshots",
+ "Request\"Q\n\035GetCompletedSnapshotsResponse" +
+ "\0220\n\tsnapshots\030\001 \003(\0132\035.hbase.pb.SnapshotD" +
+ "escription\"H\n\025DeleteSnapshotRequest\022/\n\010s" +
+ "napshot\030\001 \002(\0132\035.hbase.pb.SnapshotDescrip" +
+ "tion\"\030\n\026DeleteSnapshotResponse\"s\n\026Restor" +
"eSnapshotRequest\022/\n\010snapshot\030\001 \002(\0132\035.hba" +
- "se.pb.SnapshotDescription\"\030\n\026DeleteSnaps" +
- "hotResponse\"s\n\026RestoreSnapshotRequest\022/\n" +
- "\010snapshot\030\001 \002(\0132\035.hbase.pb.SnapshotDescr" +
- "iption\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030" +
- "\003 \001(\004:\0010\"*\n\027RestoreSnapshotResponse\022\017\n\007p" +
- "roc_id\030\001 \002(\004\"H\n\025IsSnapshotDoneRequest\022/\n" +
- "\010snapshot\030\001 \001(\0132\035.hbase.pb.SnapshotDescr" +
- "iption\"^\n\026IsSnapshotDoneResponse\022\023\n\004done",
- "\030\001 \001(\010:\005false\022/\n\010snapshot\030\002 \001(\0132\035.hbase." +
- "pb.SnapshotDescription\"O\n\034IsRestoreSnaps" +
- "hotDoneRequest\022/\n\010snapshot\030\001 \001(\0132\035.hbase" +
- ".pb.SnapshotDescription\"4\n\035IsRestoreSnap" +
- "shotDoneResponse\022\023\n\004done\030\001 \001(\010:\005false\"F\n" +
- "\033GetSchemaAlterStatusRequest\022\'\n\ntable_na" +
- "me\030\001 \002(\0132\023.hbase.pb.TableName\"T\n\034GetSche" +
- "maAlterStatusResponse\022\035\n\025yet_to_update_r" +
- "egions\030\001 \001(\r\022\025\n\rtotal_regions\030\002 \001(\r\"\213\001\n\032" +
- "GetTableDescriptorsRequest\022(\n\013table_name",
- "s\030\001 \003(\0132\023.hbase.pb.TableName\022\r\n\005regex\030\002 " +
- "\001(\t\022!\n\022include_sys_tables\030\003 \001(\010:\005false\022\021" +
- "\n\tnamespace\030\004 \001(\t\"J\n\033GetTableDescriptors" +
- "Response\022+\n\014table_schema\030\001 \003(\0132\025.hbase.p" +
- "b.TableSchema\"[\n\024GetTableNamesRequest\022\r\n" +
- "\005regex\030\001 \001(\t\022!\n\022include_sys_tables\030\002 \001(\010" +
- ":\005false\022\021\n\tnamespace\030\003 \001(\t\"A\n\025GetTableNa" +
- "mesResponse\022(\n\013table_names\030\001 \003(\0132\023.hbase" +
- ".pb.TableName\"?\n\024GetTableStateRequest\022\'\n" +
- "\ntable_name\030\001 \002(\0132\023.hbase.pb.TableName\"B",
- "\n\025GetTableStateResponse\022)\n\013table_state\030\001" +
- " \002(\0132\024.hbase.pb.TableState\"\031\n\027GetCluster" +
- "StatusRequest\"K\n\030GetClusterStatusRespons" +
- "e\022/\n\016cluster_status\030\001 \002(\0132\027.hbase.pb.Clu" +
- "sterStatus\"\030\n\026IsMasterRunningRequest\"4\n\027" +
- "IsMasterRunningResponse\022\031\n\021is_master_run" +
- "ning\030\001 \002(\010\"I\n\024ExecProcedureRequest\0221\n\tpr" +
- "ocedure\030\001 \002(\0132\036.hbase.pb.ProcedureDescri" +
- "ption\"F\n\025ExecProcedureResponse\022\030\n\020expect" +
- "ed_timeout\030\001 \001(\003\022\023\n\013return_data\030\002 \001(\014\"K\n",
- "\026IsProcedureDoneRequest\0221\n\tprocedure\030\001 \001" +
- "(\0132\036.hbase.pb.ProcedureDescription\"`\n\027Is" +
- "ProcedureDoneResponse\022\023\n\004done\030\001 \001(\010:\005fal" +
- "se\0220\n\010snapshot\030\002 \001(\0132\036.hbase.pb.Procedur" +
- "eDescription\",\n\031GetProcedureResultReques" +
- "t\022\017\n\007proc_id\030\001 \002(\004\"\371\001\n\032GetProcedureResul" +
- "tResponse\0229\n\005state\030\001 \002(\0162*.hbase.pb.GetP" +
- "rocedureResultResponse.State\022\022\n\nstart_ti" +
- "me\030\002 \001(\004\022\023\n\013last_update\030\003 \001(\004\022\016\n\006result\030" +
- "\004 \001(\014\0224\n\texception\030\005 \001(\0132!.hbase.pb.Fore",
- "ignExceptionMessage\"1\n\005State\022\r\n\tNOT_FOUN" +
- "D\020\000\022\013\n\007RUNNING\020\001\022\014\n\010FINISHED\020\002\"M\n\025AbortP" +
- "rocedureRequest\022\017\n\007proc_id\030\001 \002(\004\022#\n\025mayI" +
- "nterruptIfRunning\030\002 \001(\010:\004true\"6\n\026AbortPr" +
- "ocedureResponse\022\034\n\024is_procedure_aborted\030" +
- "\001 \002(\010\"\027\n\025ListProceduresRequest\"@\n\026ListPr" +
- "oceduresResponse\022&\n\tprocedure\030\001 \003(\0132\023.hb" +
- "ase.pb.Procedure\"\315\001\n\017SetQuotaRequest\022\021\n\t" +
- "user_name\030\001 \001(\t\022\022\n\nuser_group\030\002 \001(\t\022\021\n\tn" +
- "amespace\030\003 \001(\t\022\'\n\ntable_name\030\004 \001(\0132\023.hba",
- "se.pb.TableName\022\022\n\nremove_all\030\005 \001(\010\022\026\n\016b" +
- "ypass_globals\030\006 \001(\010\022+\n\010throttle\030\007 \001(\0132\031." +
- "hbase.pb.ThrottleRequest\"\022\n\020SetQuotaResp" +
- "onse\"J\n\037MajorCompactionTimestampRequest\022" +
- "\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.TableName" +
- "\"U\n(MajorCompactionTimestampForRegionReq" +
- "uest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSp" +
- "ecifier\"@\n MajorCompactionTimestampRespo" +
- "nse\022\034\n\024compaction_timestamp\030\001 \002(\003\"\035\n\033Sec" +
- "urityCapabilitiesRequest\"\354\001\n\034SecurityCap",
- "abilitiesResponse\022G\n\014capabilities\030\001 \003(\0162" +
- "1.hbase.pb.SecurityCapabilitiesResponse." +
- "Capability\"\202\001\n\nCapability\022\031\n\025SIMPLE_AUTH" +
- "ENTICATION\020\000\022\031\n\025SECURE_AUTHENTICATION\020\001\022" +
- "\021\n\rAUTHORIZATION\020\002\022\026\n\022CELL_AUTHORIZATION" +
- "\020\003\022\023\n\017CELL_VISIBILITY\020\004*(\n\020MasterSwitchT" +
- "ype\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\373(\n\rMasterServ" +
- "ice\022e\n\024GetSchemaAlterStatus\022%.hbase.pb.G" +
- "etSchemaAlterStatusRequest\032&.hbase.pb.Ge" +
- "tSchemaAlterStatusResponse\022b\n\023GetTableDe",
- "scriptors\022$.hbase.pb.GetTableDescriptors" +
- "Request\032%.hbase.pb.GetTableDescriptorsRe" +
- "sponse\022P\n\rGetTableNames\022\036.hbase.pb.GetTa" +
- "bleNamesRequest\032\037.hbase.pb.GetTableNames" +
- "Response\022Y\n\020GetClusterStatus\022!.hbase.pb." +
- "GetClusterStatusRequest\032\".hbase.pb.GetCl" +
- "usterStatusResponse\022V\n\017IsMasterRunning\022 " +
- ".hbase.pb.IsMasterRunningRequest\032!.hbase" +
- ".pb.IsMasterRunningResponse\022D\n\tAddColumn" +
- "\022\032.hbase.pb.AddColumnRequest\032\033.hbase.pb.",
- "AddColumnResponse\022M\n\014DeleteColumn\022\035.hbas" +
- "e.pb.DeleteColumnRequest\032\036.hbase.pb.Dele" +
- "teColumnResponse\022M\n\014ModifyColumn\022\035.hbase" +
- ".pb.ModifyColumnRequest\032\036.hbase.pb.Modif" +
- "yColumnResponse\022G\n\nMoveRegion\022\033.hbase.pb" +
- ".MoveRegionRequest\032\034.hbase.pb.MoveRegion" +
- "Response\022k\n\026DispatchMergingRegions\022\'.hba" +
- "se.pb.DispatchMergingRegionsRequest\032(.hb" +
- "ase.pb.DispatchMergingRegionsResponse\022M\n" +
- "\014AssignRegion\022\035.hbase.pb.AssignRegionReq",
- "uest\032\036.hbase.pb.AssignRegionResponse\022S\n\016" +
- "UnassignRegion\022\037.hbase.pb.UnassignRegion" +
- "Request\032 .hbase.pb.UnassignRegionRespons" +
- "e\022P\n\rOfflineRegion\022\036.hbase.pb.OfflineReg" +
- "ionRequest\032\037.hbase.pb.OfflineRegionRespo" +
- "nse\022J\n\013DeleteTable\022\034.hbase.pb.DeleteTabl" +
- "eRequest\032\035.hbase.pb.DeleteTableResponse\022" +
- "P\n\rtruncateTable\022\036.hbase.pb.TruncateTabl" +
- "eRequest\032\037.hbase.pb.TruncateTableRespons" +
- "e\022J\n\013EnableTable\022\034.hbase.pb.EnableTableR",
- "equest\032\035.hbase.pb.EnableTableResponse\022M\n" +
- "\014DisableTable\022\035.hbase.pb.DisableTableReq" +
- "uest\032\036.hbase.pb.DisableTableResponse\022J\n\013" +
- "ModifyTable\022\034.hbase.pb.ModifyTableReques" +
- "t\032\035.hbase.pb.ModifyTableResponse\022J\n\013Crea" +
- "teTable\022\034.hbase.pb.CreateTableRequest\032\035." +
- "hbase.pb.CreateTableResponse\022A\n\010Shutdown" +
- "\022\031.hbase.pb.ShutdownRequest\032\032.hbase.pb.S" +
- "hutdownResponse\022G\n\nStopMaster\022\033.hbase.pb" +
- ".StopMasterRequest\032\034.hbase.pb.StopMaster",
- "Response\022>\n\007Balance\022\030.hbase.pb.BalanceRe" +
- "quest\032\031.hbase.pb.BalanceResponse\022_\n\022SetB" +
- "alancerRunning\022#.hbase.pb.SetBalancerRun" +
- "ningRequest\032$.hbase.pb.SetBalancerRunnin" +
- "gResponse\022\\\n\021IsBalancerEnabled\022\".hbase.p" +
- "b.IsBalancerEnabledRequest\032#.hbase.pb.Is" +
- "BalancerEnabledResponse\022k\n\026SetSplitOrMer" +
- "geEnabled\022\'.hbase.pb.SetSplitOrMergeEnab" +
- "ledRequest\032(.hbase.pb.SetSplitOrMergeEna" +
- "bledResponse\022h\n\025IsSplitOrMergeEnabled\022&.",
- "hbase.pb.IsSplitOrMergeEnabledRequest\032\'." +
- "hbase.pb.IsSplitOrMergeEnabledResponse\022\217" +
- "\001\n\"ReleaseSplitOrMergeLockAndRollback\0223." +
- "hbase.pb.ReleaseSplitOrMergeLockAndRollb" +
- "ackRequest\0324.hbase.pb.ReleaseSplitOrMerg" +
- "eLockAndRollbackResponse\022D\n\tNormalize\022\032." +
- "hbase.pb.NormalizeRequest\032\033.hbase.pb.Nor" +
- "malizeResponse\022e\n\024SetNormalizerRunning\022%" +
- ".hbase.pb.SetNormalizerRunningRequest\032&." +
- "hbase.pb.SetNormalizerRunningResponse\022b\n",
- "\023IsNormalizerEnabled\022$.hbase.pb.IsNormal" +
- "izerEnabledRequest\032%.hbase.pb.IsNormaliz" +
- "erEnabledResponse\022S\n\016RunCatalogScan\022\037.hb" +
- "ase.pb.RunCatalogScanRequest\032 .hbase.pb." +
- "RunCatalogScanResponse\022e\n\024EnableCatalogJ" +
- "anitor\022%.hbase.pb.EnableCatalogJanitorRe" +
- "quest\032&.hbase.pb.EnableCatalogJanitorRes" +
- "ponse\022n\n\027IsCatalogJanitorEnabled\022(.hbase" +
- ".pb.IsCatalogJanitorEnabledRequest\032).hba" +
- "se.pb.IsCatalogJanitorEnabledResponse\022^\n",
- "\021ExecMasterService\022#.hbase.pb.Coprocesso" +
- "rServiceRequest\032$.hbase.pb.CoprocessorSe" +
- "rviceResponse\022A\n\010Snapshot\022\031.hbase.pb.Sna" +
- "pshotRequest\032\032.hbase.pb.SnapshotResponse" +
- "\022h\n\025GetCompletedSnapshots\022&.hbase.pb.Get" +
- "CompletedSnapshotsRequest\032\'.hbase.pb.Get" +
- "CompletedSnapshotsResponse\022S\n\016DeleteSnap" +
- "shot\022\037.hbase.pb.DeleteSnapshotRequest\032 ." +
- "hbase.pb.DeleteSnapshotResponse\022S\n\016IsSna" +
- "pshotDone\022\037.hbase.pb.IsSnapshotDoneReque",
- "st\032 .hbase.pb.IsSnapshotDoneResponse\022V\n\017" +
- "RestoreSnapshot\022 .hbase.pb.RestoreSnapsh" +
- "otRequest\032!.hbase.pb.RestoreSnapshotResp" +
- "onse\022P\n\rExecProcedure\022\036.hbase.pb.ExecPro" +
- "cedureRequest\032\037.hbase.pb.ExecProcedureRe" +
- "sponse\022W\n\024ExecProcedureWithRet\022\036.hbase.p" +
- "b.ExecProcedureRequest\032\037.hbase.pb.ExecPr" +
- "ocedureResponse\022V\n\017IsProcedureDone\022 .hba" +
- "se.pb.IsProcedureDoneRequest\032!.hbase.pb." +
- "IsProcedureDoneResponse\022V\n\017ModifyNamespa",
- "ce\022 .hbase.pb.ModifyNamespaceRequest\032!.h" +
- "base.pb.ModifyNamespaceResponse\022V\n\017Creat" +
- "eNamespace\022 .hbase.pb.CreateNamespaceReq" +
- "uest\032!.hbase.pb.CreateNamespaceResponse\022" +
- "V\n\017DeleteNamespace\022 .hbase.pb.DeleteName" +
- "spaceRequest\032!.hbase.pb.DeleteNamespaceR" +
- "esponse\022k\n\026GetNamespaceDescriptor\022\'.hbas" +
- "e.pb.GetNamespaceDescriptorRequest\032(.hba" +
- "se.pb.GetNamespaceDescriptorResponse\022q\n\030" +
- "ListNamespaceDescriptors\022).hbase.pb.List",
- "NamespaceDescriptorsRequest\032*.hbase.pb.L" +
- "istNamespaceDescriptorsResponse\022\206\001\n\037List" +
- "TableDescriptorsByNamespace\0220.hbase.pb.L" +
- "istTableDescriptorsByNamespaceRequest\0321." +
- "hbase.pb.ListTableDescriptorsByNamespace" +
- "Response\022t\n\031ListTableNamesByNamespace\022*." +
- "hbase.pb.ListTableNamesByNamespaceReques" +
- "t\032+.hbase.pb.ListTableNamesByNamespaceRe" +
- "sponse\022P\n\rGetTableState\022\036.hbase.pb.GetTa" +
- "bleStateRequest\032\037.hbase.pb.GetTableState",
- "Response\022A\n\010SetQuota\022\031.hbase.pb.SetQuota" +
- "Request\032\032.hbase.pb.SetQuotaResponse\022x\n\037g" +
- "etLastMajorCompactionTimestamp\022).hbase.p" +
- "b.MajorCompactionTimestampRequest\032*.hbas" +
- "e.pb.MajorCompactionTimestampResponse\022\212\001" +
- "\n(getLastMajorCompactionTimestampForRegi" +
- "on\0222.hbase.pb.MajorCompactionTimestampFo" +
- "rRegionRequest\032*.hbase.pb.MajorCompactio" +
- "nTimestampResponse\022_\n\022getProcedureResult" +
- "\022#.hbase.pb.GetProcedureResultRequest\032$.",
- "hbase.pb.GetProcedureResultResponse\022h\n\027g" +
- "etSecurityCapabilities\022%.hbase.pb.Securi" +
- "tyCapabilitiesRequest\032&.hbase.pb.Securit" +
- "yCapabilitiesResponse\022S\n\016AbortProcedure\022" +
- "\037.hbase.pb.AbortProcedureRequest\032 .hbase" +
- ".pb.AbortProcedureResponse\022S\n\016ListProced" +
- "ures\022\037.hbase.pb.ListProceduresRequest\032 ." +
- "hbase.pb.ListProceduresResponseBB\n*org.a" +
- "pache.hadoop.hbase.protobuf.generatedB\014M" +
- "asterProtosH\001\210\001\001\240\001\001"
+ "se.pb.SnapshotDescription\022\026\n\013nonce_group" +
+ "\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"*\n\027RestoreSn" +
+ "apshotResponse\022\017\n\007proc_id\030\001 \002(\004\"H\n\025IsSna" +
+ "pshotDoneRequest\022/\n\010snapshot\030\001 \001(\0132\035.hba",
+ "se.pb.SnapshotDescription\"^\n\026IsSnapshotD" +
+ "oneResponse\022\023\n\004done\030\001 \001(\010:\005false\022/\n\010snap" +
+ "shot\030\002 \001(\0132\035.hbase.pb.SnapshotDescriptio" +
+ "n\"O\n\034IsRestoreSnapshotDoneRequest\022/\n\010sna" +
+ "pshot\030\001 \001(\0132\035.hbase.pb.SnapshotDescripti" +
+ "on\"4\n\035IsRestoreSnapshotDoneResponse\022\023\n\004d" +
+ "one\030\001 \001(\010:\005false\"F\n\033GetSchemaAlterStatus" +
+ "Request\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.T" +
+ "ableName\"T\n\034GetSchemaAlterStatusResponse" +
+ "\022\035\n\025yet_to_update_regions\030\001 \001(\r\022\025\n\rtotal",
+ "_regions\030\002 \001(\r\"\213\001\n\032GetTableDescriptorsRe" +
+ "quest\022(\n\013table_names\030\001 \003(\0132\023.hbase.pb.Ta" +
+ "bleName\022\r\n\005regex\030\002 \001(\t\022!\n\022include_sys_ta" +
+ "bles\030\003 \001(\010:\005false\022\021\n\tnamespace\030\004 \001(\t\"J\n\033" +
+ "GetTableDescriptorsResponse\022+\n\014table_sch" +
+ "ema\030\001 \003(\0132\025.hbase.pb.TableSchema\"[\n\024GetT" +
+ "ableNamesRequest\022\r\n\005regex\030\001 \001(\t\022!\n\022inclu" +
+ "de_sys_tables\030\002 \001(\010:\005false\022\021\n\tnamespace\030" +
+ "\003 \001(\t\"A\n\025GetTableNamesResponse\022(\n\013table_" +
+ "names\030\001 \003(\0132\023.hbase.pb.TableName\"?\n\024GetT",
+ "ableStateRequest\022\'\n\ntable_name\030\001 \002(\0132\023.h" +
+ "base.pb.TableName\"B\n\025GetTableStateRespon" +
+ "se\022)\n\013table_state\030\001 \002(\0132\024.hbase.pb.Table" +
+ "State\"\031\n\027GetClusterStatusRequest\"K\n\030GetC" +
+ "lusterStatusResponse\022/\n\016cluster_status\030\001" +
+ " \002(\0132\027.hbase.pb.ClusterStatus\"\030\n\026IsMaste" +
+ "rRunningRequest\"4\n\027IsMasterRunningRespon" +
+ "se\022\031\n\021is_master_running\030\001 \002(\010\"I\n\024ExecPro" +
+ "cedureRequest\0221\n\tprocedure\030\001 \002(\0132\036.hbase" +
+ ".pb.ProcedureDescription\"F\n\025ExecProcedur",
+ "eResponse\022\030\n\020expected_timeout\030\001 \001(\003\022\023\n\013r" +
+ "eturn_data\030\002 \001(\014\"K\n\026IsProcedureDoneReque" +
+ "st\0221\n\tprocedure\030\001 \001(\0132\036.hbase.pb.Procedu" +
+ "reDescription\"`\n\027IsProcedureDoneResponse" +
+ "\022\023\n\004done\030\001 \001(\010:\005false\0220\n\010snapshot\030\002 \001(\0132" +
+ "\036.hbase.pb.ProcedureDescription\",\n\031GetPr" +
+ "ocedureResultRequest\022\017\n\007proc_id\030\001 \002(\004\"\371\001" +
+ "\n\032GetProcedureResultResponse\0229\n\005state\030\001 " +
+ "\002(\0162*.hbase.pb.GetProcedureResultRespons" +
+ "e.State\022\022\n\nstart_time\030\002 \001(\004\022\023\n\013last_upda",
+ "te\030\003 \001(\004\022\016\n\006result\030\004 \001(\014\0224\n\texception\030\005 " +
+ "\001(\0132!.hbase.pb.ForeignExceptionMessage\"1" +
+ "\n\005State\022\r\n\tNOT_FOUND\020\000\022\013\n\007RUNNING\020\001\022\014\n\010F" +
+ "INISHED\020\002\"M\n\025AbortProcedureRequest\022\017\n\007pr" +
+ "oc_id\030\001 \002(\004\022#\n\025mayInterruptIfRunning\030\002 \001" +
+ "(\010:\004true\"6\n\026AbortProcedureResponse\022\034\n\024is" +
+ "_procedure_aborted\030\001 \002(\010\"\027\n\025ListProcedur" +
+ "esRequest\"@\n\026ListProceduresResponse\022&\n\tp" +
+ "rocedure\030\001 \003(\0132\023.hbase.pb.Procedure\"\315\001\n\017" +
+ "SetQuotaRequest\022\021\n\tuser_name\030\001 \001(\t\022\022\n\nus",
+ "er_group\030\002 \001(\t\022\021\n\tnamespace\030\003 \001(\t\022\'\n\ntab" +
+ "le_name\030\004 \001(\0132\023.hbase.pb.TableName\022\022\n\nre" +
+ "move_all\030\005 \001(\010\022\026\n\016bypass_globals\030\006 \001(\010\022+" +
+ "\n\010throttle\030\007 \001(\0132\031.hbase.pb.ThrottleRequ" +
+ "est\"\022\n\020SetQuotaResponse\"J\n\037MajorCompacti" +
+ "onTimestampRequest\022\'\n\ntable_name\030\001 \002(\0132\023" +
+ ".hbase.pb.TableName\"U\n(MajorCompactionTi" +
+ "mestampForRegionRequest\022)\n\006region\030\001 \002(\0132" +
+ "\031.hbase.pb.RegionSpecifier\"@\n MajorCompa" +
+ "ctionTimestampResponse\022\034\n\024compaction_tim",
+ "estamp\030\001 \002(\003\"\035\n\033SecurityCapabilitiesRequ" +
+ "est\"\354\001\n\034SecurityCapabilitiesResponse\022G\n\014" +
+ "capabilities\030\001 \003(\01621.hbase.pb.SecurityCa" +
+ "pabilitiesResponse.Capability\"\202\001\n\nCapabi" +
+ "lity\022\031\n\025SIMPLE_AUTHENTICATION\020\000\022\031\n\025SECUR" +
+ "E_AUTHENTICATION\020\001\022\021\n\rAUTHORIZATION\020\002\022\026\n" +
+ "\022CELL_AUTHORIZATION\020\003\022\023\n\017CELL_VISIBILITY" +
+ "\020\004*(\n\020MasterSwitchType\022\t\n\005SPLIT\020\000\022\t\n\005MER" +
+ "GE\020\0012\373(\n\rMasterService\022e\n\024GetSchemaAlter" +
+ "Status\022%.hbase.pb.GetSchemaAlterStatusRe",
+ "quest\032&.hbase.pb.GetSchemaAlterStatusRes" +
+ "ponse\022b\n\023GetTableDescriptors\022$.hbase.pb." +
+ "GetTableDescriptorsRequest\032%.hbase.pb.Ge" +
+ "tTableDescriptorsResponse\022P\n\rGetTableNam" +
+ "es\022\036.hbase.pb.GetTableNamesRequest\032\037.hba" +
+ "se.pb.GetTableNamesResponse\022Y\n\020GetCluste" +
+ "rStatus\022!.hbase.pb.GetClusterStatusReque" +
+ "st\032\".hbase.pb.GetClusterStatusResponse\022V" +
+ "\n\017IsMasterRunning\022 .hbase.pb.IsMasterRun" +
+ "ningRequest\032!.hbase.pb.IsMasterRunningRe",
+ "sponse\022D\n\tAddColumn\022\032.hbase.pb.AddColumn" +
+ "Request\032\033.hbase.pb.AddColumnResponse\022M\n\014" +
+ "DeleteColumn\022\035.hbase.pb.DeleteColumnRequ" +
+ "est\032\036.hbase.pb.DeleteColumnResponse\022M\n\014M" +
+ "odifyColumn\022\035.hbase.pb.ModifyColumnReque" +
+ "st\032\036.hbase.pb.ModifyColumnResponse\022G\n\nMo" +
+ "veRegion\022\033.hbase.pb.MoveRegionRequest\032\034." +
+ "hbase.pb.MoveRegionResponse\022k\n\026DispatchM" +
+ "ergingRegions\022\'.hbase.pb.DispatchMerging" +
+ "RegionsRequest\032(.hbase.pb.DispatchMergin",
+ "gRegionsResponse\022M\n\014AssignRegion\022\035.hbase" +
+ ".pb.AssignRegionRequest\032\036.hbase.pb.Assig" +
+ "nRegionResponse\022S\n\016UnassignRegion\022\037.hbas" +
+ "e.pb.UnassignRegionRequest\032 .hbase.pb.Un" +
+ "assignRegionResponse\022P\n\rOfflineRegion\022\036." +
+ "hbase.pb.OfflineRegionRequest\032\037.hbase.pb" +
+ ".OfflineRegionResponse\022J\n\013DeleteTable\022\034." +
+ "hbase.pb.DeleteTableRequest\032\035.hbase.pb.D" +
+ "eleteTableResponse\022P\n\rtruncateTable\022\036.hb" +
+ "ase.pb.TruncateTableRequest\032\037.hbase.pb.T",
+ "runcateTableResponse\022J\n\013EnableTable\022\034.hb" +
+ "ase.pb.EnableTableRequest\032\035.hbase.pb.Ena" +
+ "bleTableResponse\022M\n\014DisableTable\022\035.hbase" +
+ ".pb.DisableTableRequest\032\036.hbase.pb.Disab" +
+ "leTableResponse\022J\n\013ModifyTable\022\034.hbase.p" +
+ "b.ModifyTableRequest\032\035.hbase.pb.ModifyTa" +
+ "bleResponse\022J\n\013CreateTable\022\034.hbase.pb.Cr" +
+ "eateTableRequest\032\035.hbase.pb.CreateTableR" +
+ "esponse\022A\n\010Shutdown\022\031.hbase.pb.ShutdownR" +
+ "equest\032\032.hbase.pb.ShutdownResponse\022G\n\nSt",
+ "opMaster\022\033.hbase.pb.StopMasterRequest\032\034." +
+ "hbase.pb.StopMasterResponse\022>\n\007Balance\022\030" +
+ ".hbase.pb.BalanceRequest\032\031.hbase.pb.Bala" +
+ "nceResponse\022_\n\022SetBalancerRunning\022#.hbas" +
+ "e.pb.SetBalancerRunningRequest\032$.hbase.p" +
+ "b.SetBalancerRunningResponse\022\\\n\021IsBalanc" +
+ "erEnabled\022\".hbase.pb.IsBalancerEnabledRe" +
+ "quest\032#.hbase.pb.IsBalancerEnabledRespon" +
+ "se\022k\n\026SetSplitOrMergeEnabled\022\'.hbase.pb." +
+ "SetSplitOrMergeEnabledRequest\032(.hbase.pb",
+ ".SetSplitOrMergeEnabledResponse\022h\n\025IsSpl" +
+ "itOrMergeEnabled\022&.hbase.pb.IsSplitOrMer" +
+ "geEnabledRequest\032\'.hbase.pb.IsSplitOrMer" +
+ "geEnabledResponse\022\217\001\n\"ReleaseSplitOrMerg" +
+ "eLockAndRollback\0223.hbase.pb.ReleaseSplit" +
+ "OrMergeLockAndRollbackRequest\0324.hbase.pb" +
+ ".ReleaseSplitOrMergeLockAndRollbackRespo" +
+ "nse\022D\n\tNormalize\022\032.hbase.pb.NormalizeReq" +
+ "uest\032\033.hbase.pb.NormalizeResponse\022e\n\024Set" +
+ "NormalizerRunning\022%.hbase.pb.SetNormaliz",
+ "erRunningRequest\032&.hbase.pb.SetNormalize" +
+ "rRunningResponse\022b\n\023IsNormalizerEnabled\022" +
+ "$.hbase.pb.IsNormalizerEnabledRequest\032%." +
+ "hbase.pb.IsNormalizerEnabledResponse\022S\n\016" +
+ "RunCatalogScan\022\037.hbase.pb.RunCatalogScan" +
+ "Request\032 .hbase.pb.RunCatalogScanRespons" +
+ "e\022e\n\024EnableCatalogJanitor\022%.hbase.pb.Ena" +
+ "bleCatalogJanitorRequest\032&.hbase.pb.Enab" +
+ "leCatalogJanitorResponse\022n\n\027IsCatalogJan" +
+ "itorEnabled\022(.hbase.pb.IsCatalogJanitorE",
+ "nabledRequest\032).hbase.pb.IsCatalogJanito" +
+ "rEnabledResponse\022^\n\021ExecMasterService\022#." +
+ "hbase.pb.CoprocessorServiceRequest\032$.hba" +
+ "se.pb.CoprocessorServiceResponse\022A\n\010Snap" +
+ "shot\022\031.hbase.pb.SnapshotRequest\032\032.hbase." +
+ "pb.SnapshotResponse\022h\n\025GetCompletedSnaps" +
+ "hots\022&.hbase.pb.GetCompletedSnapshotsReq" +
+ "uest\032\'.hbase.pb.GetCompletedSnapshotsRes" +
+ "ponse\022S\n\016DeleteSnapshot\022\037.hbase.pb.Delet" +
+ "eSnapshotRequest\032 .hbase.pb.DeleteSnapsh",
+ "otResponse\022S\n\016IsSnapshotDone\022\037.hbase.pb." +
+ "IsSnapshotDoneRequest\032 .hbase.pb.IsSnaps" +
+ "hotDoneResponse\022V\n\017RestoreSnapshot\022 .hba" +
+ "se.pb.RestoreSnapshotRequest\032!.hbase.pb." +
+ "RestoreSnapshotResponse\022P\n\rExecProcedure" +
+ "\022\036.hbase.pb.ExecProcedureRequest\032\037.hbase" +
+ ".pb.ExecProcedureResponse\022W\n\024ExecProcedu" +
+ "reWithRet\022\036.hbase.pb.ExecProcedureReques" +
+ "t\032\037.hbase.pb.ExecProcedureResponse\022V\n\017Is" +
+ "ProcedureDone\022 .hbase.pb.IsProcedureDone",
+ "Request\032!.hbase.pb.IsProcedureDoneRespon" +
+ "se\022V\n\017ModifyNamespace\022 .hbase.pb.ModifyN" +
+ "amespaceRequest\032!.hbase.pb.ModifyNamespa" +
+ "ceResponse\022V\n\017CreateNamespace\022 .hbase.pb" +
+ ".CreateNamespaceRequest\032!.hbase.pb.Creat" +
+ "eNamespaceResponse\022V\n\017DeleteNamespace\022 ." +
+ "hbase.pb.DeleteNamespaceRequest\032!.hbase." +
+ "pb.DeleteNamespaceResponse\022k\n\026GetNamespa" +
+ "ceDescriptor\022\'.hbase.pb.GetNamespaceDesc" +
+ "riptorRequest\032(.hbase.pb.GetNamespaceDes",
+ "criptorResponse\022q\n\030ListNamespaceDescript" +
+ "ors\022).hbase.pb.ListNamespaceDescriptorsR" +
+ "equest\032*.hbase.pb.ListNamespaceDescripto" +
+ "rsResponse\022\206\001\n\037ListTableDescriptorsByNam" +
+ "espace\0220.hbase.pb.ListTableDescriptorsBy" +
+ "NamespaceRequest\0321.hbase.pb.ListTableDes" +
+ "criptorsByNamespaceResponse\022t\n\031ListTable" +
+ "NamesByNamespace\022*.hbase.pb.ListTableNam" +
+ "esByNamespaceRequest\032+.hbase.pb.ListTabl" +
+ "eNamesByNamespaceResponse\022P\n\rGetTableSta",
+ "te\022\036.hbase.pb.GetTableStateRequest\032\037.hba" +
+ "se.pb.GetTableStateResponse\022A\n\010SetQuota\022" +
+ "\031.hbase.pb.SetQuotaRequest\032\032.hbase.pb.Se" +
+ "tQuotaResponse\022x\n\037getLastMajorCompaction" +
+ "Timestamp\022).hbase.pb.MajorCompactionTime" +
+ "stampRequest\032*.hbase.pb.MajorCompactionT" +
+ "imestampResponse\022\212\001\n(getLastMajorCompact" +
+ "ionTimestampForRegion\0222.hbase.pb.MajorCo" +
+ "mpactionTimestampForRegionRequest\032*.hbas" +
+ "e.pb.MajorCompactionTimestampResponse\022_\n",
+ "\022getProcedureResult\022#.hbase.pb.GetProced" +
+ "ureResultRequest\032$.hbase.pb.GetProcedure" +
+ "ResultResponse\022h\n\027getSecurityCapabilitie" +
+ "s\022%.hbase.pb.SecurityCapabilitiesRequest" +
+ "\032&.hbase.pb.SecurityCapabilitiesResponse" +
+ "\022S\n\016AbortProcedure\022\037.hbase.pb.AbortProce" +
+ "dureRequest\032 .hbase.pb.AbortProcedureRes" +
+ "ponse\022S\n\016ListProcedures\022\037.hbase.pb.ListP" +
+ "roceduresRequest\032 .hbase.pb.ListProcedur" +
+ "esResponseBB\n*org.apache.hadoop.hbase.pr",
+ "otobuf.generatedB\014MasterProtosH\001\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -65940,13 +66217,13 @@ public final class MasterProtos {
internal_static_hbase_pb_DispatchMergingRegionsRequest_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_DispatchMergingRegionsRequest_descriptor,
- new java.lang.String[] { "RegionA", "RegionB", "Forcible", });
+ new java.lang.String[] { "RegionA", "RegionB", "Forcible", "NonceGroup", "Nonce", });
internal_static_hbase_pb_DispatchMergingRegionsResponse_descriptor =
getDescriptor().getMessageTypes().get(9);
internal_static_hbase_pb_DispatchMergingRegionsResponse_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_DispatchMergingRegionsResponse_descriptor,
- new java.lang.String[] { });
+ new java.lang.String[] { "ProcId", });
internal_static_hbase_pb_AssignRegionRequest_descriptor =
getDescriptor().getMessageTypes().get(10);
internal_static_hbase_pb_AssignRegionRequest_fieldAccessorTable = new
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-protocol/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Master.proto b/hbase-protocol/src/main/protobuf/Master.proto
index ad8111e..23bbbf8 100644
--- a/hbase-protocol/src/main/protobuf/Master.proto
+++ b/hbase-protocol/src/main/protobuf/Master.proto
@@ -85,9 +85,12 @@ message DispatchMergingRegionsRequest {
required RegionSpecifier region_a = 1;
required RegionSpecifier region_b = 2;
optional bool forcible = 3 [default = false];
+ optional uint64 nonce_group = 4 [default = 0];
+ optional uint64 nonce = 5 [default = 0];
}
message DispatchMergingRegionsResponse {
+ optional uint64 proc_id = 1;
}
message AssignRegionRequest {
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-protocol/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/MasterProcedure.proto b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
index 87aae6a..4ef2cd7 100644
--- a/hbase-protocol/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
@@ -262,6 +262,21 @@ message RestoreSnapshotStateData {
repeated RestoreParentToChildRegionsPair parent_to_child_regions_pair_list = 7;
}
+enum DispatchMergingRegionsState {
+ DISPATCH_MERGING_REGIONS_PREPARE = 1;
+ DISPATCH_MERGING_REGIONS_PRE_OPERATION = 2;
+ DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS = 3;
+ DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS = 4;
+ DISPATCH_MERGING_REGIONS_POST_OPERATION = 5;
+}
+
+message DispatchMergingRegionsStateData {
+ required UserInformation user_info = 1;
+ required TableName table_name = 2;
+ repeated RegionInfo region_info = 3;
+ optional bool forcible = 4;
+}
+
message ServerCrashStateData {
required ServerName server_name = 1;
optional bool distributed_log_replay = 2;
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/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 6d77e9d..93e836a 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
@@ -24,7 +24,6 @@ import com.google.common.collect.Maps;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Service;
-
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.reflect.Constructor;
@@ -72,7 +71,6 @@ import org.apache.hadoop.hbase.PleaseHoldException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.RegionStateListener;
import org.apache.hadoop.hbase.ScheduledChore;
-import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
@@ -87,6 +85,7 @@ import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.coprocessor.BypassCoprocessorException;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.exceptions.MergeRegionException;
import org.apache.hadoop.hbase.executor.ExecutorType;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
import org.apache.hadoop.hbase.ipc.RpcServer;
@@ -98,7 +97,6 @@ import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
-import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
@@ -109,6 +107,7 @@ import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
+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;
@@ -140,7 +139,6 @@ import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolic
import org.apache.hadoop.hbase.regionserver.compactions.FIFOCompactionPolicy;
import org.apache.hadoop.hbase.replication.master.TableCFsUpdater;
import org.apache.hadoop.hbase.replication.regionserver.Replication;
-import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
@@ -1270,11 +1268,49 @@ public class HMaster extends HRegionServer implements MasterServices {
}
@Override
- public void dispatchMergingRegions(final HRegionInfo region_a,
- final HRegionInfo region_b, final boolean forcible, final User user) throws IOException {
+ public long dispatchMergingRegions(
+ final HRegionInfo regionInfoA,
+ final HRegionInfo regionInfoB,
+ final boolean forcible,
+ final long nonceGroup,
+ final long nonce) throws IOException {
checkInitialized();
- this.service.submit(new DispatchMergingRegionHandler(this,
- this.catalogJanitorChore, region_a, region_b, forcible, user));
+
+ TableName tableName = regionInfoA.getTable();
+ if (tableName == null || regionInfoB.getTable() == null) {
+ throw new UnknownRegionException ("Can't merge regions without table associated");
+ }
+
+ if (!tableName.equals(regionInfoB.getTable())) {
+ throw new IOException ("Cannot merge regions from two different tables");
+ }
+
+ if (regionInfoA.compareTo(regionInfoB) == 0) {
+ throw new MergeRegionException(
+ "Cannot merge a region to itself " + regionInfoA + ", " + regionInfoB);
+ }
+
+ HRegionInfo [] regionsToMerge = new HRegionInfo[2];
+ regionsToMerge [0] = regionInfoA;
+ regionsToMerge [1] = regionInfoB;
+
+ if (cpHost != null) {
+ cpHost.preDispatchMerge(regionInfoA, regionInfoB);
+ }
+
+ LOG.info(getClientIdAuditPrefix() + " Merge regions "
+ + regionInfoA.getEncodedName() + " and " + regionInfoB.getEncodedName());
+
+ long procId = this.procedureExecutor.submitProcedure(
+ new DispatchMergingRegionsProcedure(
+ procedureExecutor.getEnvironment(), tableName, regionsToMerge, forcible),
+ nonceGroup,
+ nonce);
+
+ if (cpHost != null) {
+ cpHost.postDispatchMerge(regionInfoA, regionInfoB);
+ }
+ return procId;
}
void move(final byte[] encodedRegionName,
@@ -2133,6 +2169,11 @@ public class HMaster extends HRegionServer implements MasterServices {
return this.assignmentManager;
}
+ @Override
+ public CatalogJanitor getCatalogJanitor() {
+ return this.catalogJanitorChore;
+ }
+
public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
return rsFatals;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/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 ef4688f..986ff6e 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
@@ -54,11 +54,9 @@ import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.exceptions.MergeRegionException;
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
import org.apache.hadoop.hbase.ipc.PriorityFunction;
import org.apache.hadoop.hbase.ipc.QosPriority;
-import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.mob.MobUtils;
@@ -497,7 +495,6 @@ public class MasterRpcServices extends RSRpcServices
.toByteArray();
final byte[] encodedNameOfRegionB = request.getRegionB().getValue()
.toByteArray();
- final boolean forcible = request.getForcible();
if (request.getRegionA().getType() != RegionSpecifierType.ENCODED_REGION_NAME
|| request.getRegionB().getType() != RegionSpecifierType.ENCODED_REGION_NAME) {
LOG.warn("mergeRegions specifier type: expected: "
@@ -505,6 +502,7 @@ public class MasterRpcServices extends RSRpcServices
+ request.getRegionA().getType() + ", region_b="
+ request.getRegionB().getType());
}
+
RegionStates regionStates = master.getAssignmentManager().getRegionStates();
RegionState regionStateA = regionStates.getRegionState(Bytes.toString(encodedNameOfRegionA));
RegionState regionStateB = regionStates.getRegionState(Bytes.toString(encodedNameOfRegionB));
@@ -514,43 +512,20 @@ public class MasterRpcServices extends RSRpcServices
: encodedNameOfRegionB)));
}
- if (!regionStateA.isOpened() || !regionStateB.isOpened()) {
- throw new ServiceException(new MergeRegionException(
- "Unable to merge regions not online " + regionStateA + ", " + regionStateB));
- }
-
final HRegionInfo regionInfoA = regionStateA.getRegion();
final HRegionInfo regionInfoB = regionStateB.getRegion();
- if (regionInfoA.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID ||
- regionInfoB.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
- throw new ServiceException(new MergeRegionException("Can't merge non-default replicas"));
- }
- if (regionInfoA.compareTo(regionInfoB) == 0) {
- throw new ServiceException(new MergeRegionException(
- "Unable to merge a region to itself " + regionInfoA + ", " + regionInfoB));
- }
- try {
- master.cpHost.preDispatchMerge(regionInfoA, regionInfoB);
- } catch (IOException ioe) {
- throw new ServiceException(ioe);
- }
-
- if (!forcible && !HRegionInfo.areAdjacent(regionInfoA, regionInfoB)) {
- throw new ServiceException(new MergeRegionException(
- "Unable to merge not adjacent regions "
- + regionInfoA.getRegionNameAsString() + ", "
- + regionInfoB.getRegionNameAsString()
- + " where forcible = " + forcible));
- }
try {
- master.dispatchMergingRegions(regionInfoA, regionInfoB, forcible, RpcServer.getRequestUser());
- master.cpHost.postDispatchMerge(regionInfoA, regionInfoB);
+ long procId = master.dispatchMergingRegions(
+ regionInfoA,
+ regionInfoB,
+ request.getForcible(),
+ request.getNonceGroup(),
+ request.getNonce());
+ return DispatchMergingRegionsResponse.newBuilder().setProcId(procId).build();
} catch (IOException ioe) {
throw new ServiceException(ioe);
}
-
- return DispatchMergingRegionsResponse.newBuilder().build();
}
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/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 21f14e8..9b91572 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
@@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
-import org.apache.hadoop.hbase.security.User;
import com.google.protobuf.Service;
@@ -113,6 +112,11 @@ public interface MasterServices extends Server {
RegionNormalizer getRegionNormalizer();
/**
+ * @return Master's instance of {@link CatalogJanitor}
+ */
+ CatalogJanitor getCatalogJanitor();
+
+ /**
* @return Master's instance of {@link ProcedureExecutor}
*/
ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor();
@@ -285,12 +289,15 @@ public interface MasterServices extends Server {
* @param region_b region to merge
* @param forcible true if do a compulsory merge, otherwise we will only merge
* two adjacent regions
- * @param user effective user
+ * @return procedure Id
* @throws IOException
*/
- void dispatchMergingRegions(
- final HRegionInfo region_a, final HRegionInfo region_b, final boolean forcible, final User user
- ) throws IOException;
+ long dispatchMergingRegions(
+ final HRegionInfo region_a,
+ final HRegionInfo region_b,
+ final boolean forcible,
+ final long nonceGroup,
+ final long nonce) throws IOException;
/**
* @return true if master is the active one
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/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 9da8033..b95b894 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
@@ -1043,7 +1043,7 @@ public class RegionStates {
return regionsByServer;
}
- protected synchronized RegionState getRegionState(final String encodedName) {
+ public synchronized RegionState getRegionState(final String encodedName) {
return regionStates.get(encodedName);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/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 f8ab30f..ffdbd17 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
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionLoad;
-import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.YouAreDeadException;
@@ -68,13 +67,13 @@ import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSeque
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
-import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.RetryCounterFactory;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.zookeeper.KeeperException;
import com.google.common.annotations.VisibleForTesting;
@@ -885,7 +884,7 @@ public class ServerManager {
* @throws IOException
*/
public void sendRegionsMerge(ServerName server, HRegionInfo region_a,
- HRegionInfo region_b, boolean forcible, User user) throws IOException {
+ HRegionInfo region_b, boolean forcible, final UserGroupInformation user) throws IOException {
if (server == null)
throw new NullPointerException("Passed server is null");
if (region_a == null || region_b == null)
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DispatchMergingRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DispatchMergingRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DispatchMergingRegionHandler.java
deleted file mode 100644
index 872cf3f..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DispatchMergingRegionHandler.java
+++ /dev/null
@@ -1,191 +0,0 @@
-/**
- * Copyright The Apache Software Foundation
- *
- * 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.handler;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.RegionLoad;
-import org.apache.hadoop.hbase.ServerLoad;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
-import org.apache.hadoop.hbase.executor.EventHandler;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.master.CatalogJanitor;
-import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.RegionPlan;
-import org.apache.hadoop.hbase.master.RegionStates;
-import org.apache.hadoop.hbase.master.ServerManager;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
-/**
- * Handles MERGE regions request on master: move the regions together(on the
- * same regionserver) and send MERGE RPC to regionserver.
- *
- * NOTE:The real merge is executed on the regionserver
- *
- */
-@InterfaceAudience.Private
-public class DispatchMergingRegionHandler extends EventHandler {
- private static final Log LOG = LogFactory.getLog(DispatchMergingRegionHandler.class);
- private final MasterServices masterServices;
- private final CatalogJanitor catalogJanitor;
- private HRegionInfo region_a;
- private HRegionInfo region_b;
- private final boolean forcible;
- private final int timeout;
- private final User user;
-
- public DispatchMergingRegionHandler(final MasterServices services,
- final CatalogJanitor catalogJanitor, final HRegionInfo region_a,
- final HRegionInfo region_b, final boolean forcible, final User user) {
- super(services, EventType.C_M_MERGE_REGION);
- this.masterServices = services;
- this.catalogJanitor = catalogJanitor;
- this.region_a = region_a;
- this.region_b = region_b;
- this.forcible = forcible;
- this.user = user;
- this.timeout = server.getConfiguration().getInt(
- "hbase.master.regionmerge.timeout", 120 * 1000);
- }
-
- @Override
- public void process() throws IOException {
- boolean regionAHasMergeQualifier = !catalogJanitor.cleanMergeQualifier(region_a);
- if (regionAHasMergeQualifier
- || !catalogJanitor.cleanMergeQualifier(region_b)) {
- LOG.info("Skip merging regions " + region_a.getRegionNameAsString()
- + ", " + region_b.getRegionNameAsString() + ", because region "
- + (regionAHasMergeQualifier ? region_a.getEncodedName() : region_b
- .getEncodedName()) + " has merge qualifier");
- return;
- }
-
- RegionStates regionStates = masterServices.getAssignmentManager()
- .getRegionStates();
- ServerName region_a_location = regionStates.getRegionServerOfRegion(region_a);
- ServerName region_b_location = regionStates.getRegionServerOfRegion(region_b);
- if (region_a_location == null || region_b_location == null) {
- LOG.info("Skip merging regions " + region_a.getRegionNameAsString()
- + ", " + region_b.getRegionNameAsString() + ", because region "
- + (region_a_location == null ? region_a.getEncodedName() : region_b
- .getEncodedName()) + " is not online now");
- return;
- }
- long startTime = EnvironmentEdgeManager.currentTime();
- boolean onSameRS = region_a_location.equals(region_b_location);
-
- // Make sure regions are on the same regionserver before send merge
- // regions request to regionserver
- if (!onSameRS) {
- // Move region_b to region a's location, switch region_a and region_b if
- // region_a's load lower than region_b's, so we will always move lower
- // load region
- RegionLoad loadOfRegionA = getRegionLoad(region_a_location, region_a);
- RegionLoad loadOfRegionB = getRegionLoad(region_b_location, region_b);
- if (loadOfRegionA != null && loadOfRegionB != null
- && loadOfRegionA.getRequestsCount() < loadOfRegionB
- .getRequestsCount()) {
- // switch region_a and region_b
- HRegionInfo tmpRegion = this.region_a;
- this.region_a = this.region_b;
- this.region_b = tmpRegion;
- ServerName tmpLocation = region_a_location;
- region_a_location = region_b_location;
- region_b_location = tmpLocation;
- }
-
- RegionPlan regionPlan = new RegionPlan(region_b, region_b_location,
- region_a_location);
- LOG.info("Moving regions to same server for merge: " + regionPlan.toString());
- masterServices.getAssignmentManager().balance(regionPlan);
- while (!masterServices.isStopped()) {
- 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(region_b);
- region_b_location = masterServices.getAssignmentManager()
- .getRegionStates().getRegionServerOfRegion(region_b);
- onSameRS = region_a_location.equals(region_b_location);
- if (onSameRS || !isRIT) {
- // Regions are on the same RS, or region_b is not in
- // RegionInTransition any more
- break;
- }
- if ((EnvironmentEdgeManager.currentTime() - startTime) > timeout) break;
- } catch (InterruptedException e) {
- InterruptedIOException iioe = new InterruptedIOException();
- iioe.initCause(e);
- throw iioe;
- }
- }
- }
-
- if (onSameRS) {
- startTime = EnvironmentEdgeManager.currentTime();
- while (!masterServices.isStopped()) {
- try {
- masterServices.getServerManager().sendRegionsMerge(region_a_location,
- region_a, region_b, forcible, user);
- LOG.info("Sent merge to server " + region_a_location + " for region " +
- region_a.getEncodedName() + "," + region_b.getEncodedName() + ", focible=" + forcible);
- break;
- } catch (RegionOpeningException roe) {
- if ((EnvironmentEdgeManager.currentTime() - startTime) > timeout) {
- LOG.warn("Failed sending merge to " + region_a_location + " after " + timeout + "ms",
- roe);
- break;
- }
- // Do a retry since region should be online on RS immediately
- } catch (IOException ie) {
- LOG.warn("Failed sending merge to " + region_a_location + " for region " +
- region_a.getEncodedName() + "," + region_b.getEncodedName() + ", focible=" + forcible,
- ie);
- break;
- }
- }
- } else {
- LOG.info("Cancel merging regions " + region_a.getRegionNameAsString()
- + ", " + region_b.getRegionNameAsString()
- + ", because can't move them together after "
- + (EnvironmentEdgeManager.currentTime() - startTime) + "ms");
- }
- }
-
- private RegionLoad getRegionLoad(ServerName sn, HRegionInfo hri) {
- ServerManager serverManager = masterServices.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;
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/f04eeecf/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
index e2035bb..cefd4c1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
@@ -70,7 +70,7 @@ public class MergeNormalizationPlan implements NormalizationPlan {
public void execute(Admin admin) {
LOG.info("Executing merging normalization plan: " + this);
try {
- admin.mergeRegions(firstRegion.getEncodedNameAsBytes(),
+ admin.mergeRegionsAsync(firstRegion.getEncodedNameAsBytes(),
secondRegion.getEncodedNameAsBytes(), true);
} catch (IOException ex) {
LOG.error("Error during region merge: ", ex);