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 2015/08/26 22:55:13 UTC
[1/4] hbase git commit: HBASE-13212 Procedure V2 - master
Create/Modify/Delete namespace (Stephen Yuan Jiang)
Repository: hbase
Updated Branches:
refs/heads/branch-1 9b55f1cd2 -> e4b96cf78
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
new file mode 100644
index 0000000..c91092a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
@@ -0,0 +1,364 @@
+/**
+ * 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.OutputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceExistException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.master.TableNamespaceManager;
+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.CreateNamespaceState;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+/**
+ * The procedure to create a new namespace.
+ */
+@InterfaceAudience.Private
+public class CreateNamespaceProcedure
+ extends StateMachineProcedure<MasterProcedureEnv, CreateNamespaceState>
+ implements TableProcedureInterface {
+ private static final Log LOG = LogFactory.getLog(CreateNamespaceProcedure.class);
+
+ private final AtomicBoolean aborted = new AtomicBoolean(false);
+
+ private NamespaceDescriptor nsDescriptor;
+ private Boolean traceEnabled;
+
+ public CreateNamespaceProcedure() {
+ this.traceEnabled = null;
+ }
+
+ public CreateNamespaceProcedure(
+ final MasterProcedureEnv env,
+ final NamespaceDescriptor nsDescriptor) throws IOException {
+ this.nsDescriptor = nsDescriptor;
+ this.traceEnabled = null;
+ }
+
+ @Override
+ protected Flow executeFromState(final MasterProcedureEnv env, final CreateNamespaceState state)
+ throws InterruptedException {
+ if (isTraceEnabled()) {
+ LOG.trace(this + " execute state=" + state);
+ }
+
+ try {
+ switch (state) {
+ case CREATE_NAMESPACE_PREPARE:
+ prepareCreate(env);
+ setNextState(CreateNamespaceState.CREATE_NAMESPACE_CREATE_DIRECTORY);
+ break;
+ case CREATE_NAMESPACE_CREATE_DIRECTORY:
+ createDirectory(env, nsDescriptor);
+ setNextState(CreateNamespaceState.CREATE_NAMESPACE_INSERT_INTO_NS_TABLE);
+ break;
+ case CREATE_NAMESPACE_INSERT_INTO_NS_TABLE:
+ insertIntoNSTable(env, nsDescriptor);
+ setNextState(CreateNamespaceState.CREATE_NAMESPACE_UPDATE_ZK);
+ break;
+ case CREATE_NAMESPACE_UPDATE_ZK:
+ updateZKNamespaceManager(env, nsDescriptor);
+ setNextState(CreateNamespaceState.CREATE_NAMESPACE_SET_NAMESPACE_QUOTA);
+ break;
+ case CREATE_NAMESPACE_SET_NAMESPACE_QUOTA:
+ setNamespaceQuota(env, nsDescriptor);
+ return Flow.NO_MORE_STATE;
+ default:
+ throw new UnsupportedOperationException(this + " unhandled state=" + state);
+ }
+ } catch (IOException e) {
+ LOG.warn("Error trying to create the namespace" + nsDescriptor.getName()
+ + " (in state=" + state + ")", e);
+
+ setFailure("master-create-namespace", e);
+ }
+ return Flow.HAS_MORE_STATE;
+ }
+
+ @Override
+ protected void rollbackState(final MasterProcedureEnv env, final CreateNamespaceState state)
+ throws IOException {
+ if (isTraceEnabled()) {
+ LOG.trace(this + " rollback state=" + state);
+ }
+ try {
+ switch (state) {
+ case CREATE_NAMESPACE_SET_NAMESPACE_QUOTA:
+ rollbackSetNamespaceQuota(env);
+ break;
+ case CREATE_NAMESPACE_UPDATE_ZK:
+ rollbackZKNamespaceManagerChange(env);
+ break;
+ case CREATE_NAMESPACE_INSERT_INTO_NS_TABLE:
+ rollbackInsertIntoNSTable(env);
+ break;
+ case CREATE_NAMESPACE_CREATE_DIRECTORY:
+ rollbackCreateDirectory(env);
+ break;
+ case CREATE_NAMESPACE_PREPARE:
+ break; // nothing to do
+ default:
+ throw new UnsupportedOperationException(this + " unhandled state=" + state);
+ }
+ } catch (IOException 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 creating the namespace "
+ + nsDescriptor.getName(), e);
+ throw e;
+ }
+ }
+
+ @Override
+ protected CreateNamespaceState getState(final int stateId) {
+ return CreateNamespaceState.valueOf(stateId);
+ }
+
+ @Override
+ protected int getStateId(final CreateNamespaceState state) {
+ return state.getNumber();
+ }
+
+ @Override
+ protected CreateNamespaceState getInitialState() {
+ return CreateNamespaceState.CREATE_NAMESPACE_PREPARE;
+ }
+
+ @Override
+ protected void setNextState(CreateNamespaceState state) {
+ if (aborted.get()) {
+ setAbortFailure("create-namespace", "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.CreateNamespaceStateData.Builder createNamespaceMsg =
+ MasterProcedureProtos.CreateNamespaceStateData.newBuilder().setNamespaceDescriptor(
+ ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor));
+ createNamespaceMsg.build().writeDelimitedTo(stream);
+ }
+
+ @Override
+ public void deserializeStateData(final InputStream stream) throws IOException {
+ super.deserializeStateData(stream);
+
+ MasterProcedureProtos.CreateNamespaceStateData createNamespaceMsg =
+ MasterProcedureProtos.CreateNamespaceStateData.parseDelimitedFrom(stream);
+ nsDescriptor = ProtobufUtil.toNamespaceDescriptor(createNamespaceMsg.getNamespaceDescriptor());
+ }
+
+ @Override
+ public void toStringClassDetails(StringBuilder sb) {
+ sb.append(getClass().getSimpleName());
+ sb.append(" (Namespace=");
+ sb.append(nsDescriptor.getName());
+ sb.append(")");
+ }
+
+ @Override
+ protected boolean acquireLock(final MasterProcedureEnv env) {
+ if (!env.getMasterServices().isInitialized()) {
+ // Namespace manager might not be ready if master is not fully initialized,
+ // return false to reject user namespace creation; return true for default
+ // and system namespace creation (this is part of master initialization).
+ if (nsDescriptor.equals(NamespaceDescriptor.DEFAULT_NAMESPACE) ||
+ nsDescriptor.equals(NamespaceDescriptor.SYSTEM_NAMESPACE)) {
+ return true;
+ }
+
+ return false;
+ }
+ return getTableNamespaceManager(env).acquireExclusiveLock();
+ }
+
+ @Override
+ protected void releaseLock(final MasterProcedureEnv env) {
+ if (env.getMasterServices().isInitialized()) {
+ getTableNamespaceManager(env).releaseExclusiveLock();
+ }
+ }
+
+ @Override
+ public TableName getTableName() {
+ return TableName.NAMESPACE_TABLE_NAME;
+ }
+
+ @Override
+ public TableOperationType getTableOperationType() {
+ return TableOperationType.EDIT;
+ }
+
+ /**
+ * Action before any real action of creating namespace.
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void prepareCreate(final MasterProcedureEnv env) throws IOException {
+ if (getTableNamespaceManager(env).doesNamespaceExist(nsDescriptor.getName())) {
+ throw new NamespaceExistException(nsDescriptor.getName());
+ }
+ getTableNamespaceManager(env).validateTableAndRegionCount(nsDescriptor);
+ }
+
+ /**
+ * Create the namespace directory
+ * @param env MasterProcedureEnv
+ * @param nsDescriptor NamespaceDescriptor
+ * @throws IOException
+ */
+ protected static void createDirectory(
+ final MasterProcedureEnv env,
+ final NamespaceDescriptor nsDescriptor) throws IOException {
+ MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+ mfs.getFileSystem().mkdirs(
+ FSUtils.getNamespaceDir(mfs.getRootDir(), nsDescriptor.getName()));
+ }
+
+ /**
+ * undo create directory
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void rollbackCreateDirectory(final MasterProcedureEnv env) throws IOException {
+ try {
+ DeleteNamespaceProcedure.deleteDirectory(env, nsDescriptor.getName());
+ } catch (Exception e) {
+ // Ignore exception
+ LOG.debug("Rollback of createDirectory throws exception: " + e);
+ }
+ }
+
+ /**
+ * Insert the row into ns table
+ * @param env MasterProcedureEnv
+ * @param nsDescriptor NamespaceDescriptor
+ * @throws IOException
+ */
+ protected static void insertIntoNSTable(
+ final MasterProcedureEnv env,
+ final NamespaceDescriptor nsDescriptor) throws IOException {
+ getTableNamespaceManager(env).insertIntoNSTable(nsDescriptor);
+ }
+
+ /**
+ * Undo the insert.
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void rollbackInsertIntoNSTable(final MasterProcedureEnv env) throws IOException {
+ try {
+ DeleteNamespaceProcedure.deleteFromNSTable(env, nsDescriptor.getName());
+ } catch (Exception e) {
+ // Ignore exception
+ LOG.debug("Rollback of insertIntoNSTable throws exception: " + e);
+ }
+ }
+
+ /**
+ * Update Zookeeper.
+ * @param env MasterProcedureEnv
+ * @param nsDescriptor NamespaceDescriptor
+ * @throws IOException
+ */
+ protected static void updateZKNamespaceManager(
+ final MasterProcedureEnv env,
+ final NamespaceDescriptor nsDescriptor) throws IOException {
+ getTableNamespaceManager(env).updateZKNamespaceManager(nsDescriptor);
+ }
+
+ /**
+ * rollback Zookeeper update.
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void rollbackZKNamespaceManagerChange(final MasterProcedureEnv env) throws IOException {
+ try {
+ DeleteNamespaceProcedure.removeFromZKNamespaceManager(env, nsDescriptor.getName());
+ } catch (Exception e) {
+ // Ignore exception
+ LOG.debug("Rollback of updateZKNamespaceManager throws exception: " + e);
+ }
+ }
+
+ /**
+ * Set quota for the namespace
+ * @param env MasterProcedureEnv
+ * @param nsDescriptor NamespaceDescriptor
+ * @throws IOException
+ **/
+ protected static void setNamespaceQuota(
+ final MasterProcedureEnv env,
+ final NamespaceDescriptor nsDescriptor) throws IOException {
+ if (env.getMasterServices().isInitialized()) {
+ env.getMasterServices().getMasterQuotaManager().setNamespaceQuota(nsDescriptor);
+ }
+ }
+
+ /**
+ * remove quota for the namespace if exists
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ **/
+ private void rollbackSetNamespaceQuota(final MasterProcedureEnv env) throws IOException {
+ try {
+ DeleteNamespaceProcedure.removeNamespaceQuota(env, nsDescriptor.getName());
+ } catch (Exception e) {
+ // Ignore exception
+ LOG.debug("Rollback of setNamespaceQuota throws exception: " + e);
+ }
+ }
+
+ private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
+ return env.getMasterServices().getTableNamespaceManager();
+ }
+
+ /**
+ * 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;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
new file mode 100644
index 0000000..8715b0b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
@@ -0,0 +1,398 @@
+/**
+ * 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.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.constraint.ConstraintException;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.master.TableNamespaceManager;
+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.DeleteNamespaceState;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+/**
+ * The procedure to remove a namespace.
+ */
+@InterfaceAudience.Private
+public class DeleteNamespaceProcedure
+ extends StateMachineProcedure<MasterProcedureEnv, DeleteNamespaceState>
+ implements TableProcedureInterface {
+ private static final Log LOG = LogFactory.getLog(DeleteNamespaceProcedure.class);
+
+ private final AtomicBoolean aborted = new AtomicBoolean(false);
+
+ private NamespaceDescriptor nsDescriptor;
+ private String namespaceName;
+ private Boolean traceEnabled;
+
+ public DeleteNamespaceProcedure() {
+ this.nsDescriptor = null;
+ this.traceEnabled = null;
+ }
+
+ public DeleteNamespaceProcedure(
+ final MasterProcedureEnv env,
+ final String namespaceName) throws IOException {
+ this.namespaceName = namespaceName;
+ this.nsDescriptor = null;
+ this.traceEnabled = null;
+ }
+
+ @Override
+ protected Flow executeFromState(final MasterProcedureEnv env, final DeleteNamespaceState state)
+ throws InterruptedException {
+ if (isTraceEnabled()) {
+ LOG.trace(this + " execute state=" + state);
+ }
+
+ try {
+ switch (state) {
+ case DELETE_NAMESPACE_PREPARE:
+ prepareDelete(env);
+ setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_FROM_NS_TABLE);
+ break;
+ case DELETE_NAMESPACE_DELETE_FROM_NS_TABLE:
+ deleteFromNSTable(env, namespaceName);
+ setNextState(DeleteNamespaceState.DELETE_NAMESPACE_REMOVE_FROM_ZK);
+ break;
+ case DELETE_NAMESPACE_REMOVE_FROM_ZK:
+ removeFromZKNamespaceManager(env, namespaceName);
+ setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_DIRECTORIES);
+ break;
+ case DELETE_NAMESPACE_DELETE_DIRECTORIES:
+ deleteDirectory(env, namespaceName);
+ setNextState(DeleteNamespaceState.DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA);
+ break;
+ case DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA:
+ removeNamespaceQuota(env, namespaceName);
+ return Flow.NO_MORE_STATE;
+ default:
+ throw new UnsupportedOperationException(this + " unhandled state=" + state);
+ }
+ } catch (IOException e) {
+ LOG.warn("Error trying to delete the namespace" + namespaceName
+ + " (in state=" + state + ")", e);
+
+ setFailure("master-delete-namespace", e);
+ }
+ return Flow.HAS_MORE_STATE;
+ }
+
+ @Override
+ protected void rollbackState(final MasterProcedureEnv env, final DeleteNamespaceState state)
+ throws IOException {
+ if (isTraceEnabled()) {
+ LOG.trace(this + " rollback state=" + state);
+ }
+ try {
+ switch (state) {
+ case DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA:
+ rollbacRemoveNamespaceQuota(env);
+ break;
+ case DELETE_NAMESPACE_DELETE_DIRECTORIES:
+ rollbackDeleteDirectory(env);
+ break;
+ case DELETE_NAMESPACE_REMOVE_FROM_ZK:
+ undoRemoveFromZKNamespaceManager(env);
+ break;
+ case DELETE_NAMESPACE_DELETE_FROM_NS_TABLE:
+ undoDeleteFromNSTable(env);
+ break;
+ case DELETE_NAMESPACE_PREPARE:
+ break; // nothing to do
+ default:
+ throw new UnsupportedOperationException(this + " unhandled state=" + state);
+ }
+ } catch (IOException 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 deleting the namespace "
+ + namespaceName, e);
+ throw e;
+ }
+ }
+
+ @Override
+ protected DeleteNamespaceState getState(final int stateId) {
+ return DeleteNamespaceState.valueOf(stateId);
+ }
+
+ @Override
+ protected int getStateId(final DeleteNamespaceState state) {
+ return state.getNumber();
+ }
+
+ @Override
+ protected DeleteNamespaceState getInitialState() {
+ return DeleteNamespaceState.DELETE_NAMESPACE_PREPARE;
+ }
+
+ @Override
+ protected void setNextState(DeleteNamespaceState state) {
+ if (aborted.get()) {
+ setAbortFailure("delete-namespace", "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.DeleteNamespaceStateData.Builder deleteNamespaceMsg =
+ MasterProcedureProtos.DeleteNamespaceStateData.newBuilder().setNamespaceName(namespaceName);
+ if (this.nsDescriptor != null) {
+ deleteNamespaceMsg.setNamespaceDescriptor(
+ ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor));
+ }
+ deleteNamespaceMsg.build().writeDelimitedTo(stream);
+ }
+
+ @Override
+ public void deserializeStateData(final InputStream stream) throws IOException {
+ super.deserializeStateData(stream);
+
+ MasterProcedureProtos.DeleteNamespaceStateData deleteNamespaceMsg =
+ MasterProcedureProtos.DeleteNamespaceStateData.parseDelimitedFrom(stream);
+ namespaceName = deleteNamespaceMsg.getNamespaceName();
+ if (deleteNamespaceMsg.hasNamespaceDescriptor()) {
+ nsDescriptor =
+ ProtobufUtil.toNamespaceDescriptor(deleteNamespaceMsg.getNamespaceDescriptor());
+ }
+ }
+
+ @Override
+ public void toStringClassDetails(StringBuilder sb) {
+ sb.append(getClass().getSimpleName());
+ sb.append(" (Namespace=");
+ sb.append(namespaceName);
+ sb.append(")");
+ }
+
+ @Override
+ protected boolean acquireLock(final MasterProcedureEnv env) {
+ return getTableNamespaceManager(env).acquireExclusiveLock();
+ }
+
+ @Override
+ protected void releaseLock(final MasterProcedureEnv env) {
+ getTableNamespaceManager(env).releaseExclusiveLock();
+ }
+
+ @Override
+ public TableName getTableName() {
+ return TableName.NAMESPACE_TABLE_NAME;
+ }
+
+ @Override
+ public TableOperationType getTableOperationType() {
+ return TableOperationType.EDIT;
+ }
+
+ /**
+ * Action before any real action of deleting namespace.
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void prepareDelete(final MasterProcedureEnv env) throws IOException {
+ if (getTableNamespaceManager(env).doesNamespaceExist(namespaceName) == false) {
+ throw new NamespaceNotFoundException(namespaceName);
+ }
+ if (NamespaceDescriptor.RESERVED_NAMESPACES.contains(namespaceName)) {
+ throw new ConstraintException("Reserved namespace "+ namespaceName +" cannot be removed.");
+ }
+
+ int tableCount = 0;
+ try {
+ tableCount = env.getMasterServices().listTableDescriptorsByNamespace(namespaceName).size();
+ } catch (FileNotFoundException fnfe) {
+ throw new NamespaceNotFoundException(namespaceName);
+ }
+ if (tableCount > 0) {
+ throw new ConstraintException("Only empty namespaces can be removed. " +
+ "Namespace "+ namespaceName + " has "+ tableCount +" tables");
+ }
+
+ // This is used for rollback
+ nsDescriptor = getTableNamespaceManager(env).get(namespaceName);
+ }
+
+ /**
+ * delete the row from namespace table
+ * @param env MasterProcedureEnv
+ * @param namespaceName name of the namespace in string format
+ * @throws IOException
+ */
+ protected static void deleteFromNSTable(
+ final MasterProcedureEnv env,
+ final String namespaceName) throws IOException {
+ getTableNamespaceManager(env).removeFromNSTable(namespaceName);
+ }
+
+ /**
+ * undo the delete
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void undoDeleteFromNSTable(final MasterProcedureEnv env) {
+ try {
+ if (nsDescriptor != null) {
+ CreateNamespaceProcedure.insertIntoNSTable(env, nsDescriptor);
+ }
+ } catch (Exception e) {
+ // Ignore
+ LOG.debug("Rollback of deleteFromNSTable throws exception: " + e);
+ }
+ }
+
+ /**
+ * remove from Zookeeper.
+ * @param env MasterProcedureEnv
+ * @param namespaceName name of the namespace in string format
+ * @throws IOException
+ */
+ protected static void removeFromZKNamespaceManager(
+ final MasterProcedureEnv env,
+ final String namespaceName) throws IOException {
+ getTableNamespaceManager(env).removeFromZKNamespaceManager(namespaceName);
+ }
+
+ /**
+ * undo the remove from Zookeeper
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void undoRemoveFromZKNamespaceManager(final MasterProcedureEnv env) {
+ try {
+ if (nsDescriptor != null) {
+ CreateNamespaceProcedure.updateZKNamespaceManager(env, nsDescriptor);
+ }
+ } catch (Exception e) {
+ // Ignore
+ LOG.debug("Rollback of removeFromZKNamespaceManager throws exception: " + e);
+ }
+ }
+
+ /**
+ * Delete the namespace directories from the file system
+ * @param env MasterProcedureEnv
+ * @param namespaceName name of the namespace in string format
+ * @throws IOException
+ */
+ protected static void deleteDirectory(
+ final MasterProcedureEnv env,
+ final String namespaceName) throws IOException {
+ MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+ FileSystem fs = mfs.getFileSystem();
+ Path p = FSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName);
+
+ try {
+ for(FileStatus status : fs.listStatus(p)) {
+ if (!HConstants.HBASE_NON_TABLE_DIRS.contains(status.getPath().getName())) {
+ throw new IOException("Namespace directory contains table dir: " + status.getPath());
+ }
+ }
+ if (!fs.delete(FSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName), true)) {
+ throw new IOException("Failed to remove namespace: " + namespaceName);
+ }
+ } catch (FileNotFoundException e) {
+ // File already deleted, continue
+ LOG.debug("deleteDirectory throws exception: " + e);
+ }
+ }
+
+ /**
+ * undo delete directory
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void rollbackDeleteDirectory(final MasterProcedureEnv env) throws IOException {
+ try {
+ CreateNamespaceProcedure.createDirectory(env, nsDescriptor);
+ } catch (Exception e) {
+ // Ignore exception
+ LOG.debug("Rollback of deleteDirectory throws exception: " + e);
+ }
+ }
+
+ /**
+ * remove quota for the namespace
+ * @param env MasterProcedureEnv
+ * @param namespaceName name of the namespace in string format
+ * @throws IOException
+ **/
+ protected static void removeNamespaceQuota(
+ final MasterProcedureEnv env,
+ final String namespaceName) throws IOException {
+ env.getMasterServices().getMasterQuotaManager().removeNamespaceQuota(namespaceName);
+ }
+
+ /**
+ * undo remove quota for the namespace
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ **/
+ private void rollbacRemoveNamespaceQuota(final MasterProcedureEnv env) throws IOException {
+ try {
+ CreateNamespaceProcedure.setNamespaceQuota(env, nsDescriptor);
+ } catch (Exception e) {
+ // Ignore exception
+ LOG.debug("Rollback of removeNamespaceQuota throws exception: " + e);
+ }
+ }
+
+ private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
+ return env.getMasterServices().getTableNamespaceManager();
+ }
+ /**
+ * 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;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
new file mode 100644
index 0000000..ba5c0ad
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
@@ -0,0 +1,281 @@
+/**
+ * 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.OutputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.master.TableNamespaceManager;
+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.ModifyNamespaceState;
+
+/**
+ * The procedure to add a namespace to an existing table.
+ */
+@InterfaceAudience.Private
+public class ModifyNamespaceProcedure
+ extends StateMachineProcedure<MasterProcedureEnv, ModifyNamespaceState>
+ implements TableProcedureInterface {
+ private static final Log LOG = LogFactory.getLog(ModifyNamespaceProcedure.class);
+
+ private final AtomicBoolean aborted = new AtomicBoolean(false);
+
+ private NamespaceDescriptor oldNsDescriptor;
+ private NamespaceDescriptor newNsDescriptor;
+ private Boolean traceEnabled;
+
+ public ModifyNamespaceProcedure() {
+ this.oldNsDescriptor = null;
+ this.traceEnabled = null;
+ }
+
+ public ModifyNamespaceProcedure(
+ final MasterProcedureEnv env,
+ final NamespaceDescriptor newNsDescriptor) throws IOException {
+ this.oldNsDescriptor = null;
+ this.newNsDescriptor = newNsDescriptor;
+ this.traceEnabled = null;
+ }
+
+ @Override
+ protected Flow executeFromState(final MasterProcedureEnv env, final ModifyNamespaceState state)
+ throws InterruptedException {
+ if (isTraceEnabled()) {
+ LOG.trace(this + " execute state=" + state);
+ }
+
+ try {
+ switch (state) {
+ case MODIFY_NAMESPACE_PREPARE:
+ prepareModify(env);
+ setNextState(ModifyNamespaceState.MODIFY_NAMESPACE_UPDATE_NS_TABLE);
+ break;
+ case MODIFY_NAMESPACE_UPDATE_NS_TABLE:
+ insertIntoNSTable(env);
+ setNextState(ModifyNamespaceState.MODIFY_NAMESPACE_UPDATE_ZK);
+ break;
+ case MODIFY_NAMESPACE_UPDATE_ZK:
+ updateZKNamespaceManager(env);
+ return Flow.NO_MORE_STATE;
+ default:
+ throw new UnsupportedOperationException(this + " unhandled state=" + state);
+ }
+ } catch (IOException e) {
+ LOG.warn("Error trying to modify the namespace" + newNsDescriptor.getName()
+ + " (in state=" + state + ")", e);
+
+ setFailure("master-modify-namespace", e);
+ }
+ return Flow.HAS_MORE_STATE;
+ }
+
+ @Override
+ protected void rollbackState(final MasterProcedureEnv env, final ModifyNamespaceState state)
+ throws IOException {
+ if (isTraceEnabled()) {
+ LOG.trace(this + " rollback state=" + state);
+ }
+ try {
+ switch (state) {
+ case MODIFY_NAMESPACE_UPDATE_ZK:
+ rollbackZKNamespaceManagerChange(env);
+ break;
+ case MODIFY_NAMESPACE_UPDATE_NS_TABLE:
+ rollbackUpdateInNSTable(env);
+ break;
+ case MODIFY_NAMESPACE_PREPARE:
+ break; // nothing to do
+ default:
+ throw new UnsupportedOperationException(this + " unhandled state=" + state);
+ }
+ } catch (IOException 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 creating the namespace "
+ + newNsDescriptor.getName(), e);
+ throw e;
+ }
+ }
+
+ @Override
+ protected ModifyNamespaceState getState(final int stateId) {
+ return ModifyNamespaceState.valueOf(stateId);
+ }
+
+ @Override
+ protected int getStateId(final ModifyNamespaceState state) {
+ return state.getNumber();
+ }
+
+ @Override
+ protected ModifyNamespaceState getInitialState() {
+ return ModifyNamespaceState.MODIFY_NAMESPACE_PREPARE;
+ }
+
+ @Override
+ protected void setNextState(ModifyNamespaceState state) {
+ if (aborted.get()) {
+ setAbortFailure("modify-namespace", "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.ModifyNamespaceStateData.Builder modifyNamespaceMsg =
+ MasterProcedureProtos.ModifyNamespaceStateData.newBuilder().setNamespaceDescriptor(
+ ProtobufUtil.toProtoNamespaceDescriptor(this.newNsDescriptor));
+ if (this.oldNsDescriptor != null) {
+ modifyNamespaceMsg.setUnmodifiedNamespaceDescriptor(
+ ProtobufUtil.toProtoNamespaceDescriptor(this.oldNsDescriptor));
+ }
+ modifyNamespaceMsg.build().writeDelimitedTo(stream);
+ }
+
+ @Override
+ public void deserializeStateData(final InputStream stream) throws IOException {
+ super.deserializeStateData(stream);
+
+ MasterProcedureProtos.ModifyNamespaceStateData modifyNamespaceMsg =
+ MasterProcedureProtos.ModifyNamespaceStateData.parseDelimitedFrom(stream);
+ newNsDescriptor =
+ ProtobufUtil.toNamespaceDescriptor(modifyNamespaceMsg.getNamespaceDescriptor());
+ if (modifyNamespaceMsg.hasUnmodifiedNamespaceDescriptor()) {
+ oldNsDescriptor =
+ ProtobufUtil.toNamespaceDescriptor(modifyNamespaceMsg.getUnmodifiedNamespaceDescriptor());
+ }
+ }
+
+ @Override
+ public void toStringClassDetails(StringBuilder sb) {
+ sb.append(getClass().getSimpleName());
+ sb.append(" (Namespace=");
+ sb.append(newNsDescriptor.getName());
+ sb.append(")");
+ }
+
+ @Override
+ protected boolean acquireLock(final MasterProcedureEnv env) {
+ return getTableNamespaceManager(env).acquireExclusiveLock();
+ }
+
+ @Override
+ protected void releaseLock(final MasterProcedureEnv env) {
+ getTableNamespaceManager(env).releaseExclusiveLock();
+ }
+
+ @Override
+ public TableName getTableName() {
+ return TableName.NAMESPACE_TABLE_NAME;
+ }
+
+ @Override
+ public TableOperationType getTableOperationType() {
+ return TableOperationType.EDIT;
+ }
+
+ /**
+ * Action before any real action of adding namespace.
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void prepareModify(final MasterProcedureEnv env) throws IOException {
+ if (getTableNamespaceManager(env).doesNamespaceExist(newNsDescriptor.getName()) == false) {
+ throw new NamespaceNotFoundException(newNsDescriptor.getName());
+ }
+ getTableNamespaceManager(env).validateTableAndRegionCount(newNsDescriptor);
+
+ // This is used for rollback
+ oldNsDescriptor = getTableNamespaceManager(env).get(newNsDescriptor.getName());
+ }
+
+ /**
+ * Insert/update the row into namespace table
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void insertIntoNSTable(final MasterProcedureEnv env) throws IOException {
+ getTableNamespaceManager(env).insertIntoNSTable(newNsDescriptor);
+ }
+
+ /**
+ * rollback the row into namespace table
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void rollbackUpdateInNSTable(final MasterProcedureEnv env) throws IOException {
+ if (oldNsDescriptor != null) {
+ getTableNamespaceManager(env).insertIntoNSTable(oldNsDescriptor);
+ }
+ }
+
+ /**
+ * Update Zookeeper.
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void updateZKNamespaceManager(final MasterProcedureEnv env) throws IOException {
+ getTableNamespaceManager(env).updateZKNamespaceManager(newNsDescriptor);
+ }
+
+ /**
+ * Update Zookeeper during undo.
+ * @param env MasterProcedureEnv
+ * @throws IOException
+ */
+ private void rollbackZKNamespaceManagerChange(final MasterProcedureEnv env) throws IOException {
+ if (oldNsDescriptor != null) {
+ getTableNamespaceManager(env).updateZKNamespaceManager(oldNsDescriptor);
+ }
+ }
+
+ private TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
+ return env.getMasterServices().getTableNamespaceManager();
+ }
+ /**
+ * 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;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index fbdf6a2..e06dd44 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -377,17 +377,34 @@ public class TestCatalogJanitor {
}
@Override
- public void createNamespace(NamespaceDescriptor descriptor) throws IOException {
+ public void createNamespace(
+ final NamespaceDescriptor descriptor,
+ final long nonceGroup,
+ final long nonce) throws IOException {
+ //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public void createNamespaceSync(
+ final NamespaceDescriptor descriptor,
+ final long nonceGroup,
+ final long nonce) throws IOException {
//To change body of implemented methods use File | Settings | File Templates.
}
@Override
- public void modifyNamespace(NamespaceDescriptor descriptor) throws IOException {
+ public void modifyNamespace(
+ final NamespaceDescriptor descriptor,
+ final long nonceGroup,
+ final long nonce) throws IOException {
//To change body of implemented methods use File | Settings | File Templates.
}
@Override
- public void deleteNamespace(String name) throws IOException {
+ public void deleteNamespace(
+ final String name,
+ final long nonceGroup,
+ final long nonce) throws IOException {
//To change body of implemented methods use File | Settings | File Templates.
}
@@ -477,6 +494,11 @@ public class TestCatalogJanitor {
}
@Override
+ public TableNamespaceManager getTableNamespaceManager() {
+ return null;
+ }
+
+ @Override
public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b,
boolean forcible) throws IOException {
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
index 77ca96e..4d5ed56 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.client.BufferedMutator;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.NonceGenerator;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
@@ -56,7 +55,6 @@ import org.apache.hadoop.hbase.util.MD5Hash;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
public class MasterProcedureTestingUtility {
private static final Log LOG = LogFactory.getLog(MasterProcedureTestingUtility.class);
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateNamespaceProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateNamespaceProcedure.java
new file mode 100644
index 0000000..7797b89
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateNamespaceProcedure.java
@@ -0,0 +1,291 @@
+/**
+ * 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 static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+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.HConstants;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceExistException;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
+import org.apache.hadoop.hbase.constraint.ConstraintException;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureResult;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceState;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+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;
+
+@Category(MediumTests.class)
+public class TestCreateNamespaceProcedure {
+ private static final Log LOG = LogFactory.getLog(TestCreateNamespaceProcedure.class);
+
+ protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+ private static long nonceGroup = HConstants.NO_NONCE;
+ private static long nonce = HConstants.NO_NONCE;
+
+ private static void setupConf(Configuration conf) {
+ conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+ }
+
+ @BeforeClass
+ public static void setupCluster() throws Exception {
+ setupConf(UTIL.getConfiguration());
+ UTIL.startMiniCluster(1);
+ }
+
+ @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 {
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
+ nonceGroup =
+ MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
+ nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
+ }
+
+ @Test(timeout = 60000)
+ public void testCreateNamespace() throws Exception {
+ final NamespaceDescriptor nsd = NamespaceDescriptor.create("testCreateNamespace").build();
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ long procId = procExec.submitProcedure(
+ new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+
+ validateNamespaceCreated(nsd);
+ }
+
+ @Test(timeout=60000)
+ public void testCreateSameNamespaceTwice() throws Exception {
+ final NamespaceDescriptor nsd =
+ NamespaceDescriptor.create("testCreateSameNamespaceTwice").build();
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ long procId1 = procExec.submitProcedure(
+ new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId1);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+
+ // Create the namespace that exists
+ long procId2 = procExec.submitProcedure(
+ new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup + 1,
+ nonce + 1);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId2);
+
+ // Second create should fail with NamespaceExistException
+ ProcedureResult result = procExec.getResult(procId2);
+ assertTrue(result.isFailed());
+ LOG.debug("Create namespace failed with exception: " + result.getException());
+ assertTrue(result.getException().getCause() instanceof NamespaceExistException);
+ }
+
+ @Test(timeout=60000)
+ public void testCreateSystemNamespace() throws Exception {
+ final NamespaceDescriptor nsd =
+ UTIL.getHBaseAdmin().getNamespaceDescriptor(NamespaceDescriptor.SYSTEM_NAMESPACE.getName());
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ long procId = procExec.submitProcedure(
+ new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ ProcedureResult result = procExec.getResult(procId);
+ assertTrue(result.isFailed());
+ LOG.debug("Create namespace failed with exception: " + result.getException());
+ assertTrue(result.getException().getCause() instanceof NamespaceExistException);
+ }
+
+ @Test(timeout=60000)
+ public void testCreateNamespaceWithInvalidRegionCount() throws Exception {
+ final NamespaceDescriptor nsd =
+ NamespaceDescriptor.create("testCreateNamespaceWithInvalidRegionCount").build();
+ final String nsKey = "hbase.namespace.quota.maxregions";
+ final String nsValue = "-1";
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ nsd.setConfiguration(nsKey, nsValue);
+
+ long procId = procExec.submitProcedure(
+ new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ ProcedureResult result = procExec.getResult(procId);
+ assertTrue(result.isFailed());
+ LOG.debug("Create namespace failed with exception: " + result.getException());
+ assertTrue(result.getException().getCause() instanceof ConstraintException);
+ }
+
+ @Test(timeout=60000)
+ public void testCreateNamespaceWithInvalidTableCount() throws Exception {
+ final NamespaceDescriptor nsd =
+ NamespaceDescriptor.create("testCreateNamespaceWithInvalidTableCount").build();
+ final String nsKey = "hbase.namespace.quota.maxtables";
+ final String nsValue = "-1";
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ nsd.setConfiguration(nsKey, nsValue);
+
+ long procId = procExec.submitProcedure(
+ new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ ProcedureResult result = procExec.getResult(procId);
+ assertTrue(result.isFailed());
+ LOG.debug("Create namespace failed with exception: " + result.getException());
+ assertTrue(result.getException().getCause() instanceof ConstraintException);
+ }
+
+ @Test(timeout=60000)
+ public void testCreateSameNamespaceTwiceWithSameNonce() throws Exception {
+ final NamespaceDescriptor nsd =
+ NamespaceDescriptor.create("testCreateSameNamespaceTwiceWithSameNonce").build();
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ long procId1 = procExec.submitProcedure(
+ new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+ long procId2 = procExec.submitProcedure(
+ new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId1);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+
+ validateNamespaceCreated(nsd);
+
+ // Wait the completion and expect not fail - because it is the same proc
+ ProcedureTestingUtility.waitProcedure(procExec, procId2);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
+ assertTrue(procId1 == procId2);
+ }
+
+ @Test(timeout = 60000)
+ public void testRecoveryAndDoubleExecution() throws Exception {
+ final NamespaceDescriptor nsd =
+ NamespaceDescriptor.create("testRecoveryAndDoubleExecution").build();
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+ // Start the CreateNamespace procedure && kill the executor
+ long procId = procExec.submitProcedure(
+ new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+
+ // Restart the executor and execute the step twice
+ int numberOfSteps = CreateNamespaceState.values().length;
+ MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
+ procExec,
+ procId,
+ numberOfSteps,
+ CreateNamespaceState.values());
+
+ // Validate the creation of namespace
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+ validateNamespaceCreated(nsd);
+ }
+
+ @Test(timeout = 60000)
+ public void testRollbackAndDoubleExecution() throws Exception {
+ final NamespaceDescriptor nsd =
+ NamespaceDescriptor.create("testRollbackAndDoubleExecution").build();
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+ // Start the CreateNamespace procedure && kill the executor
+ long procId = procExec.submitProcedure(
+ new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+
+ int numberOfSteps = CreateNamespaceState.values().length - 2; // failing in the middle of proc
+ MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
+ procExec,
+ procId,
+ numberOfSteps,
+ CreateNamespaceState.values());
+
+ // Validate the non-existence of namespace
+ try {
+ NamespaceDescriptor nsDescriptor = UTIL.getHBaseAdmin().getNamespaceDescriptor(nsd.getName());
+ assertNull(nsDescriptor);
+ } catch (NamespaceNotFoundException nsnfe) {
+ // Expected
+ LOG.info("The namespace " + nsd.getName() + " is not created.");
+ }
+ }
+
+ private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+ return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+ }
+
+ private void validateNamespaceCreated(NamespaceDescriptor nsd) throws IOException {
+ NamespaceDescriptor createdNsDescriptor =
+ UTIL.getHBaseAdmin().getNamespaceDescriptor(nsd.getName());
+ assertNotNull(createdNsDescriptor);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteNamespaceProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteNamespaceProcedure.java
new file mode 100644
index 0000000..dc250fb
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteNamespaceProcedure.java
@@ -0,0 +1,281 @@
+/**
+ * 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 static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+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.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.constraint.ConstraintException;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureResult;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteNamespaceState;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+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;
+
+@Category(MediumTests.class)
+public class TestDeleteNamespaceProcedure {
+ private static final Log LOG = LogFactory.getLog(TestDeleteNamespaceProcedure.class);
+
+ protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+ private static long nonceGroup = HConstants.NO_NONCE;
+ private static long nonce = HConstants.NO_NONCE;
+
+ private static void setupConf(Configuration conf) {
+ conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+ }
+
+ @BeforeClass
+ public static void setupCluster() throws Exception {
+ setupConf(UTIL.getConfiguration());
+ UTIL.startMiniCluster(1);
+ }
+
+ @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 {
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
+ nonceGroup =
+ MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
+ nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
+ for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
+ LOG.info("Tear down, remove table=" + htd.getTableName());
+ UTIL.deleteTable(htd.getTableName());
+ }
+ }
+
+ @Test(timeout = 60000)
+ public void testDeleteNamespace() throws Exception {
+ final String namespaceName = "testDeleteNamespace";
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ createNamespaceForTesting(namespaceName);
+
+ long procId = procExec.submitProcedure(
+ new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+
+ validateNamespaceNotExist(namespaceName);
+ }
+
+ @Test(timeout=60000)
+ public void testDeleteNonExistNamespace() throws Exception {
+ final String namespaceName = "testDeleteNonExistNamespace";
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ validateNamespaceNotExist(namespaceName);
+
+ long procId = procExec.submitProcedure(
+ new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ // Expect fail with NamespaceNotFoundException
+ ProcedureResult result = procExec.getResult(procId);
+ assertTrue(result.isFailed());
+ LOG.debug("Delete namespace failed with exception: " + result.getException());
+ assertTrue(result.getException().getCause() instanceof NamespaceNotFoundException);
+ }
+
+ @Test(timeout=60000)
+ public void testDeleteSystemNamespace() throws Exception {
+ final String namespaceName = NamespaceDescriptor.SYSTEM_NAMESPACE.getName();
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ long procId = procExec.submitProcedure(
+ new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ ProcedureResult result = procExec.getResult(procId);
+ assertTrue(result.isFailed());
+ LOG.debug("Delete namespace failed with exception: " + result.getException());
+ assertTrue(result.getException().getCause() instanceof ConstraintException);
+ }
+
+ @Test(timeout=60000)
+ public void testDeleteNonEmptyNamespace() throws Exception {
+ final String namespaceName = "testDeleteNonExistNamespace";
+ final TableName tableName = TableName.valueOf("testDeleteNonExistNamespace:t1");
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+ // create namespace
+ createNamespaceForTesting(namespaceName);
+ // create the table under the new namespace
+ MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1");
+
+ long procId = procExec.submitProcedure(
+ new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ ProcedureResult result = procExec.getResult(procId);
+ assertTrue(result.isFailed());
+ LOG.debug("Delete namespace failed with exception: " + result.getException());
+ assertTrue(result.getException().getCause() instanceof ConstraintException);
+ }
+
+ @Test(timeout=60000)
+ public void testDeleteSameNamespaceTwiceWithSameNonce() throws Exception {
+ final String namespaceName = "testDeleteSameNamespaceTwiceWithSameNonce";
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ createNamespaceForTesting(namespaceName);
+
+ long procId1 = procExec.submitProcedure(
+ new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
+ nonceGroup,
+ nonce);
+ long procId2 = procExec.submitProcedure(
+ new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId1);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+
+ validateNamespaceNotExist(namespaceName);
+
+ // Wait the completion and expect not fail - because it is the same proc
+ ProcedureTestingUtility.waitProcedure(procExec, procId2);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
+ assertTrue(procId1 == procId2);
+ }
+
+ @Test(timeout = 60000)
+ public void testRecoveryAndDoubleExecution() throws Exception {
+ final String namespaceName = "testRecoveryAndDoubleExecution";
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ createNamespaceForTesting(namespaceName);
+
+ ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+ // Start the DeleteNamespace procedure && kill the executor
+ long procId = procExec.submitProcedure(
+ new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
+ nonceGroup,
+ nonce);
+
+ // Restart the executor and execute the step twice
+ int numberOfSteps = DeleteNamespaceState.values().length;
+ MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
+ procExec,
+ procId,
+ numberOfSteps,
+ DeleteNamespaceState.values());
+
+ // Validate the deletion of namespace
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+ validateNamespaceNotExist(namespaceName);
+ }
+
+ @Test(timeout = 60000)
+ public void testRollbackAndDoubleExecution() throws Exception {
+ final String namespaceName = "testRollbackAndDoubleExecution";
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ createNamespaceForTesting(namespaceName);
+
+ ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+ // Start the DeleteNamespace procedure && kill the executor
+ long procId = procExec.submitProcedure(
+ new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
+ nonceGroup,
+ nonce);
+
+ int numberOfSteps = DeleteNamespaceState.values().length - 2; // failing in the middle of proc
+ MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
+ procExec,
+ procId,
+ numberOfSteps,
+ DeleteNamespaceState.values());
+
+ // Validate the namespace still exists
+ NamespaceDescriptor createdNsDescriptor=
+ UTIL.getHBaseAdmin().getNamespaceDescriptor(namespaceName);
+ assertNotNull(createdNsDescriptor);
+ }
+
+ private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+ return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+ }
+
+ private void createNamespaceForTesting(final String namespaceName) throws Exception {
+ final NamespaceDescriptor nsd = NamespaceDescriptor.create(namespaceName).build();
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ long procId = procExec.submitProcedure(
+ new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup + 1,
+ nonce + 1);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+ }
+
+ public static void validateNamespaceNotExist(final String nsName) throws IOException {
+ try {
+ NamespaceDescriptor nsDescriptor = UTIL.getHBaseAdmin().getNamespaceDescriptor(nsName);
+ assertNull(nsDescriptor);
+ } catch (NamespaceNotFoundException nsnfe) {
+ // Expected
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java
new file mode 100644
index 0000000..3eaaa84
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java
@@ -0,0 +1,294 @@
+/**
+ * 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 static org.junit.Assert.*;
+
+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.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
+import org.apache.hadoop.hbase.constraint.ConstraintException;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureResult;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceState;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+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;
+
+@Category(MediumTests.class)
+public class TestModifyNamespaceProcedure {
+ private static final Log LOG = LogFactory.getLog(TestModifyNamespaceProcedure.class);
+
+ protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+ private static long nonceGroup = HConstants.NO_NONCE;
+ private static long nonce = HConstants.NO_NONCE;
+
+ private static void setupConf(Configuration conf) {
+ conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+ }
+
+ @BeforeClass
+ public static void setupCluster() throws Exception {
+ setupConf(UTIL.getConfiguration());
+ UTIL.startMiniCluster(1);
+ }
+
+ @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 {
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
+ nonceGroup =
+ MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
+ nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
+ for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
+ LOG.info("Tear down, remove table=" + htd.getTableName());
+ UTIL.deleteTable(htd.getTableName());
+ }
+ }
+
+
+ @Test(timeout = 60000)
+ public void testModifyNamespace() throws Exception {
+ final NamespaceDescriptor nsd = NamespaceDescriptor.create("testModifyNamespace").build();
+ final String nsKey1 = "hbase.namespace.quota.maxregions";
+ final String nsValue1before = "1111";
+ final String nsValue1after = "9999";
+ final String nsKey2 = "hbase.namespace.quota.maxtables";
+ final String nsValue2 = "10";
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ nsd.setConfiguration(nsKey1, nsValue1before);
+ createNamespaceForTesting(nsd);
+
+ // Before modify
+ NamespaceDescriptor currentNsDescriptor =
+ UTIL.getHBaseAdmin().getNamespaceDescriptor(nsd.getName());
+ assertEquals(currentNsDescriptor.getConfigurationValue(nsKey1), nsValue1before);
+ assertNull(currentNsDescriptor.getConfigurationValue(nsKey2));
+
+ // Update
+ nsd.setConfiguration(nsKey1, nsValue1after);
+ nsd.setConfiguration(nsKey2, nsValue2);
+
+ long procId1 = procExec.submitProcedure(
+ new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId1);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+
+ // Verify the namespace is updated.
+ currentNsDescriptor =
+ UTIL.getHBaseAdmin().getNamespaceDescriptor(nsd.getName());
+ assertEquals(nsd.getConfigurationValue(nsKey1), nsValue1after);
+ assertEquals(currentNsDescriptor.getConfigurationValue(nsKey2), nsValue2);
+ }
+
+ @Test(timeout=60000)
+ public void testModifyNonExistNamespace() throws Exception {
+ final String namespaceName = "testModifyNonExistNamespace";
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ try {
+ NamespaceDescriptor nsDescriptor = UTIL.getHBaseAdmin().getNamespaceDescriptor(namespaceName);
+ assertNull(nsDescriptor);
+ } catch (NamespaceNotFoundException nsnfe) {
+ // Expected
+ LOG.debug("The namespace " + namespaceName + " does not exist. This is expected.");
+ }
+
+ final NamespaceDescriptor nsd = NamespaceDescriptor.create(namespaceName).build();
+
+ long procId = procExec.submitProcedure(
+ new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+
+ // Expect fail with NamespaceNotFoundException
+ ProcedureResult result = procExec.getResult(procId);
+ assertTrue(result.isFailed());
+ LOG.debug("modify namespace failed with exception: " + result.getException());
+ assertTrue(result.getException().getCause() instanceof NamespaceNotFoundException);
+ }
+
+ @Test(timeout=60000)
+ public void testModifyNamespaceWithInvalidRegionCount() throws Exception {
+ final NamespaceDescriptor nsd =
+ NamespaceDescriptor.create("testModifyNamespaceWithInvalidRegionCount").build();
+ final String nsKey = "hbase.namespace.quota.maxregions";
+ final String nsValue = "-1";
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ createNamespaceForTesting(nsd);
+
+ // Modify
+ nsd.setConfiguration(nsKey, nsValue);
+
+ long procId = procExec.submitProcedure(
+ new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ ProcedureResult result = procExec.getResult(procId);
+ assertTrue(result.isFailed());
+ LOG.debug("Modify namespace failed with exception: " + result.getException());
+ assertTrue(result.getException().getCause() instanceof ConstraintException);
+ }
+
+ @Test(timeout=60000)
+ public void testModifyNamespaceWithInvalidTableCount() throws Exception {
+ final NamespaceDescriptor nsd =
+ NamespaceDescriptor.create("testModifyNamespaceWithInvalidTableCount").build();
+ final String nsKey = "hbase.namespace.quota.maxtables";
+ final String nsValue = "-1";
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ createNamespaceForTesting(nsd);
+
+ // Modify
+ nsd.setConfiguration(nsKey, nsValue);
+
+ long procId = procExec.submitProcedure(
+ new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ ProcedureResult result = procExec.getResult(procId);
+ assertTrue(result.isFailed());
+ LOG.debug("Modify namespace failed with exception: " + result.getException());
+ assertTrue(result.getException().getCause() instanceof ConstraintException);
+ }
+
+ @Test(timeout = 60000)
+ public void testRecoveryAndDoubleExecution() throws Exception {
+ final NamespaceDescriptor nsd =
+ NamespaceDescriptor.create("testRecoveryAndDoubleExecution").build();
+ final String nsKey = "foo";
+ final String nsValue = "bar";
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ createNamespaceForTesting(nsd);
+ ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+ // Modify
+ nsd.setConfiguration(nsKey, nsValue);
+
+ // Start the Modify procedure && kill the executor
+ long procId = procExec.submitProcedure(
+ new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+
+ // Restart the executor and execute the step twice
+ int numberOfSteps = ModifyNamespaceState.values().length;
+ MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
+ procExec,
+ procId,
+ numberOfSteps,
+ ModifyNamespaceState.values());
+
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+ // Validate
+ NamespaceDescriptor currentNsDescriptor =
+ UTIL.getHBaseAdmin().getNamespaceDescriptor(nsd.getName());
+ assertEquals(currentNsDescriptor.getConfigurationValue(nsKey), nsValue);
+ }
+
+ @Test(timeout = 60000)
+ public void testRollbackAndDoubleExecution() throws Exception {
+ final NamespaceDescriptor nsd =
+ NamespaceDescriptor.create("testRollbackAndDoubleExecution").build();
+ final String nsKey = "foo";
+ final String nsValue = "bar";
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ createNamespaceForTesting(nsd);
+ ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+ // Modify
+ nsd.setConfiguration(nsKey, nsValue);
+
+ // Start the Modify procedure && kill the executor
+ long procId = procExec.submitProcedure(
+ new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
+ nonceGroup,
+ nonce);
+
+ // Failing in the middle of proc
+ int numberOfSteps = ModifyNamespaceState.values().length - 2;
+ MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
+ procExec,
+ procId,
+ numberOfSteps,
+ ModifyNamespaceState.values());
+
+ // Validate
+ NamespaceDescriptor currentNsDescriptor =
+ UTIL.getHBaseAdmin().getNamespaceDescriptor(nsd.getName());
+ assertNull(currentNsDescriptor.getConfigurationValue(nsKey));
+ }
+
+ private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+ return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+ }
+
+ private void createNamespaceForTesting(NamespaceDescriptor nsDescriptor) throws Exception {
+ final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+ long procId = procExec.submitProcedure(
+ new CreateNamespaceProcedure(procExec.getEnvironment(), nsDescriptor),
+ nonceGroup + 1,
+ nonce + 1);
+ // Wait the completion
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+ }
+}
[4/4] hbase git commit: HBASE-13212 Procedure V2 - master
Create/Modify/Delete namespace (Stephen Yuan Jiang)
Posted by sy...@apache.org.
HBASE-13212 Procedure V2 - master Create/Modify/Delete namespace (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/e4b96cf7
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e4b96cf7
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e4b96cf7
Branch: refs/heads/branch-1
Commit: e4b96cf78a4e7b15a1bc46a4f060f65065732786
Parents: 9b55f1c
Author: Stephen Yuan Jiang <sy...@gmail.com>
Authored: Wed Aug 26 13:54:50 2015 -0700
Committer: Stephen Yuan Jiang <sy...@gmail.com>
Committed: Wed Aug 26 13:54:50 2015 -0700
----------------------------------------------------------------------
.../generated/MasterProcedureProtos.java | 2910 ++++++++++++++++--
.../hbase/protobuf/generated/MasterProtos.java | 989 ++++--
hbase-protocol/src/main/protobuf/Master.proto | 6 +
.../src/main/protobuf/MasterProcedure.proto | 36 +
.../apache/hadoop/hbase/ZKNamespaceManager.java | 9 +-
.../org/apache/hadoop/hbase/master/HMaster.java | 56 +-
.../hadoop/hbase/master/MasterRpcServices.java | 14 +-
.../hadoop/hbase/master/MasterServices.java | 38 +-
.../hbase/master/TableNamespaceManager.java | 232 +-
.../procedure/CreateNamespaceProcedure.java | 364 +++
.../procedure/DeleteNamespaceProcedure.java | 398 +++
.../procedure/ModifyNamespaceProcedure.java | 281 ++
.../hadoop/hbase/master/TestCatalogJanitor.java | 28 +-
.../MasterProcedureTestingUtility.java | 2 -
.../procedure/TestCreateNamespaceProcedure.java | 291 ++
.../procedure/TestDeleteNamespaceProcedure.java | 281 ++
.../procedure/TestModifyNamespaceProcedure.java | 294 ++
17 files changed, 5655 insertions(+), 574 deletions(-)
----------------------------------------------------------------------
[3/4] hbase git commit: HBASE-13212 Procedure V2 - master
Create/Modify/Delete namespace (Stephen Yuan Jiang)
Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/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 9bf4c98..d40c1f7 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
@@ -499,6 +499,315 @@ public final class MasterProcedureProtos {
}
/**
+ * Protobuf enum {@code hbase.pb.CreateNamespaceState}
+ */
+ public enum CreateNamespaceState
+ implements com.google.protobuf.ProtocolMessageEnum {
+ /**
+ * <code>CREATE_NAMESPACE_PREPARE = 1;</code>
+ */
+ CREATE_NAMESPACE_PREPARE(0, 1),
+ /**
+ * <code>CREATE_NAMESPACE_CREATE_DIRECTORY = 2;</code>
+ */
+ CREATE_NAMESPACE_CREATE_DIRECTORY(1, 2),
+ /**
+ * <code>CREATE_NAMESPACE_INSERT_INTO_NS_TABLE = 3;</code>
+ */
+ CREATE_NAMESPACE_INSERT_INTO_NS_TABLE(2, 3),
+ /**
+ * <code>CREATE_NAMESPACE_UPDATE_ZK = 4;</code>
+ */
+ CREATE_NAMESPACE_UPDATE_ZK(3, 4),
+ /**
+ * <code>CREATE_NAMESPACE_SET_NAMESPACE_QUOTA = 5;</code>
+ */
+ CREATE_NAMESPACE_SET_NAMESPACE_QUOTA(4, 5),
+ ;
+
+ /**
+ * <code>CREATE_NAMESPACE_PREPARE = 1;</code>
+ */
+ public static final int CREATE_NAMESPACE_PREPARE_VALUE = 1;
+ /**
+ * <code>CREATE_NAMESPACE_CREATE_DIRECTORY = 2;</code>
+ */
+ public static final int CREATE_NAMESPACE_CREATE_DIRECTORY_VALUE = 2;
+ /**
+ * <code>CREATE_NAMESPACE_INSERT_INTO_NS_TABLE = 3;</code>
+ */
+ public static final int CREATE_NAMESPACE_INSERT_INTO_NS_TABLE_VALUE = 3;
+ /**
+ * <code>CREATE_NAMESPACE_UPDATE_ZK = 4;</code>
+ */
+ public static final int CREATE_NAMESPACE_UPDATE_ZK_VALUE = 4;
+ /**
+ * <code>CREATE_NAMESPACE_SET_NAMESPACE_QUOTA = 5;</code>
+ */
+ public static final int CREATE_NAMESPACE_SET_NAMESPACE_QUOTA_VALUE = 5;
+
+
+ public final int getNumber() { return value; }
+
+ public static CreateNamespaceState valueOf(int value) {
+ switch (value) {
+ case 1: return CREATE_NAMESPACE_PREPARE;
+ case 2: return CREATE_NAMESPACE_CREATE_DIRECTORY;
+ case 3: return CREATE_NAMESPACE_INSERT_INTO_NS_TABLE;
+ case 4: return CREATE_NAMESPACE_UPDATE_ZK;
+ case 5: return CREATE_NAMESPACE_SET_NAMESPACE_QUOTA;
+ default: return null;
+ }
+ }
+
+ public static com.google.protobuf.Internal.EnumLiteMap<CreateNamespaceState>
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static com.google.protobuf.Internal.EnumLiteMap<CreateNamespaceState>
+ internalValueMap =
+ new com.google.protobuf.Internal.EnumLiteMap<CreateNamespaceState>() {
+ public CreateNamespaceState findValueByNumber(int number) {
+ return CreateNamespaceState.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(4);
+ }
+
+ private static final CreateNamespaceState[] VALUES = values();
+
+ public static CreateNamespaceState 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 CreateNamespaceState(int index, int value) {
+ this.index = index;
+ this.value = value;
+ }
+
+ // @@protoc_insertion_point(enum_scope:hbase.pb.CreateNamespaceState)
+ }
+
+ /**
+ * Protobuf enum {@code hbase.pb.ModifyNamespaceState}
+ */
+ public enum ModifyNamespaceState
+ implements com.google.protobuf.ProtocolMessageEnum {
+ /**
+ * <code>MODIFY_NAMESPACE_PREPARE = 1;</code>
+ */
+ MODIFY_NAMESPACE_PREPARE(0, 1),
+ /**
+ * <code>MODIFY_NAMESPACE_UPDATE_NS_TABLE = 2;</code>
+ */
+ MODIFY_NAMESPACE_UPDATE_NS_TABLE(1, 2),
+ /**
+ * <code>MODIFY_NAMESPACE_UPDATE_ZK = 3;</code>
+ */
+ MODIFY_NAMESPACE_UPDATE_ZK(2, 3),
+ ;
+
+ /**
+ * <code>MODIFY_NAMESPACE_PREPARE = 1;</code>
+ */
+ public static final int MODIFY_NAMESPACE_PREPARE_VALUE = 1;
+ /**
+ * <code>MODIFY_NAMESPACE_UPDATE_NS_TABLE = 2;</code>
+ */
+ public static final int MODIFY_NAMESPACE_UPDATE_NS_TABLE_VALUE = 2;
+ /**
+ * <code>MODIFY_NAMESPACE_UPDATE_ZK = 3;</code>
+ */
+ public static final int MODIFY_NAMESPACE_UPDATE_ZK_VALUE = 3;
+
+
+ public final int getNumber() { return value; }
+
+ public static ModifyNamespaceState valueOf(int value) {
+ switch (value) {
+ case 1: return MODIFY_NAMESPACE_PREPARE;
+ case 2: return MODIFY_NAMESPACE_UPDATE_NS_TABLE;
+ case 3: return MODIFY_NAMESPACE_UPDATE_ZK;
+ default: return null;
+ }
+ }
+
+ public static com.google.protobuf.Internal.EnumLiteMap<ModifyNamespaceState>
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static com.google.protobuf.Internal.EnumLiteMap<ModifyNamespaceState>
+ internalValueMap =
+ new com.google.protobuf.Internal.EnumLiteMap<ModifyNamespaceState>() {
+ public ModifyNamespaceState findValueByNumber(int number) {
+ return ModifyNamespaceState.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(5);
+ }
+
+ private static final ModifyNamespaceState[] VALUES = values();
+
+ public static ModifyNamespaceState 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 ModifyNamespaceState(int index, int value) {
+ this.index = index;
+ this.value = value;
+ }
+
+ // @@protoc_insertion_point(enum_scope:hbase.pb.ModifyNamespaceState)
+ }
+
+ /**
+ * Protobuf enum {@code hbase.pb.DeleteNamespaceState}
+ */
+ public enum DeleteNamespaceState
+ implements com.google.protobuf.ProtocolMessageEnum {
+ /**
+ * <code>DELETE_NAMESPACE_PREPARE = 1;</code>
+ */
+ DELETE_NAMESPACE_PREPARE(0, 1),
+ /**
+ * <code>DELETE_NAMESPACE_DELETE_FROM_NS_TABLE = 2;</code>
+ */
+ DELETE_NAMESPACE_DELETE_FROM_NS_TABLE(1, 2),
+ /**
+ * <code>DELETE_NAMESPACE_REMOVE_FROM_ZK = 3;</code>
+ */
+ DELETE_NAMESPACE_REMOVE_FROM_ZK(2, 3),
+ /**
+ * <code>DELETE_NAMESPACE_DELETE_DIRECTORIES = 4;</code>
+ */
+ DELETE_NAMESPACE_DELETE_DIRECTORIES(3, 4),
+ /**
+ * <code>DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA = 5;</code>
+ */
+ DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA(4, 5),
+ ;
+
+ /**
+ * <code>DELETE_NAMESPACE_PREPARE = 1;</code>
+ */
+ public static final int DELETE_NAMESPACE_PREPARE_VALUE = 1;
+ /**
+ * <code>DELETE_NAMESPACE_DELETE_FROM_NS_TABLE = 2;</code>
+ */
+ public static final int DELETE_NAMESPACE_DELETE_FROM_NS_TABLE_VALUE = 2;
+ /**
+ * <code>DELETE_NAMESPACE_REMOVE_FROM_ZK = 3;</code>
+ */
+ public static final int DELETE_NAMESPACE_REMOVE_FROM_ZK_VALUE = 3;
+ /**
+ * <code>DELETE_NAMESPACE_DELETE_DIRECTORIES = 4;</code>
+ */
+ public static final int DELETE_NAMESPACE_DELETE_DIRECTORIES_VALUE = 4;
+ /**
+ * <code>DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA = 5;</code>
+ */
+ public static final int DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA_VALUE = 5;
+
+
+ public final int getNumber() { return value; }
+
+ public static DeleteNamespaceState valueOf(int value) {
+ switch (value) {
+ case 1: return DELETE_NAMESPACE_PREPARE;
+ case 2: return DELETE_NAMESPACE_DELETE_FROM_NS_TABLE;
+ case 3: return DELETE_NAMESPACE_REMOVE_FROM_ZK;
+ case 4: return DELETE_NAMESPACE_DELETE_DIRECTORIES;
+ case 5: return DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA;
+ default: return null;
+ }
+ }
+
+ public static com.google.protobuf.Internal.EnumLiteMap<DeleteNamespaceState>
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static com.google.protobuf.Internal.EnumLiteMap<DeleteNamespaceState>
+ internalValueMap =
+ new com.google.protobuf.Internal.EnumLiteMap<DeleteNamespaceState>() {
+ public DeleteNamespaceState findValueByNumber(int number) {
+ return DeleteNamespaceState.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(6);
+ }
+
+ private static final DeleteNamespaceState[] VALUES = values();
+
+ public static DeleteNamespaceState 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 DeleteNamespaceState(int index, int value) {
+ this.index = index;
+ this.value = value;
+ }
+
+ // @@protoc_insertion_point(enum_scope:hbase.pb.DeleteNamespaceState)
+ }
+
+ /**
* Protobuf enum {@code hbase.pb.AddColumnFamilyState}
*/
public enum AddColumnFamilyState
@@ -582,7 +891,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(4);
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(7);
}
private static final AddColumnFamilyState[] VALUES = values();
@@ -691,7 +1000,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(5);
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(8);
}
private static final ModifyColumnFamilyState[] VALUES = values();
@@ -809,7 +1118,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(6);
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(9);
}
private static final DeleteColumnFamilyState[] VALUES = values();
@@ -927,7 +1236,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(7);
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(10);
}
private static final EnableTableState[] VALUES = values();
@@ -1045,7 +1354,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(8);
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(11);
}
private static final DisableTableState[] VALUES = values();
@@ -1198,7 +1507,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(9);
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(12);
}
private static final ServerCrashState[] VALUES = values();
@@ -5839,258 +6148,2323 @@ public final class MasterProcedureProtos {
isClean());
tableName_ = null;
}
- return tableNameBuilder_;
+ 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_;
+ }
+
+ // @@protoc_insertion_point(builder_scope:hbase.pb.DeleteTableStateData)
+ }
+
+ static {
+ defaultInstance = new DeleteTableStateData(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:hbase.pb.DeleteTableStateData)
+ }
+
+ public interface CreateNamespaceStateDataOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ boolean hasNamespaceDescriptor();
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor getNamespaceDescriptor();
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder getNamespaceDescriptorOrBuilder();
+ }
+ /**
+ * Protobuf type {@code hbase.pb.CreateNamespaceStateData}
+ */
+ public static final class CreateNamespaceStateData extends
+ com.google.protobuf.GeneratedMessage
+ implements CreateNamespaceStateDataOrBuilder {
+ // Use CreateNamespaceStateData.newBuilder() to construct.
+ private CreateNamespaceStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private CreateNamespaceStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final CreateNamespaceStateData defaultInstance;
+ public static CreateNamespaceStateData getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public CreateNamespaceStateData getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private CreateNamespaceStateData(
+ 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.NamespaceDescriptor.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ subBuilder = namespaceDescriptor_.toBuilder();
+ }
+ namespaceDescriptor_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(namespaceDescriptor_);
+ namespaceDescriptor_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000001;
+ 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 {
+ 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_CreateNamespaceStateData_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CreateNamespaceStateData_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser<CreateNamespaceStateData> PARSER =
+ new com.google.protobuf.AbstractParser<CreateNamespaceStateData>() {
+ public CreateNamespaceStateData parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new CreateNamespaceStateData(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser<CreateNamespaceStateData> getParserForType() {
+ return PARSER;
+ }
+
+ private int bitField0_;
+ // required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;
+ public static final int NAMESPACE_DESCRIPTOR_FIELD_NUMBER = 1;
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor namespaceDescriptor_;
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public boolean hasNamespaceDescriptor() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor getNamespaceDescriptor() {
+ return namespaceDescriptor_;
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder getNamespaceDescriptorOrBuilder() {
+ return namespaceDescriptor_;
+ }
+
+ private void initFields() {
+ namespaceDescriptor_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance();
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ if (!hasNamespaceDescriptor()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!getNamespaceDescriptor().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, namespaceDescriptor_);
+ }
+ 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, namespaceDescriptor_);
+ }
+ 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.CreateNamespaceStateData)) {
+ return super.equals(obj);
+ }
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData) obj;
+
+ boolean result = true;
+ result = result && (hasNamespaceDescriptor() == other.hasNamespaceDescriptor());
+ if (hasNamespaceDescriptor()) {
+ result = result && getNamespaceDescriptor()
+ .equals(other.getNamespaceDescriptor());
+ }
+ 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 (hasNamespaceDescriptor()) {
+ hash = (37 * hash) + NAMESPACE_DESCRIPTOR_FIELD_NUMBER;
+ hash = (53 * hash) + getNamespaceDescriptor().hashCode();
+ }
+ hash = (29 * hash) + getUnknownFields().hashCode();
+ memoizedHashCode = hash;
+ return hash;
+ }
+
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData 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.CreateNamespaceStateData parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData 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.CreateNamespaceStateData parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData 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.CreateNamespaceStateData parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData 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.CreateNamespaceStateData parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData 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.CreateNamespaceStateData 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.CreateNamespaceStateData}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateDataOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CreateNamespaceStateData_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CreateNamespaceStateData_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData.Builder.class);
+ }
+
+ // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ getNamespaceDescriptorFieldBuilder();
+ }
+ }
+ private static Builder create() {
+ return new Builder();
+ }
+
+ public Builder clear() {
+ super.clear();
+ if (namespaceDescriptorBuilder_ == null) {
+ namespaceDescriptor_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance();
+ } else {
+ namespaceDescriptorBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ 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_CreateNamespaceStateData_descriptor;
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData getDefaultInstanceForType() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData.getDefaultInstance();
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData build() {
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData buildPartial() {
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ if (namespaceDescriptorBuilder_ == null) {
+ result.namespaceDescriptor_ = namespaceDescriptor_;
+ } else {
+ result.namespaceDescriptor_ = namespaceDescriptorBuilder_.build();
+ }
+ 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.CreateNamespaceStateData) {
+ return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData other) {
+ if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData.getDefaultInstance()) return this;
+ if (other.hasNamespaceDescriptor()) {
+ mergeNamespaceDescriptor(other.getNamespaceDescriptor());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ if (!hasNamespaceDescriptor()) {
+
+ return false;
+ }
+ if (!getNamespaceDescriptor().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.CreateNamespaceStateData parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ // required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor namespaceDescriptor_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> namespaceDescriptorBuilder_;
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public boolean hasNamespaceDescriptor() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor getNamespaceDescriptor() {
+ if (namespaceDescriptorBuilder_ == null) {
+ return namespaceDescriptor_;
+ } else {
+ return namespaceDescriptorBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public Builder setNamespaceDescriptor(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor value) {
+ if (namespaceDescriptorBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ namespaceDescriptor_ = value;
+ onChanged();
+ } else {
+ namespaceDescriptorBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public Builder setNamespaceDescriptor(
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder builderForValue) {
+ if (namespaceDescriptorBuilder_ == null) {
+ namespaceDescriptor_ = builderForValue.build();
+ onChanged();
+ } else {
+ namespaceDescriptorBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public Builder mergeNamespaceDescriptor(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor value) {
+ if (namespaceDescriptorBuilder_ == null) {
+ if (((bitField0_ & 0x00000001) == 0x00000001) &&
+ namespaceDescriptor_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance()) {
+ namespaceDescriptor_ =
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.newBuilder(namespaceDescriptor_).mergeFrom(value).buildPartial();
+ } else {
+ namespaceDescriptor_ = value;
+ }
+ onChanged();
+ } else {
+ namespaceDescriptorBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public Builder clearNamespaceDescriptor() {
+ if (namespaceDescriptorBuilder_ == null) {
+ namespaceDescriptor_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance();
+ onChanged();
+ } else {
+ namespaceDescriptorBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder getNamespaceDescriptorBuilder() {
+ bitField0_ |= 0x00000001;
+ onChanged();
+ return getNamespaceDescriptorFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder getNamespaceDescriptorOrBuilder() {
+ if (namespaceDescriptorBuilder_ != null) {
+ return namespaceDescriptorBuilder_.getMessageOrBuilder();
+ } else {
+ return namespaceDescriptor_;
+ }
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder>
+ getNamespaceDescriptorFieldBuilder() {
+ if (namespaceDescriptorBuilder_ == null) {
+ namespaceDescriptorBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder>(
+ namespaceDescriptor_,
+ getParentForChildren(),
+ isClean());
+ namespaceDescriptor_ = null;
+ }
+ return namespaceDescriptorBuilder_;
+ }
+
+ // @@protoc_insertion_point(builder_scope:hbase.pb.CreateNamespaceStateData)
+ }
+
+ static {
+ defaultInstance = new CreateNamespaceStateData(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:hbase.pb.CreateNamespaceStateData)
+ }
+
+ public interface ModifyNamespaceStateDataOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ boolean hasNamespaceDescriptor();
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor getNamespaceDescriptor();
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder getNamespaceDescriptorOrBuilder();
+
+ // optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ boolean hasUnmodifiedNamespaceDescriptor();
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor getUnmodifiedNamespaceDescriptor();
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder getUnmodifiedNamespaceDescriptorOrBuilder();
+ }
+ /**
+ * Protobuf type {@code hbase.pb.ModifyNamespaceStateData}
+ */
+ public static final class ModifyNamespaceStateData extends
+ com.google.protobuf.GeneratedMessage
+ implements ModifyNamespaceStateDataOrBuilder {
+ // Use ModifyNamespaceStateData.newBuilder() to construct.
+ private ModifyNamespaceStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private ModifyNamespaceStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final ModifyNamespaceStateData defaultInstance;
+ public static ModifyNamespaceStateData getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public ModifyNamespaceStateData getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private ModifyNamespaceStateData(
+ 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.NamespaceDescriptor.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ subBuilder = namespaceDescriptor_.toBuilder();
+ }
+ namespaceDescriptor_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(namespaceDescriptor_);
+ namespaceDescriptor_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000001;
+ break;
+ }
+ case 18: {
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ subBuilder = unmodifiedNamespaceDescriptor_.toBuilder();
+ }
+ unmodifiedNamespaceDescriptor_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(unmodifiedNamespaceDescriptor_);
+ unmodifiedNamespaceDescriptor_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000002;
+ 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 {
+ 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_ModifyNamespaceStateData_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyNamespaceStateData_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser<ModifyNamespaceStateData> PARSER =
+ new com.google.protobuf.AbstractParser<ModifyNamespaceStateData>() {
+ public ModifyNamespaceStateData parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new ModifyNamespaceStateData(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser<ModifyNamespaceStateData> getParserForType() {
+ return PARSER;
+ }
+
+ private int bitField0_;
+ // required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;
+ public static final int NAMESPACE_DESCRIPTOR_FIELD_NUMBER = 1;
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor namespaceDescriptor_;
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public boolean hasNamespaceDescriptor() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor getNamespaceDescriptor() {
+ return namespaceDescriptor_;
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder getNamespaceDescriptorOrBuilder() {
+ return namespaceDescriptor_;
+ }
+
+ // optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;
+ public static final int UNMODIFIED_NAMESPACE_DESCRIPTOR_FIELD_NUMBER = 2;
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor unmodifiedNamespaceDescriptor_;
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ public boolean hasUnmodifiedNamespaceDescriptor() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor getUnmodifiedNamespaceDescriptor() {
+ return unmodifiedNamespaceDescriptor_;
+ }
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder getUnmodifiedNamespaceDescriptorOrBuilder() {
+ return unmodifiedNamespaceDescriptor_;
+ }
+
+ private void initFields() {
+ namespaceDescriptor_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance();
+ unmodifiedNamespaceDescriptor_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance();
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ if (!hasNamespaceDescriptor()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!getNamespaceDescriptor().isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (hasUnmodifiedNamespaceDescriptor()) {
+ if (!getUnmodifiedNamespaceDescriptor().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, namespaceDescriptor_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeMessage(2, unmodifiedNamespaceDescriptor_);
+ }
+ 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, namespaceDescriptor_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(2, unmodifiedNamespaceDescriptor_);
+ }
+ 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.ModifyNamespaceStateData)) {
+ return super.equals(obj);
+ }
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData) obj;
+
+ boolean result = true;
+ result = result && (hasNamespaceDescriptor() == other.hasNamespaceDescriptor());
+ if (hasNamespaceDescriptor()) {
+ result = result && getNamespaceDescriptor()
+ .equals(other.getNamespaceDescriptor());
+ }
+ result = result && (hasUnmodifiedNamespaceDescriptor() == other.hasUnmodifiedNamespaceDescriptor());
+ if (hasUnmodifiedNamespaceDescriptor()) {
+ result = result && getUnmodifiedNamespaceDescriptor()
+ .equals(other.getUnmodifiedNamespaceDescriptor());
+ }
+ 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 (hasNamespaceDescriptor()) {
+ hash = (37 * hash) + NAMESPACE_DESCRIPTOR_FIELD_NUMBER;
+ hash = (53 * hash) + getNamespaceDescriptor().hashCode();
+ }
+ if (hasUnmodifiedNamespaceDescriptor()) {
+ hash = (37 * hash) + UNMODIFIED_NAMESPACE_DESCRIPTOR_FIELD_NUMBER;
+ hash = (53 * hash) + getUnmodifiedNamespaceDescriptor().hashCode();
+ }
+ hash = (29 * hash) + getUnknownFields().hashCode();
+ memoizedHashCode = hash;
+ return hash;
+ }
+
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData 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.ModifyNamespaceStateData parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData 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.ModifyNamespaceStateData parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData 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.ModifyNamespaceStateData parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData 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.ModifyNamespaceStateData parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData 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.ModifyNamespaceStateData 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.ModifyNamespaceStateData}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateDataOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyNamespaceStateData_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyNamespaceStateData_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData.Builder.class);
+ }
+
+ // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ getNamespaceDescriptorFieldBuilder();
+ getUnmodifiedNamespaceDescriptorFieldBuilder();
+ }
+ }
+ private static Builder create() {
+ return new Builder();
+ }
+
+ public Builder clear() {
+ super.clear();
+ if (namespaceDescriptorBuilder_ == null) {
+ namespaceDescriptor_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance();
+ } else {
+ namespaceDescriptorBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ if (unmodifiedNamespaceDescriptorBuilder_ == null) {
+ unmodifiedNamespaceDescriptor_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance();
+ } else {
+ unmodifiedNamespaceDescriptorBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000002);
+ 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_ModifyNamespaceStateData_descriptor;
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData getDefaultInstanceForType() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData.getDefaultInstance();
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData build() {
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData buildPartial() {
+ org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ if (namespaceDescriptorBuilder_ == null) {
+ result.namespaceDescriptor_ = namespaceDescriptor_;
+ } else {
+ result.namespaceDescriptor_ = namespaceDescriptorBuilder_.build();
+ }
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ if (unmodifiedNamespaceDescriptorBuilder_ == null) {
+ result.unmodifiedNamespaceDescriptor_ = unmodifiedNamespaceDescriptor_;
+ } else {
+ result.unmodifiedNamespaceDescriptor_ = unmodifiedNamespaceDescriptorBuilder_.build();
+ }
+ 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.ModifyNamespaceStateData) {
+ return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData other) {
+ if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData.getDefaultInstance()) return this;
+ if (other.hasNamespaceDescriptor()) {
+ mergeNamespaceDescriptor(other.getNamespaceDescriptor());
+ }
+ if (other.hasUnmodifiedNamespaceDescriptor()) {
+ mergeUnmodifiedNamespaceDescriptor(other.getUnmodifiedNamespaceDescriptor());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ if (!hasNamespaceDescriptor()) {
+
+ return false;
+ }
+ if (!getNamespaceDescriptor().isInitialized()) {
+
+ return false;
+ }
+ if (hasUnmodifiedNamespaceDescriptor()) {
+ if (!getUnmodifiedNamespaceDescriptor().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.ModifyNamespaceStateData parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ // required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor namespaceDescriptor_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> namespaceDescriptorBuilder_;
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public boolean hasNamespaceDescriptor() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor getNamespaceDescriptor() {
+ if (namespaceDescriptorBuilder_ == null) {
+ return namespaceDescriptor_;
+ } else {
+ return namespaceDescriptorBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public Builder setNamespaceDescriptor(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor value) {
+ if (namespaceDescriptorBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ namespaceDescriptor_ = value;
+ onChanged();
+ } else {
+ namespaceDescriptorBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public Builder setNamespaceDescriptor(
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder builderForValue) {
+ if (namespaceDescriptorBuilder_ == null) {
+ namespaceDescriptor_ = builderForValue.build();
+ onChanged();
+ } else {
+ namespaceDescriptorBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public Builder mergeNamespaceDescriptor(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor value) {
+ if (namespaceDescriptorBuilder_ == null) {
+ if (((bitField0_ & 0x00000001) == 0x00000001) &&
+ namespaceDescriptor_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance()) {
+ namespaceDescriptor_ =
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.newBuilder(namespaceDescriptor_).mergeFrom(value).buildPartial();
+ } else {
+ namespaceDescriptor_ = value;
+ }
+ onChanged();
+ } else {
+ namespaceDescriptorBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public Builder clearNamespaceDescriptor() {
+ if (namespaceDescriptorBuilder_ == null) {
+ namespaceDescriptor_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance();
+ onChanged();
+ } else {
+ namespaceDescriptorBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ return this;
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder getNamespaceDescriptorBuilder() {
+ bitField0_ |= 0x00000001;
+ onChanged();
+ return getNamespaceDescriptorFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder getNamespaceDescriptorOrBuilder() {
+ if (namespaceDescriptorBuilder_ != null) {
+ return namespaceDescriptorBuilder_.getMessageOrBuilder();
+ } else {
+ return namespaceDescriptor_;
+ }
+ }
+ /**
+ * <code>required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder>
+ getNamespaceDescriptorFieldBuilder() {
+ if (namespaceDescriptorBuilder_ == null) {
+ namespaceDescriptorBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder>(
+ namespaceDescriptor_,
+ getParentForChildren(),
+ isClean());
+ namespaceDescriptor_ = null;
+ }
+ return namespaceDescriptorBuilder_;
+ }
+
+ // optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor unmodifiedNamespaceDescriptor_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> unmodifiedNamespaceDescriptorBuilder_;
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ public boolean hasUnmodifiedNamespaceDescriptor() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor getUnmodifiedNamespaceDescriptor() {
+ if (unmodifiedNamespaceDescriptorBuilder_ == null) {
+ return unmodifiedNamespaceDescriptor_;
+ } else {
+ return unmodifiedNamespaceDescriptorBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ public Builder setUnmodifiedNamespaceDescriptor(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor value) {
+ if (unmodifiedNamespaceDescriptorBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ unmodifiedNamespaceDescriptor_ = value;
+ onChanged();
+ } else {
+ unmodifiedNamespaceDescriptorBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ public Builder setUnmodifiedNamespaceDescriptor(
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder builderForValue) {
+ if (unmodifiedNamespaceDescriptorBuilder_ == null) {
+ unmodifiedNamespaceDescriptor_ = builderForValue.build();
+ onChanged();
+ } else {
+ unmodifiedNamespaceDescriptorBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ public Builder mergeUnmodifiedNamespaceDescriptor(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor value) {
+ if (unmodifiedNamespaceDescriptorBuilder_ == null) {
+ if (((bitField0_ & 0x00000002) == 0x00000002) &&
+ unmodifiedNamespaceDescriptor_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance()) {
+ unmodifiedNamespaceDescriptor_ =
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.newBuilder(unmodifiedNamespaceDescriptor_).mergeFrom(value).buildPartial();
+ } else {
+ unmodifiedNamespaceDescriptor_ = value;
+ }
+ onChanged();
+ } else {
+ unmodifiedNamespaceDescriptorBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ public Builder clearUnmodifiedNamespaceDescriptor() {
+ if (unmodifiedNamespaceDescriptorBuilder_ == null) {
+ unmodifiedNamespaceDescriptor_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance();
+ onChanged();
+ } else {
+ unmodifiedNamespaceDescriptorBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000002);
+ return this;
+ }
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder getUnmodifiedNamespaceDescriptorBuilder() {
+ bitField0_ |= 0x00000002;
+ onChanged();
+ return getUnmodifiedNamespaceDescriptorFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder getUnmodifiedNamespaceDescriptorOrBuilder() {
+ if (unmodifiedNamespaceDescriptorBuilder_ != null) {
+ return unmodifiedNamespaceDescriptorBuilder_.getMessageOrBuilder();
+ } else {
+ return unmodifiedNamespaceDescriptor_;
+ }
+ }
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder>
+ getUnmodifiedNamespaceDescriptorFieldBuilder() {
+ if (unmodifiedNamespaceDescriptorBuilder_ == null) {
+ unmodifiedNamespaceDescriptorBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder>(
+ unmodifiedNamespaceDescriptor_,
+ getParentForChildren(),
+ isClean());
+ unmodifiedNamespaceDescriptor_ = null;
+ }
+ return unmodifiedNamespaceDescriptorBuilder_;
+ }
+
+ // @@protoc_insertion_point(builder_scope:hbase.pb.ModifyNamespaceStateData)
+ }
+
+ static {
+ defaultInstance = new ModifyNamespaceStateData(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:hbase.pb.ModifyNamespaceStateData)
+ }
+
+ public interface DeleteNamespaceStateDataOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // required string namespace_name = 1;
+ /**
+ * <code>required string namespace_name = 1;</code>
+ */
+ boolean hasNamespaceName();
+ /**
+ * <code>required string namespace_name = 1;</code>
+ */
+ java.lang.String getNamespaceName();
+ /**
+ * <code>required string namespace_name = 1;</code>
+ */
+ com.google.protobuf.ByteString
+ getNamespaceNameBytes();
+
+ // optional .hbase.pb.NamespaceDescriptor namespace_descriptor = 2;
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor namespace_descriptor = 2;</code>
+ */
+ boolean hasNamespaceDescriptor();
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor namespace_descriptor = 2;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor getNamespaceDescriptor();
+ /**
+ * <code>optional .hbase.pb.NamespaceDescriptor namespace_descriptor = 2;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder getNamespaceDescriptorOrBuilder();
+ }
+ /**
+ * Protobuf type {@code hbase.pb.DeleteNamespaceStateData}
+ */
+ public static final class DeleteNamespaceStateData extends
+ com.google.protobuf.GeneratedMessage
+ implements DeleteNamespaceStateDataOrBuilder {
+ // Use DeleteNamespaceStateData.newBuilder() to construct.
+ private DeleteNamespaceStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private DeleteNamespaceStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final DeleteNamespaceStateData defaultInstance;
+ public static DeleteNamespaceStateData getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public DeleteNamespaceStateData getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private DeleteNamespaceStateData(
+ 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: {
+ bitField0_ |= 0x00000001;
+ namespaceName_ = input.readBytes();
+
<TRUNCATED>
[2/4] hbase git commit: HBASE-13212 Procedure V2 - master
Create/Modify/Delete namespace (Stephen Yuan Jiang)
Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/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 f1420b0..4fbded7 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
@@ -16232,6 +16232,26 @@ public final class MasterProtos {
* <code>required .hbase.pb.NamespaceDescriptor namespaceDescriptor = 1;</code>
*/
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder getNamespaceDescriptorOrBuilder();
+
+ // optional uint64 nonce_group = 2 [default = 0];
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ boolean hasNonceGroup();
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ long getNonceGroup();
+
+ // optional uint64 nonce = 3 [default = 0];
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ boolean hasNonce();
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ long getNonce();
}
/**
* Protobuf type {@code hbase.pb.CreateNamespaceRequest}
@@ -16297,6 +16317,16 @@ public final class MasterProtos {
bitField0_ |= 0x00000001;
break;
}
+ case 16: {
+ bitField0_ |= 0x00000002;
+ nonceGroup_ = input.readUInt64();
+ break;
+ }
+ case 24: {
+ bitField0_ |= 0x00000004;
+ nonce_ = input.readUInt64();
+ break;
+ }
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -16359,8 +16389,42 @@ public final class MasterProtos {
return namespaceDescriptor_;
}
+ // optional uint64 nonce_group = 2 [default = 0];
+ public static final int NONCE_GROUP_FIELD_NUMBER = 2;
+ private long nonceGroup_;
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public boolean hasNonceGroup() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public long getNonceGroup() {
+ return nonceGroup_;
+ }
+
+ // optional uint64 nonce = 3 [default = 0];
+ public static final int NONCE_FIELD_NUMBER = 3;
+ private long nonce_;
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public boolean hasNonce() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public long getNonce() {
+ return nonce_;
+ }
+
private void initFields() {
namespaceDescriptor_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance();
+ nonceGroup_ = 0L;
+ nonce_ = 0L;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -16385,6 +16449,12 @@ public final class MasterProtos {
if (((bitField0_ & 0x00000001) == 0x00000001)) {
output.writeMessage(1, namespaceDescriptor_);
}
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeUInt64(2, nonceGroup_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeUInt64(3, nonce_);
+ }
getUnknownFields().writeTo(output);
}
@@ -16398,6 +16468,14 @@ public final class MasterProtos {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(1, namespaceDescriptor_);
}
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(2, nonceGroup_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(3, nonce_);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@@ -16426,6 +16504,16 @@ public final class MasterProtos {
result = result && getNamespaceDescriptor()
.equals(other.getNamespaceDescriptor());
}
+ 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;
@@ -16443,6 +16531,14 @@ public final class MasterProtos {
hash = (37 * hash) + NAMESPACEDESCRIPTOR_FIELD_NUMBER;
hash = (53 * hash) + getNamespaceDescriptor().hashCode();
}
+ 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;
@@ -16559,6 +16655,10 @@ public final class MasterProtos {
namespaceDescriptorBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000001);
+ nonceGroup_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ nonce_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000004);
return this;
}
@@ -16595,6 +16695,14 @@ public final class MasterProtos {
} else {
result.namespaceDescriptor_ = namespaceDescriptorBuilder_.build();
}
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.nonceGroup_ = nonceGroup_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.nonce_ = nonce_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@@ -16614,6 +16722,12 @@ public final class MasterProtos {
if (other.hasNamespaceDescriptor()) {
mergeNamespaceDescriptor(other.getNamespaceDescriptor());
}
+ if (other.hasNonceGroup()) {
+ setNonceGroup(other.getNonceGroup());
+ }
+ if (other.hasNonce()) {
+ setNonce(other.getNonce());
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -16766,6 +16880,72 @@ public final class MasterProtos {
return namespaceDescriptorBuilder_;
}
+ // optional uint64 nonce_group = 2 [default = 0];
+ private long nonceGroup_ ;
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public boolean hasNonceGroup() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public long getNonceGroup() {
+ return nonceGroup_;
+ }
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public Builder setNonceGroup(long value) {
+ bitField0_ |= 0x00000002;
+ nonceGroup_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public Builder clearNonceGroup() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ nonceGroup_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // optional uint64 nonce = 3 [default = 0];
+ private long nonce_ ;
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public boolean hasNonce() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public long getNonce() {
+ return nonce_;
+ }
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public Builder setNonce(long value) {
+ bitField0_ |= 0x00000004;
+ nonce_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public Builder clearNonce() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ nonce_ = 0L;
+ onChanged();
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:hbase.pb.CreateNamespaceRequest)
}
@@ -17132,6 +17312,26 @@ public final class MasterProtos {
*/
com.google.protobuf.ByteString
getNamespaceNameBytes();
+
+ // optional uint64 nonce_group = 2 [default = 0];
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ boolean hasNonceGroup();
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ long getNonceGroup();
+
+ // optional uint64 nonce = 3 [default = 0];
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ boolean hasNonce();
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ long getNonce();
}
/**
* Protobuf type {@code hbase.pb.DeleteNamespaceRequest}
@@ -17189,6 +17389,16 @@ public final class MasterProtos {
namespaceName_ = input.readBytes();
break;
}
+ case 16: {
+ bitField0_ |= 0x00000002;
+ nonceGroup_ = input.readUInt64();
+ break;
+ }
+ case 24: {
+ bitField0_ |= 0x00000004;
+ nonce_ = input.readUInt64();
+ break;
+ }
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -17272,8 +17482,42 @@ public final class MasterProtos {
}
}
+ // optional uint64 nonce_group = 2 [default = 0];
+ public static final int NONCE_GROUP_FIELD_NUMBER = 2;
+ private long nonceGroup_;
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public boolean hasNonceGroup() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public long getNonceGroup() {
+ return nonceGroup_;
+ }
+
+ // optional uint64 nonce = 3 [default = 0];
+ public static final int NONCE_FIELD_NUMBER = 3;
+ private long nonce_;
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public boolean hasNonce() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public long getNonce() {
+ return nonce_;
+ }
+
private void initFields() {
namespaceName_ = "";
+ nonceGroup_ = 0L;
+ nonce_ = 0L;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -17294,6 +17538,12 @@ public final class MasterProtos {
if (((bitField0_ & 0x00000001) == 0x00000001)) {
output.writeBytes(1, getNamespaceNameBytes());
}
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeUInt64(2, nonceGroup_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeUInt64(3, nonce_);
+ }
getUnknownFields().writeTo(output);
}
@@ -17307,6 +17557,14 @@ public final class MasterProtos {
size += com.google.protobuf.CodedOutputStream
.computeBytesSize(1, getNamespaceNameBytes());
}
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(2, nonceGroup_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(3, nonce_);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@@ -17335,6 +17593,16 @@ public final class MasterProtos {
result = result && getNamespaceName()
.equals(other.getNamespaceName());
}
+ 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;
@@ -17352,6 +17620,14 @@ public final class MasterProtos {
hash = (37 * hash) + NAMESPACENAME_FIELD_NUMBER;
hash = (53 * hash) + getNamespaceName().hashCode();
}
+ 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;
@@ -17463,6 +17739,10 @@ public final class MasterProtos {
super.clear();
namespaceName_ = "";
bitField0_ = (bitField0_ & ~0x00000001);
+ nonceGroup_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ nonce_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000004);
return this;
}
@@ -17495,6 +17775,14 @@ public final class MasterProtos {
to_bitField0_ |= 0x00000001;
}
result.namespaceName_ = namespaceName_;
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.nonceGroup_ = nonceGroup_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.nonce_ = nonce_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@@ -17516,6 +17804,12 @@ public final class MasterProtos {
namespaceName_ = other.namespaceName_;
onChanged();
}
+ if (other.hasNonceGroup()) {
+ setNonceGroup(other.getNonceGroup());
+ }
+ if (other.hasNonce()) {
+ setNonce(other.getNonce());
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -17621,6 +17915,72 @@ public final class MasterProtos {
return this;
}
+ // optional uint64 nonce_group = 2 [default = 0];
+ private long nonceGroup_ ;
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public boolean hasNonceGroup() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public long getNonceGroup() {
+ return nonceGroup_;
+ }
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public Builder setNonceGroup(long value) {
+ bitField0_ |= 0x00000002;
+ nonceGroup_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public Builder clearNonceGroup() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ nonceGroup_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // optional uint64 nonce = 3 [default = 0];
+ private long nonce_ ;
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public boolean hasNonce() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public long getNonce() {
+ return nonce_;
+ }
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public Builder setNonce(long value) {
+ bitField0_ |= 0x00000004;
+ nonce_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public Builder clearNonce() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ nonce_ = 0L;
+ onChanged();
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:hbase.pb.DeleteNamespaceRequest)
}
@@ -17986,6 +18346,26 @@ public final class MasterProtos {
* <code>required .hbase.pb.NamespaceDescriptor namespaceDescriptor = 1;</code>
*/
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder getNamespaceDescriptorOrBuilder();
+
+ // optional uint64 nonce_group = 2 [default = 0];
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ boolean hasNonceGroup();
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ long getNonceGroup();
+
+ // optional uint64 nonce = 3 [default = 0];
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ boolean hasNonce();
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ long getNonce();
}
/**
* Protobuf type {@code hbase.pb.ModifyNamespaceRequest}
@@ -18051,6 +18431,16 @@ public final class MasterProtos {
bitField0_ |= 0x00000001;
break;
}
+ case 16: {
+ bitField0_ |= 0x00000002;
+ nonceGroup_ = input.readUInt64();
+ break;
+ }
+ case 24: {
+ bitField0_ |= 0x00000004;
+ nonce_ = input.readUInt64();
+ break;
+ }
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -18113,8 +18503,42 @@ public final class MasterProtos {
return namespaceDescriptor_;
}
+ // optional uint64 nonce_group = 2 [default = 0];
+ public static final int NONCE_GROUP_FIELD_NUMBER = 2;
+ private long nonceGroup_;
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public boolean hasNonceGroup() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public long getNonceGroup() {
+ return nonceGroup_;
+ }
+
+ // optional uint64 nonce = 3 [default = 0];
+ public static final int NONCE_FIELD_NUMBER = 3;
+ private long nonce_;
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public boolean hasNonce() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public long getNonce() {
+ return nonce_;
+ }
+
private void initFields() {
namespaceDescriptor_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NamespaceDescriptor.getDefaultInstance();
+ nonceGroup_ = 0L;
+ nonce_ = 0L;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -18139,6 +18563,12 @@ public final class MasterProtos {
if (((bitField0_ & 0x00000001) == 0x00000001)) {
output.writeMessage(1, namespaceDescriptor_);
}
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeUInt64(2, nonceGroup_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeUInt64(3, nonce_);
+ }
getUnknownFields().writeTo(output);
}
@@ -18152,6 +18582,14 @@ public final class MasterProtos {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(1, namespaceDescriptor_);
}
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(2, nonceGroup_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(3, nonce_);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@@ -18180,6 +18618,16 @@ public final class MasterProtos {
result = result && getNamespaceDescriptor()
.equals(other.getNamespaceDescriptor());
}
+ 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;
@@ -18197,6 +18645,14 @@ public final class MasterProtos {
hash = (37 * hash) + NAMESPACEDESCRIPTOR_FIELD_NUMBER;
hash = (53 * hash) + getNamespaceDescriptor().hashCode();
}
+ 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;
@@ -18313,6 +18769,10 @@ public final class MasterProtos {
namespaceDescriptorBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000001);
+ nonceGroup_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ nonce_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000004);
return this;
}
@@ -18349,6 +18809,14 @@ public final class MasterProtos {
} else {
result.namespaceDescriptor_ = namespaceDescriptorBuilder_.build();
}
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.nonceGroup_ = nonceGroup_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.nonce_ = nonce_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@@ -18368,6 +18836,12 @@ public final class MasterProtos {
if (other.hasNamespaceDescriptor()) {
mergeNamespaceDescriptor(other.getNamespaceDescriptor());
}
+ if (other.hasNonceGroup()) {
+ setNonceGroup(other.getNonceGroup());
+ }
+ if (other.hasNonce()) {
+ setNonce(other.getNonce());
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -18520,6 +18994,72 @@ public final class MasterProtos {
return namespaceDescriptorBuilder_;
}
+ // optional uint64 nonce_group = 2 [default = 0];
+ private long nonceGroup_ ;
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public boolean hasNonceGroup() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public long getNonceGroup() {
+ return nonceGroup_;
+ }
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public Builder setNonceGroup(long value) {
+ bitField0_ |= 0x00000002;
+ nonceGroup_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional uint64 nonce_group = 2 [default = 0];</code>
+ */
+ public Builder clearNonceGroup() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ nonceGroup_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // optional uint64 nonce = 3 [default = 0];
+ private long nonce_ ;
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public boolean hasNonce() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public long getNonce() {
+ return nonce_;
+ }
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public Builder setNonce(long value) {
+ bitField0_ |= 0x00000004;
+ nonce_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional uint64 nonce = 3 [default = 0];</code>
+ */
+ public Builder clearNonce() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ nonce_ = 0L;
+ onChanged();
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:hbase.pb.ModifyNamespaceRequest)
}
@@ -54638,227 +55178,230 @@ public final class MasterProtos {
"name\030\001 \002(\0132\023.hbase.pb.TableName\022+\n\014table" +
"_schema\030\002 \002(\0132\025.hbase.pb.TableSchema\022\026\n\013" +
"nonce_group\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"\025" +
- "\n\023ModifyTableResponse\"T\n\026CreateNamespace" +
+ "\n\023ModifyTableResponse\"~\n\026CreateNamespace" +
"Request\022:\n\023namespaceDescriptor\030\001 \002(\0132\035.h" +
- "base.pb.NamespaceDescriptor\"\031\n\027CreateNam" +
- "espaceResponse\"/\n\026DeleteNamespaceRequest" +
- "\022\025\n\rnamespaceName\030\001 \002(\t\"\031\n\027DeleteNamespa" +
- "ceResponse\"T\n\026ModifyNamespaceRequest\022:\n\023",
- "namespaceDescriptor\030\001 \002(\0132\035.hbase.pb.Nam" +
- "espaceDescriptor\"\031\n\027ModifyNamespaceRespo" +
- "nse\"6\n\035GetNamespaceDescriptorRequest\022\025\n\r" +
- "namespaceName\030\001 \002(\t\"\\\n\036GetNamespaceDescr" +
- "iptorResponse\022:\n\023namespaceDescriptor\030\001 \002" +
- "(\0132\035.hbase.pb.NamespaceDescriptor\"!\n\037Lis" +
- "tNamespaceDescriptorsRequest\"^\n ListName" +
- "spaceDescriptorsResponse\022:\n\023namespaceDes" +
- "criptor\030\001 \003(\0132\035.hbase.pb.NamespaceDescri" +
- "ptor\"?\n&ListTableDescriptorsByNamespaceR",
- "equest\022\025\n\rnamespaceName\030\001 \002(\t\"U\n\'ListTab" +
- "leDescriptorsByNamespaceResponse\022*\n\013tabl" +
- "eSchema\030\001 \003(\0132\025.hbase.pb.TableSchema\"9\n " +
- "ListTableNamesByNamespaceRequest\022\025\n\rname" +
- "spaceName\030\001 \002(\t\"K\n!ListTableNamesByNames" +
- "paceResponse\022&\n\ttableName\030\001 \003(\0132\023.hbase." +
- "pb.TableName\"\021\n\017ShutdownRequest\"\022\n\020Shutd" +
- "ownResponse\"\023\n\021StopMasterRequest\"\024\n\022Stop" +
- "MasterResponse\"\020\n\016BalanceRequest\"\'\n\017Bala" +
- "nceResponse\022\024\n\014balancer_ran\030\001 \002(\010\"<\n\031Set",
- "BalancerRunningRequest\022\n\n\002on\030\001 \002(\010\022\023\n\013sy" +
- "nchronous\030\002 \001(\010\"8\n\032SetBalancerRunningRes" +
- "ponse\022\032\n\022prev_balance_value\030\001 \001(\010\"\032\n\030IsB" +
- "alancerEnabledRequest\",\n\031IsBalancerEnabl" +
- "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" +
- "eSnapshotRequest\022/\n\010snapshot\030\001 \002(\0132\035.hba" +
- "se.pb.SnapshotDescription\"\030\n\026DeleteSnaps" +
- "hotResponse\"I\n\026RestoreSnapshotRequest\022/\n",
- "\010snapshot\030\001 \002(\0132\035.hbase.pb.SnapshotDescr" +
- "iption\"\031\n\027RestoreSnapshotResponse\"H\n\025IsS" +
- "napshotDoneRequest\022/\n\010snapshot\030\001 \001(\0132\035.h" +
- "base.pb.SnapshotDescription\"^\n\026IsSnapsho" +
- "tDoneResponse\022\023\n\004done\030\001 \001(\010:\005false\022/\n\010sn" +
- "apshot\030\002 \001(\0132\035.hbase.pb.SnapshotDescript" +
- "ion\"O\n\034IsRestoreSnapshotDoneRequest\022/\n\010s" +
- "napshot\030\001 \001(\0132\035.hbase.pb.SnapshotDescrip" +
- "tion\"4\n\035IsRestoreSnapshotDoneResponse\022\023\n" +
- "\004done\030\001 \001(\010:\005false\"F\n\033GetSchemaAlterStat",
- "usRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb" +
- ".TableName\"T\n\034GetSchemaAlterStatusRespon" +
- "se\022\035\n\025yet_to_update_regions\030\001 \001(\r\022\025\n\rtot" +
- "al_regions\030\002 \001(\r\"\213\001\n\032GetTableDescriptors" +
- "Request\022(\n\013table_names\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\033GetTableDescriptorsResponse\022+\n\014table_s" +
- "chema\030\001 \003(\0132\025.hbase.pb.TableSchema\"[\n\024Ge" +
- "tTableNamesRequest\022\r\n\005regex\030\001 \001(\t\022!\n\022inc",
- "lude_sys_tables\030\002 \001(\010:\005false\022\021\n\tnamespac" +
- "e\030\003 \001(\t\"A\n\025GetTableNamesResponse\022(\n\013tabl" +
- "e_names\030\001 \003(\0132\023.hbase.pb.TableName\"\031\n\027Ge" +
- "tClusterStatusRequest\"K\n\030GetClusterStatu" +
- "sResponse\022/\n\016cluster_status\030\001 \002(\0132\027.hbas" +
- "e.pb.ClusterStatus\"\030\n\026IsMasterRunningReq" +
- "uest\"4\n\027IsMasterRunningResponse\022\031\n\021is_ma" +
- "ster_running\030\001 \002(\010\"I\n\024ExecProcedureReque" +
- "st\0221\n\tprocedure\030\001 \002(\0132\036.hbase.pb.Procedu" +
- "reDescription\"F\n\025ExecProcedureResponse\022\030",
- "\n\020expected_timeout\030\001 \001(\003\022\023\n\013return_data\030" +
- "\002 \001(\014\"K\n\026IsProcedureDoneRequest\0221\n\tproce" +
- "dure\030\001 \001(\0132\036.hbase.pb.ProcedureDescripti" +
- "on\"`\n\027IsProcedureDoneResponse\022\023\n\004done\030\001 " +
- "\001(\010:\005false\0220\n\010snapshot\030\002 \001(\0132\036.hbase.pb." +
- "ProcedureDescription\",\n\031GetProcedureResu" +
- "ltRequest\022\017\n\007proc_id\030\001 \002(\004\"\371\001\n\032GetProced" +
- "ureResultResponse\0229\n\005state\030\001 \002(\0162*.hbase" +
- ".pb.GetProcedureResultResponse.State\022\022\n\n" +
- "start_time\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.ForeignExceptionMessage\"1\n\005State\022\r\n\t" +
- "NOT_FOUND\020\000\022\013\n\007RUNNING\020\001\022\014\n\010FINISHED\020\002\"\315" +
- "\001\n\017SetQuotaRequest\022\021\n\tuser_name\030\001 \001(\t\022\022\n" +
- "\nuser_group\030\002 \001(\t\022\021\n\tnamespace\030\003 \001(\t\022\'\n\n" +
- "table_name\030\004 \001(\0132\023.hbase.pb.TableName\022\022\n" +
- "\nremove_all\030\005 \001(\010\022\026\n\016bypass_globals\030\006 \001(" +
- "\010\022+\n\010throttle\030\007 \001(\0132\031.hbase.pb.ThrottleR" +
- "equest\"\022\n\020SetQuotaResponse\"J\n\037MajorCompa" +
- "ctionTimestampRequest\022\'\n\ntable_name\030\001 \002(",
- "\0132\023.hbase.pb.TableName\"U\n(MajorCompactio" +
- "nTimestampForRegionRequest\022)\n\006region\030\001 \002" +
- "(\0132\031.hbase.pb.RegionSpecifier\"@\n MajorCo" +
- "mpactionTimestampResponse\022\034\n\024compaction_" +
- "timestamp\030\001 \002(\003\"\035\n\033SecurityCapabilitiesR" +
- "equest\"\354\001\n\034SecurityCapabilitiesResponse\022" +
- "G\n\014capabilities\030\001 \003(\01621.hbase.pb.Securit" +
- "yCapabilitiesResponse.Capability\"\202\001\n\nCap" +
- "ability\022\031\n\025SIMPLE_AUTHENTICATION\020\000\022\031\n\025SE" +
- "CURE_AUTHENTICATION\020\001\022\021\n\rAUTHORIZATION\020\002",
- "\022\026\n\022CELL_AUTHORIZATION\020\003\022\023\n\017CELL_VISIBIL" +
- "ITY\020\0042\357\"\n\rMasterService\022e\n\024GetSchemaAlte" +
- "rStatus\022%.hbase.pb.GetSchemaAlterStatusR" +
- "equest\032&.hbase.pb.GetSchemaAlterStatusRe" +
- "sponse\022b\n\023GetTableDescriptors\022$.hbase.pb" +
- ".GetTableDescriptorsRequest\032%.hbase.pb.G" +
- "etTableDescriptorsResponse\022P\n\rGetTableNa" +
- "mes\022\036.hbase.pb.GetTableNamesRequest\032\037.hb" +
- "ase.pb.GetTableNamesResponse\022Y\n\020GetClust" +
- "erStatus\022!.hbase.pb.GetClusterStatusRequ",
- "est\032\".hbase.pb.GetClusterStatusResponse\022" +
- "V\n\017IsMasterRunning\022 .hbase.pb.IsMasterRu" +
- "nningRequest\032!.hbase.pb.IsMasterRunningR" +
- "esponse\022D\n\tAddColumn\022\032.hbase.pb.AddColum" +
- "nRequest\032\033.hbase.pb.AddColumnResponse\022M\n" +
- "\014DeleteColumn\022\035.hbase.pb.DeleteColumnReq" +
- "uest\032\036.hbase.pb.DeleteColumnResponse\022M\n\014" +
- "ModifyColumn\022\035.hbase.pb.ModifyColumnRequ" +
- "est\032\036.hbase.pb.ModifyColumnResponse\022G\n\nM" +
- "oveRegion\022\033.hbase.pb.MoveRegionRequest\032\034",
- ".hbase.pb.MoveRegionResponse\022k\n\026Dispatch" +
- "MergingRegions\022\'.hbase.pb.DispatchMergin" +
- "gRegionsRequest\032(.hbase.pb.DispatchMergi" +
- "ngRegionsResponse\022M\n\014AssignRegion\022\035.hbas" +
- "e.pb.AssignRegionRequest\032\036.hbase.pb.Assi" +
- "gnRegionResponse\022S\n\016UnassignRegion\022\037.hba" +
- "se.pb.UnassignRegionRequest\032 .hbase.pb.U" +
- "nassignRegionResponse\022P\n\rOfflineRegion\022\036" +
- ".hbase.pb.OfflineRegionRequest\032\037.hbase.p" +
- "b.OfflineRegionResponse\022J\n\013DeleteTable\022\034",
- ".hbase.pb.DeleteTableRequest\032\035.hbase.pb." +
- "DeleteTableResponse\022P\n\rtruncateTable\022\036.h" +
- "base.pb.TruncateTableRequest\032\037.hbase.pb." +
- "TruncateTableResponse\022J\n\013EnableTable\022\034.h" +
- "base.pb.EnableTableRequest\032\035.hbase.pb.En" +
- "ableTableResponse\022M\n\014DisableTable\022\035.hbas" +
- "e.pb.DisableTableRequest\032\036.hbase.pb.Disa" +
- "bleTableResponse\022J\n\013ModifyTable\022\034.hbase." +
- "pb.ModifyTableRequest\032\035.hbase.pb.ModifyT" +
- "ableResponse\022J\n\013CreateTable\022\034.hbase.pb.C",
- "reateTableRequest\032\035.hbase.pb.CreateTable" +
- "Response\022A\n\010Shutdown\022\031.hbase.pb.Shutdown" +
- "Request\032\032.hbase.pb.ShutdownResponse\022G\n\nS" +
- "topMaster\022\033.hbase.pb.StopMasterRequest\032\034" +
- ".hbase.pb.StopMasterResponse\022>\n\007Balance\022" +
- "\030.hbase.pb.BalanceRequest\032\031.hbase.pb.Bal" +
- "anceResponse\022_\n\022SetBalancerRunning\022#.hba" +
- "se.pb.SetBalancerRunningRequest\032$.hbase." +
- "pb.SetBalancerRunningResponse\022\\\n\021IsBalan" +
- "cerEnabled\022\".hbase.pb.IsBalancerEnabledR",
- "equest\032#.hbase.pb.IsBalancerEnabledRespo" +
- "nse\022S\n\016RunCatalogScan\022\037.hbase.pb.RunCata" +
- "logScanRequest\032 .hbase.pb.RunCatalogScan" +
- "Response\022e\n\024EnableCatalogJanitor\022%.hbase" +
- ".pb.EnableCatalogJanitorRequest\032&.hbase." +
- "pb.EnableCatalogJanitorResponse\022n\n\027IsCat" +
- "alogJanitorEnabled\022(.hbase.pb.IsCatalogJ" +
- "anitorEnabledRequest\032).hbase.pb.IsCatalo" +
- "gJanitorEnabledResponse\022^\n\021ExecMasterSer" +
- "vice\022#.hbase.pb.CoprocessorServiceReques",
- "t\032$.hbase.pb.CoprocessorServiceResponse\022" +
- "A\n\010Snapshot\022\031.hbase.pb.SnapshotRequest\032\032" +
- ".hbase.pb.SnapshotResponse\022h\n\025GetComplet" +
- "edSnapshots\022&.hbase.pb.GetCompletedSnaps" +
- "hotsRequest\032\'.hbase.pb.GetCompletedSnaps" +
- "hotsResponse\022S\n\016DeleteSnapshot\022\037.hbase.p" +
- "b.DeleteSnapshotRequest\032 .hbase.pb.Delet" +
- "eSnapshotResponse\022S\n\016IsSnapshotDone\022\037.hb" +
- "ase.pb.IsSnapshotDoneRequest\032 .hbase.pb." +
- "IsSnapshotDoneResponse\022V\n\017RestoreSnapsho",
- "t\022 .hbase.pb.RestoreSnapshotRequest\032!.hb" +
- "ase.pb.RestoreSnapshotResponse\022h\n\025IsRest" +
- "oreSnapshotDone\022&.hbase.pb.IsRestoreSnap" +
- "shotDoneRequest\032\'.hbase.pb.IsRestoreSnap" +
- "shotDoneResponse\022P\n\rExecProcedure\022\036.hbas" +
- "e.pb.ExecProcedureRequest\032\037.hbase.pb.Exe" +
- "cProcedureResponse\022W\n\024ExecProcedureWithR" +
- "et\022\036.hbase.pb.ExecProcedureRequest\032\037.hba" +
- "se.pb.ExecProcedureResponse\022V\n\017IsProcedu" +
- "reDone\022 .hbase.pb.IsProcedureDoneRequest",
- "\032!.hbase.pb.IsProcedureDoneResponse\022V\n\017M" +
- "odifyNamespace\022 .hbase.pb.ModifyNamespac" +
- "eRequest\032!.hbase.pb.ModifyNamespaceRespo" +
- "nse\022V\n\017CreateNamespace\022 .hbase.pb.Create" +
- "NamespaceRequest\032!.hbase.pb.CreateNamesp" +
- "aceResponse\022V\n\017DeleteNamespace\022 .hbase.p" +
- "b.DeleteNamespaceRequest\032!.hbase.pb.Dele" +
- "teNamespaceResponse\022k\n\026GetNamespaceDescr" +
- "iptor\022\'.hbase.pb.GetNamespaceDescriptorR" +
- "equest\032(.hbase.pb.GetNamespaceDescriptor",
- "Response\022q\n\030ListNamespaceDescriptors\022).h" +
- "base.pb.ListNamespaceDescriptorsRequest\032" +
- "*.hbase.pb.ListNamespaceDescriptorsRespo" +
- "nse\022\206\001\n\037ListTableDescriptorsByNamespace\022" +
- "0.hbase.pb.ListTableDescriptorsByNamespa" +
- "ceRequest\0321.hbase.pb.ListTableDescriptor" +
- "sByNamespaceResponse\022t\n\031ListTableNamesBy" +
- "Namespace\022*.hbase.pb.ListTableNamesByNam" +
- "espaceRequest\032+.hbase.pb.ListTableNamesB" +
- "yNamespaceResponse\022A\n\010SetQuota\022\031.hbase.p",
- "b.SetQuotaRequest\032\032.hbase.pb.SetQuotaRes" +
- "ponse\022x\n\037getLastMajorCompactionTimestamp" +
- "\022).hbase.pb.MajorCompactionTimestampRequ" +
- "est\032*.hbase.pb.MajorCompactionTimestampR" +
- "esponse\022\212\001\n(getLastMajorCompactionTimest" +
- "ampForRegion\0222.hbase.pb.MajorCompactionT" +
- "imestampForRegionRequest\032*.hbase.pb.Majo" +
- "rCompactionTimestampResponse\022_\n\022getProce" +
- "dureResult\022#.hbase.pb.GetProcedureResult" +
- "Request\032$.hbase.pb.GetProcedureResultRes",
- "ponse\022h\n\027getSecurityCapabilities\022%.hbase" +
- ".pb.SecurityCapabilitiesRequest\032&.hbase." +
- "pb.SecurityCapabilitiesResponseBB\n*org.a" +
- "pache.hadoop.hbase.protobuf.generatedB\014M" +
- "asterProtosH\001\210\001\001\240\001\001"
+ "base.pb.NamespaceDescriptor\022\026\n\013nonce_gro" +
+ "up\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"\031\n\027CreateN" +
+ "amespaceResponse\"Y\n\026DeleteNamespaceReque" +
+ "st\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\"\031\n\027DeleteNam" +
+ "espaceResponse\"~\n\026ModifyNamespaceRequest" +
+ "\022:\n\023namespaceDescriptor\030\001 \002(\0132\035.hbase.pb" +
+ ".NamespaceDescriptor\022\026\n\013nonce_group\030\002 \001(" +
+ "\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"\031\n\027ModifyNamespac" +
+ "eResponse\"6\n\035GetNamespaceDescriptorReque" +
+ "st\022\025\n\rnamespaceName\030\001 \002(\t\"\\\n\036GetNamespac" +
+ "eDescriptorResponse\022:\n\023namespaceDescript" +
+ "or\030\001 \002(\0132\035.hbase.pb.NamespaceDescriptor\"" +
+ "!\n\037ListNamespaceDescriptorsRequest\"^\n Li",
+ "stNamespaceDescriptorsResponse\022:\n\023namesp" +
+ "aceDescriptor\030\001 \003(\0132\035.hbase.pb.Namespace" +
+ "Descriptor\"?\n&ListTableDescriptorsByName" +
+ "spaceRequest\022\025\n\rnamespaceName\030\001 \002(\t\"U\n\'L" +
+ "istTableDescriptorsByNamespaceResponse\022*" +
+ "\n\013tableSchema\030\001 \003(\0132\025.hbase.pb.TableSche" +
+ "ma\"9\n ListTableNamesByNamespaceRequest\022\025" +
+ "\n\rnamespaceName\030\001 \002(\t\"K\n!ListTableNamesB" +
+ "yNamespaceResponse\022&\n\ttableName\030\001 \003(\0132\023." +
+ "hbase.pb.TableName\"\021\n\017ShutdownRequest\"\022\n",
+ "\020ShutdownResponse\"\023\n\021StopMasterRequest\"\024" +
+ "\n\022StopMasterResponse\"\020\n\016BalanceRequest\"\'" +
+ "\n\017BalanceResponse\022\024\n\014balancer_ran\030\001 \002(\010\"" +
+ "<\n\031SetBalancerRunningRequest\022\n\n\002on\030\001 \002(\010" +
+ "\022\023\n\013synchronous\030\002 \001(\010\"8\n\032SetBalancerRunn" +
+ "ingResponse\022\032\n\022prev_balance_value\030\001 \001(\010\"" +
+ "\032\n\030IsBalancerEnabledRequest\",\n\031IsBalance" +
+ "rEnabledResponse\022\017\n\007enabled\030\001 \002(\010\"\027\n\025Run" +
+ "CatalogScanRequest\"-\n\026RunCatalogScanResp" +
+ "onse\022\023\n\013scan_result\030\001 \001(\005\"-\n\033EnableCatal",
+ "ogJanitorRequest\022\016\n\006enable\030\001 \002(\010\"2\n\034Enab" +
+ "leCatalogJanitorResponse\022\022\n\nprev_value\030\001" +
+ " \001(\010\" \n\036IsCatalogJanitorEnabledRequest\"0" +
+ "\n\037IsCatalogJanitorEnabledResponse\022\r\n\005val" +
+ "ue\030\001 \002(\010\"B\n\017SnapshotRequest\022/\n\010snapshot\030" +
+ "\001 \002(\0132\035.hbase.pb.SnapshotDescription\",\n\020" +
+ "SnapshotResponse\022\030\n\020expected_timeout\030\001 \002" +
+ "(\003\"\036\n\034GetCompletedSnapshotsRequest\"Q\n\035Ge" +
+ "tCompletedSnapshotsResponse\0220\n\tsnapshots" +
+ "\030\001 \003(\0132\035.hbase.pb.SnapshotDescription\"H\n",
+ "\025DeleteSnapshotRequest\022/\n\010snapshot\030\001 \002(\013" +
+ "2\035.hbase.pb.SnapshotDescription\"\030\n\026Delet" +
+ "eSnapshotResponse\"I\n\026RestoreSnapshotRequ" +
+ "est\022/\n\010snapshot\030\001 \002(\0132\035.hbase.pb.Snapsho" +
+ "tDescription\"\031\n\027RestoreSnapshotResponse\"" +
+ "H\n\025IsSnapshotDoneRequest\022/\n\010snapshot\030\001 \001" +
+ "(\0132\035.hbase.pb.SnapshotDescription\"^\n\026IsS" +
+ "napshotDoneResponse\022\023\n\004done\030\001 \001(\010:\005false" +
+ "\022/\n\010snapshot\030\002 \001(\0132\035.hbase.pb.SnapshotDe" +
+ "scription\"O\n\034IsRestoreSnapshotDoneReques",
+ "t\022/\n\010snapshot\030\001 \001(\0132\035.hbase.pb.SnapshotD" +
+ "escription\"4\n\035IsRestoreSnapshotDoneRespo" +
+ "nse\022\023\n\004done\030\001 \001(\010:\005false\"F\n\033GetSchemaAlt" +
+ "erStatusRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hb" +
+ "ase.pb.TableName\"T\n\034GetSchemaAlterStatus" +
+ "Response\022\035\n\025yet_to_update_regions\030\001 \001(\r\022" +
+ "\025\n\rtotal_regions\030\002 \001(\r\"\213\001\n\032GetTableDescr" +
+ "iptorsRequest\022(\n\013table_names\030\001 \003(\0132\023.hba" +
+ "se.pb.TableName\022\r\n\005regex\030\002 \001(\t\022!\n\022includ" +
+ "e_sys_tables\030\003 \001(\010:\005false\022\021\n\tnamespace\030\004",
+ " \001(\t\"J\n\033GetTableDescriptorsResponse\022+\n\014t" +
+ "able_schema\030\001 \003(\0132\025.hbase.pb.TableSchema" +
+ "\"[\n\024GetTableNamesRequest\022\r\n\005regex\030\001 \001(\t\022" +
+ "!\n\022include_sys_tables\030\002 \001(\010:\005false\022\021\n\tna" +
+ "mespace\030\003 \001(\t\"A\n\025GetTableNamesResponse\022(" +
+ "\n\013table_names\030\001 \003(\0132\023.hbase.pb.TableName" +
+ "\"\031\n\027GetClusterStatusRequest\"K\n\030GetCluste" +
+ "rStatusResponse\022/\n\016cluster_status\030\001 \002(\0132" +
+ "\027.hbase.pb.ClusterStatus\"\030\n\026IsMasterRunn" +
+ "ingRequest\"4\n\027IsMasterRunningResponse\022\031\n",
+ "\021is_master_running\030\001 \002(\010\"I\n\024ExecProcedur" +
+ "eRequest\0221\n\tprocedure\030\001 \002(\0132\036.hbase.pb.P" +
+ "rocedureDescription\"F\n\025ExecProcedureResp" +
+ "onse\022\030\n\020expected_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.ProcedureDes" +
+ "cription\"`\n\027IsProcedureDoneResponse\022\023\n\004d" +
+ "one\030\001 \001(\010:\005false\0220\n\010snapshot\030\002 \001(\0132\036.hba" +
+ "se.pb.ProcedureDescription\",\n\031GetProcedu" +
+ "reResultRequest\022\017\n\007proc_id\030\001 \002(\004\"\371\001\n\032Get",
+ "ProcedureResultResponse\0229\n\005state\030\001 \002(\0162*" +
+ ".hbase.pb.GetProcedureResultResponse.Sta" +
+ "te\022\022\n\nstart_time\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.ForeignExceptionMessage\"1\n\005Sta" +
+ "te\022\r\n\tNOT_FOUND\020\000\022\013\n\007RUNNING\020\001\022\014\n\010FINISH" +
+ "ED\020\002\"\315\001\n\017SetQuotaRequest\022\021\n\tuser_name\030\001 " +
+ "\001(\t\022\022\n\nuser_group\030\002 \001(\t\022\021\n\tnamespace\030\003 \001" +
+ "(\t\022\'\n\ntable_name\030\004 \001(\0132\023.hbase.pb.TableN" +
+ "ame\022\022\n\nremove_all\030\005 \001(\010\022\026\n\016bypass_global",
+ "s\030\006 \001(\010\022+\n\010throttle\030\007 \001(\0132\031.hbase.pb.Thr" +
+ "ottleRequest\"\022\n\020SetQuotaResponse\"J\n\037Majo" +
+ "rCompactionTimestampRequest\022\'\n\ntable_nam" +
+ "e\030\001 \002(\0132\023.hbase.pb.TableName\"U\n(MajorCom" +
+ "pactionTimestampForRegionRequest\022)\n\006regi" +
+ "on\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\"@\n M" +
+ "ajorCompactionTimestampResponse\022\034\n\024compa" +
+ "ction_timestamp\030\001 \002(\003\"\035\n\033SecurityCapabil" +
+ "itiesRequest\"\354\001\n\034SecurityCapabilitiesRes" +
+ "ponse\022G\n\014capabilities\030\001 \003(\01621.hbase.pb.S",
+ "ecurityCapabilitiesResponse.Capability\"\202" +
+ "\001\n\nCapability\022\031\n\025SIMPLE_AUTHENTICATION\020\000" +
+ "\022\031\n\025SECURE_AUTHENTICATION\020\001\022\021\n\rAUTHORIZA" +
+ "TION\020\002\022\026\n\022CELL_AUTHORIZATION\020\003\022\023\n\017CELL_V" +
+ "ISIBILITY\020\0042\357\"\n\rMasterService\022e\n\024GetSche" +
+ "maAlterStatus\022%.hbase.pb.GetSchemaAlterS" +
+ "tatusRequest\032&.hbase.pb.GetSchemaAlterSt" +
+ "atusResponse\022b\n\023GetTableDescriptors\022$.hb" +
+ "ase.pb.GetTableDescriptorsRequest\032%.hbas" +
+ "e.pb.GetTableDescriptorsResponse\022P\n\rGetT",
+ "ableNames\022\036.hbase.pb.GetTableNamesReques" +
+ "t\032\037.hbase.pb.GetTableNamesResponse\022Y\n\020Ge" +
+ "tClusterStatus\022!.hbase.pb.GetClusterStat" +
+ "usRequest\032\".hbase.pb.GetClusterStatusRes" +
+ "ponse\022V\n\017IsMasterRunning\022 .hbase.pb.IsMa" +
+ "sterRunningRequest\032!.hbase.pb.IsMasterRu" +
+ "nningResponse\022D\n\tAddColumn\022\032.hbase.pb.Ad" +
+ "dColumnRequest\032\033.hbase.pb.AddColumnRespo" +
+ "nse\022M\n\014DeleteColumn\022\035.hbase.pb.DeleteCol" +
+ "umnRequest\032\036.hbase.pb.DeleteColumnRespon",
+ "se\022M\n\014ModifyColumn\022\035.hbase.pb.ModifyColu" +
+ "mnRequest\032\036.hbase.pb.ModifyColumnRespons" +
+ "e\022G\n\nMoveRegion\022\033.hbase.pb.MoveRegionReq" +
+ "uest\032\034.hbase.pb.MoveRegionResponse\022k\n\026Di" +
+ "spatchMergingRegions\022\'.hbase.pb.Dispatch" +
+ "MergingRegionsRequest\032(.hbase.pb.Dispatc" +
+ "hMergingRegionsResponse\022M\n\014AssignRegion\022" +
+ "\035.hbase.pb.AssignRegionRequest\032\036.hbase.p" +
+ "b.AssignRegionResponse\022S\n\016UnassignRegion" +
+ "\022\037.hbase.pb.UnassignRegionRequest\032 .hbas",
+ "e.pb.UnassignRegionResponse\022P\n\rOfflineRe" +
+ "gion\022\036.hbase.pb.OfflineRegionRequest\032\037.h" +
+ "base.pb.OfflineRegionResponse\022J\n\013DeleteT" +
+ "able\022\034.hbase.pb.DeleteTableRequest\032\035.hba" +
+ "se.pb.DeleteTableResponse\022P\n\rtruncateTab" +
+ "le\022\036.hbase.pb.TruncateTableRequest\032\037.hba" +
+ "se.pb.TruncateTableResponse\022J\n\013EnableTab" +
+ "le\022\034.hbase.pb.EnableTableRequest\032\035.hbase" +
+ ".pb.EnableTableResponse\022M\n\014DisableTable\022" +
+ "\035.hbase.pb.DisableTableRequest\032\036.hbase.p",
+ "b.DisableTableResponse\022J\n\013ModifyTable\022\034." +
+ "hbase.pb.ModifyTableRequest\032\035.hbase.pb.M" +
+ "odifyTableResponse\022J\n\013CreateTable\022\034.hbas" +
+ "e.pb.CreateTableRequest\032\035.hbase.pb.Creat" +
+ "eTableResponse\022A\n\010Shutdown\022\031.hbase.pb.Sh" +
+ "utdownRequest\032\032.hbase.pb.ShutdownRespons" +
+ "e\022G\n\nStopMaster\022\033.hbase.pb.StopMasterReq" +
+ "uest\032\034.hbase.pb.StopMasterResponse\022>\n\007Ba" +
+ "lance\022\030.hbase.pb.BalanceRequest\032\031.hbase." +
+ "pb.BalanceResponse\022_\n\022SetBalancerRunning",
+ "\022#.hbase.pb.SetBalancerRunningRequest\032$." +
+ "hbase.pb.SetBalancerRunningResponse\022\\\n\021I" +
+ "sBalancerEnabled\022\".hbase.pb.IsBalancerEn" +
+ "abledRequest\032#.hbase.pb.IsBalancerEnable" +
+ "dResponse\022S\n\016RunCatalogScan\022\037.hbase.pb.R" +
+ "unCatalogScanRequest\032 .hbase.pb.RunCatal" +
+ "ogScanResponse\022e\n\024EnableCatalogJanitor\022%" +
+ ".hbase.pb.EnableCatalogJanitorRequest\032&." +
+ "hbase.pb.EnableCatalogJanitorResponse\022n\n" +
+ "\027IsCatalogJanitorEnabled\022(.hbase.pb.IsCa",
+ "talogJanitorEnabledRequest\032).hbase.pb.Is" +
+ "CatalogJanitorEnabledResponse\022^\n\021ExecMas" +
+ "terService\022#.hbase.pb.CoprocessorService" +
+ "Request\032$.hbase.pb.CoprocessorServiceRes" +
+ "ponse\022A\n\010Snapshot\022\031.hbase.pb.SnapshotReq" +
+ "uest\032\032.hbase.pb.SnapshotResponse\022h\n\025GetC" +
+ "ompletedSnapshots\022&.hbase.pb.GetComplete" +
+ "dSnapshotsRequest\032\'.hbase.pb.GetComplete" +
+ "dSnapshotsResponse\022S\n\016DeleteSnapshot\022\037.h" +
+ "base.pb.DeleteSnapshotRequest\032 .hbase.pb",
+ ".DeleteSnapshotResponse\022S\n\016IsSnapshotDon" +
+ "e\022\037.hbase.pb.IsSnapshotDoneRequest\032 .hba" +
+ "se.pb.IsSnapshotDoneResponse\022V\n\017RestoreS" +
+ "napshot\022 .hbase.pb.RestoreSnapshotReques" +
+ "t\032!.hbase.pb.RestoreSnapshotResponse\022h\n\025" +
+ "IsRestoreSnapshotDone\022&.hbase.pb.IsResto" +
+ "reSnapshotDoneRequest\032\'.hbase.pb.IsResto" +
+ "reSnapshotDoneResponse\022P\n\rExecProcedure\022" +
+ "\036.hbase.pb.ExecProcedureRequest\032\037.hbase." +
+ "pb.ExecProcedureResponse\022W\n\024ExecProcedur",
+ "eWithRet\022\036.hbase.pb.ExecProcedureRequest" +
+ "\032\037.hbase.pb.ExecProcedureResponse\022V\n\017IsP" +
+ "rocedureDone\022 .hbase.pb.IsProcedureDoneR" +
+ "equest\032!.hbase.pb.IsProcedureDoneRespons" +
+ "e\022V\n\017ModifyNamespace\022 .hbase.pb.ModifyNa" +
+ "mespaceRequest\032!.hbase.pb.ModifyNamespac" +
+ "eResponse\022V\n\017CreateNamespace\022 .hbase.pb." +
+ "CreateNamespaceRequest\032!.hbase.pb.Create" +
+ "NamespaceResponse\022V\n\017DeleteNamespace\022 .h" +
+ "base.pb.DeleteNamespaceRequest\032!.hbase.p",
+ "b.DeleteNamespaceResponse\022k\n\026GetNamespac" +
+ "eDescriptor\022\'.hbase.pb.GetNamespaceDescr" +
+ "iptorRequest\032(.hbase.pb.GetNamespaceDesc" +
+ "riptorResponse\022q\n\030ListNamespaceDescripto" +
+ "rs\022).hbase.pb.ListNamespaceDescriptorsRe" +
+ "quest\032*.hbase.pb.ListNamespaceDescriptor" +
+ "sResponse\022\206\001\n\037ListTableDescriptorsByName" +
+ "space\0220.hbase.pb.ListTableDescriptorsByN" +
+ "amespaceRequest\0321.hbase.pb.ListTableDesc" +
+ "riptorsByNamespaceResponse\022t\n\031ListTableN",
+ "amesByNamespace\022*.hbase.pb.ListTableName" +
+ "sByNamespaceRequest\032+.hbase.pb.ListTable" +
+ "NamesByNamespaceResponse\022A\n\010SetQuota\022\031.h" +
+ "base.pb.SetQuotaRequest\032\032.hbase.pb.SetQu" +
+ "otaResponse\022x\n\037getLastMajorCompactionTim" +
+ "estamp\022).hbase.pb.MajorCompactionTimesta" +
+ "mpRequest\032*.hbase.pb.MajorCompactionTime" +
+ "stampResponse\022\212\001\n(getLastMajorCompaction" +
+ "TimestampForRegion\0222.hbase.pb.MajorCompa" +
+ "ctionTimestampForRegionRequest\032*.hbase.p",
+ "b.MajorCompactionTimestampResponse\022_\n\022ge" +
+ "tProcedureResult\022#.hbase.pb.GetProcedure" +
+ "ResultRequest\032$.hbase.pb.GetProcedureRes" +
+ "ultResponse\022h\n\027getSecurityCapabilities\022%" +
+ ".hbase.pb.SecurityCapabilitiesRequest\032&." +
+ "hbase.pb.SecurityCapabilitiesResponseBB\n" +
+ "*org.apache.hadoop.hbase.protobuf.genera" +
+ "tedB\014MasterProtosH\001\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -55038,7 +55581,7 @@ public final class MasterProtos {
internal_static_hbase_pb_CreateNamespaceRequest_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_CreateNamespaceRequest_descriptor,
- new java.lang.String[] { "NamespaceDescriptor", });
+ new java.lang.String[] { "NamespaceDescriptor", "NonceGroup", "Nonce", });
internal_static_hbase_pb_CreateNamespaceResponse_descriptor =
getDescriptor().getMessageTypes().get(29);
internal_static_hbase_pb_CreateNamespaceResponse_fieldAccessorTable = new
@@ -55050,7 +55593,7 @@ public final class MasterProtos {
internal_static_hbase_pb_DeleteNamespaceRequest_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_DeleteNamespaceRequest_descriptor,
- new java.lang.String[] { "NamespaceName", });
+ new java.lang.String[] { "NamespaceName", "NonceGroup", "Nonce", });
internal_static_hbase_pb_DeleteNamespaceResponse_descriptor =
getDescriptor().getMessageTypes().get(31);
internal_static_hbase_pb_DeleteNamespaceResponse_fieldAccessorTable = new
@@ -55062,7 +55605,7 @@ public final class MasterProtos {
internal_static_hbase_pb_ModifyNamespaceRequest_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_ModifyNamespaceRequest_descriptor,
- new java.lang.String[] { "NamespaceDescriptor", });
+ new java.lang.String[] { "NamespaceDescriptor", "NonceGroup", "Nonce", });
internal_static_hbase_pb_ModifyNamespaceResponse_descriptor =
getDescriptor().getMessageTypes().get(33);
internal_static_hbase_pb_ModifyNamespaceResponse_fieldAccessorTable = new
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/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 86cd655..cb81317 100644
--- a/hbase-protocol/src/main/protobuf/Master.proto
+++ b/hbase-protocol/src/main/protobuf/Master.proto
@@ -175,6 +175,8 @@ message ModifyTableResponse {
message CreateNamespaceRequest {
required NamespaceDescriptor namespaceDescriptor = 1;
+ optional uint64 nonce_group = 2 [default = 0];
+ optional uint64 nonce = 3 [default = 0];
}
message CreateNamespaceResponse {
@@ -182,6 +184,8 @@ message CreateNamespaceResponse {
message DeleteNamespaceRequest {
required string namespaceName = 1;
+ optional uint64 nonce_group = 2 [default = 0];
+ optional uint64 nonce = 3 [default = 0];
}
message DeleteNamespaceResponse {
@@ -189,6 +193,8 @@ message DeleteNamespaceResponse {
message ModifyNamespaceRequest {
required NamespaceDescriptor namespaceDescriptor = 1;
+ optional uint64 nonce_group = 2 [default = 0];
+ optional uint64 nonce = 3 [default = 0];
}
message ModifyNamespaceResponse {
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/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 c445434..2d2aff4 100644
--- a/hbase-protocol/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
@@ -110,6 +110,42 @@ message DeleteTableStateData {
repeated RegionInfo region_info = 3;
}
+enum CreateNamespaceState {
+ CREATE_NAMESPACE_PREPARE = 1;
+ CREATE_NAMESPACE_CREATE_DIRECTORY = 2;
+ CREATE_NAMESPACE_INSERT_INTO_NS_TABLE = 3;
+ CREATE_NAMESPACE_UPDATE_ZK = 4;
+ CREATE_NAMESPACE_SET_NAMESPACE_QUOTA = 5;
+}
+
+message CreateNamespaceStateData {
+ required NamespaceDescriptor namespace_descriptor = 1;
+}
+
+enum ModifyNamespaceState {
+ MODIFY_NAMESPACE_PREPARE = 1;
+ MODIFY_NAMESPACE_UPDATE_NS_TABLE = 2;
+ MODIFY_NAMESPACE_UPDATE_ZK = 3;
+}
+
+message ModifyNamespaceStateData {
+ required NamespaceDescriptor namespace_descriptor = 1;
+ optional NamespaceDescriptor unmodified_namespace_descriptor = 2;
+}
+
+enum DeleteNamespaceState {
+ DELETE_NAMESPACE_PREPARE = 1;
+ DELETE_NAMESPACE_DELETE_FROM_NS_TABLE = 2;
+ DELETE_NAMESPACE_REMOVE_FROM_ZK = 3;
+ DELETE_NAMESPACE_DELETE_DIRECTORIES = 4;
+ DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA = 5;
+}
+
+message DeleteNamespaceStateData {
+ required string namespace_name = 1;
+ optional NamespaceDescriptor namespace_descriptor = 2;
+}
+
enum AddColumnFamilyState {
ADD_COLUMN_FAMILY_PREPARE = 1;
ADD_COLUMN_FAMILY_PRE_OPERATION = 2;
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
index 7f5faa6..b0aabb2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
@@ -169,8 +169,13 @@ public class ZKNamespaceManager extends ZooKeeperListener {
try {
ZKUtil.deleteNode(watcher, zNode);
} catch (KeeperException e) {
- LOG.error("Failed updating permissions for namespace "+name, e);
- throw new IOException("Failed updating permissions for namespace "+name, e);
+ if (e instanceof KeeperException.NoNodeException) {
+ // If the node does not exist, it could be already deleted. Continue without fail.
+ LOG.warn("The ZNode " + zNode + " for namespace " + name + " does not exist.");
+ } else {
+ LOG.error("Failed updating permissions for namespace " + name, e);
+ throw new IOException("Failed updating permissions for namespace " + name, e);
+ }
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/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 b57b993..3ce9b40 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
@@ -93,14 +93,17 @@ 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.procedure.AddColumnFamilyProcedure;
+import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
+import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
+import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
@@ -1039,6 +1042,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
return this.fileSystemManager;
}
+ @Override
+ public TableNamespaceManager getTableNamespaceManager() {
+ return tableNamespaceManager;
+ }
+
/*
* Start up all services. If any of these threads gets an unhandled exception
* then they just die with a logged message. This should be fine because
@@ -2186,7 +2194,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
void checkNamespaceManagerReady() throws IOException {
checkInitialized();
if (tableNamespaceManager == null ||
- !tableNamespaceManager.isTableAvailableAndInitialized()) {
+ !tableNamespaceManager.isTableAvailableAndInitialized(true)) {
throw new IOException("Table Namespace Manager not ready yet, try again later");
}
}
@@ -2327,7 +2335,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
@Override
- public void createNamespace(NamespaceDescriptor descriptor) throws IOException {
+ public void createNamespace(
+ final NamespaceDescriptor descriptor,
+ final long nonceGroup,
+ final long nonce) throws IOException {
TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
checkNamespaceManagerReady();
if (cpHost != null) {
@@ -2335,15 +2346,31 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
return;
}
}
- LOG.info(getClientIdAuditPrefix() + " creating " + descriptor);
- tableNamespaceManager.create(descriptor);
+ createNamespaceSync(descriptor, nonceGroup, nonce);
if (cpHost != null) {
cpHost.postCreateNamespace(descriptor);
}
}
@Override
- public void modifyNamespace(NamespaceDescriptor descriptor) throws IOException {
+ public void createNamespaceSync(
+ final NamespaceDescriptor descriptor,
+ final long nonceGroup,
+ final long nonce) throws IOException {
+ LOG.info(getClientIdAuditPrefix() + " creating " + descriptor);
+ // Execute the operation synchronously - wait for the operation to complete before continuing.
+ long procId = this.procedureExecutor.submitProcedure(
+ new CreateNamespaceProcedure(procedureExecutor.getEnvironment(), descriptor),
+ nonceGroup,
+ nonce);
+ ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+ }
+
+ @Override
+ public void modifyNamespace(
+ final NamespaceDescriptor descriptor,
+ final long nonceGroup,
+ final long nonce) throws IOException {
TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
checkNamespaceManagerReady();
if (cpHost != null) {
@@ -2352,14 +2379,22 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
}
LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
- tableNamespaceManager.update(descriptor);
+ // Execute the operation synchronously - wait for the operation to complete before continuing.
+ long procId = this.procedureExecutor.submitProcedure(
+ new ModifyNamespaceProcedure(procedureExecutor.getEnvironment(), descriptor),
+ nonceGroup,
+ nonce);
+ ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
if (cpHost != null) {
cpHost.postModifyNamespace(descriptor);
}
}
@Override
- public void deleteNamespace(String name) throws IOException {
+ public void deleteNamespace(
+ final String name,
+ final long nonceGroup,
+ final long nonce) throws IOException {
checkNamespaceManagerReady();
if (cpHost != null) {
if (cpHost.preDeleteNamespace(name)) {
@@ -2367,7 +2402,12 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
}
LOG.info(getClientIdAuditPrefix() + " delete " + name);
- tableNamespaceManager.remove(name);
+ // Execute the operation synchronously - wait for the operation to complete before continuing.
+ long procId = this.procedureExecutor.submitProcedure(
+ new DeleteNamespaceProcedure(procedureExecutor.getEnvironment(), name),
+ nonceGroup,
+ nonce);
+ ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
if (cpHost != null) {
cpHost.postDeleteNamespace(name);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/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 d7bbc91..a0d7765 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
@@ -410,7 +410,10 @@ public class MasterRpcServices extends RSRpcServices
public CreateNamespaceResponse createNamespace(RpcController controller,
CreateNamespaceRequest request) throws ServiceException {
try {
- master.createNamespace(ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()));
+ master.createNamespace(
+ ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
+ request.getNonceGroup(),
+ request.getNonce());
return CreateNamespaceResponse.getDefaultInstance();
} catch (IOException e) {
throw new ServiceException(e);
@@ -450,7 +453,10 @@ public class MasterRpcServices extends RSRpcServices
public DeleteNamespaceResponse deleteNamespace(RpcController controller,
DeleteNamespaceRequest request) throws ServiceException {
try {
- master.deleteNamespace(request.getNamespaceName());
+ master.deleteNamespace(
+ request.getNamespaceName(),
+ request.getNonceGroup(),
+ request.getNonce());
return DeleteNamespaceResponse.getDefaultInstance();
} catch (IOException e) {
throw new ServiceException(e);
@@ -1089,7 +1095,9 @@ public class MasterRpcServices extends RSRpcServices
ModifyNamespaceRequest request) throws ServiceException {
try {
master.modifyNamespace(
- ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()));
+ ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
+ request.getNonceGroup(),
+ request.getNonce());
return ModifyNamespaceResponse.getDefaultInstance();
} catch (IOException e) {
throw new ServiceException(e);
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/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 c22f0ed..980a66d 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
@@ -74,6 +74,11 @@ public interface MasterServices extends Server {
MasterCoprocessorHost getMasterCoprocessorHost();
/**
+ * @return Master's instance of {@link TableNamespaceManager}
+ */
+ TableNamespaceManager getTableNamespaceManager();
+
+ /**
* @return Master's instance of {@link MasterQuotaManager}
*/
MasterQuotaManager getMasterQuotaManager();
@@ -265,23 +270,50 @@ public interface MasterServices extends Server {
/**
* Create a new namespace
* @param descriptor descriptor which describes the new namespace
+ * @param nonceGroup
+ * @param nonce
+ * @throws IOException
+ */
+ public void createNamespace(
+ final NamespaceDescriptor descriptor,
+ final long nonceGroup,
+ final long nonce) throws IOException;
+
+ /**
+ * Create a new namespace synchronously.
+ * @param descriptor descriptor which describes the new namespace
+ * @param nonceGroup
+ * @param nonce
* @throws IOException
*/
- public void createNamespace(NamespaceDescriptor descriptor) throws IOException;
+ public void createNamespaceSync(
+ final NamespaceDescriptor descriptor,
+ final long nonceGroup,
+ final long nonce) throws IOException;
/**
* Modify an existing namespace
* @param descriptor descriptor which updates the existing namespace
+ * @param nonceGroup
+ * @param nonce
* @throws IOException
*/
- public void modifyNamespace(NamespaceDescriptor descriptor) throws IOException;
+ public void modifyNamespace(
+ final NamespaceDescriptor descriptor,
+ final long nonceGroup,
+ final long nonce) throws IOException;
/**
* Delete an existing namespace. Only empty namespaces (no tables) can be removed.
* @param name namespace name
+ * @param nonceGroup
+ * @param nonce
* @throws IOException
*/
- public void deleteNamespace(String name) throws IOException;
+ public void deleteNamespace(
+ final String name,
+ final long nonceGroup,
+ final long nonce) throws IOException;
/**
* Get a namespace descriptor by name
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b96cf7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
index 74d1339..f4be427 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
@@ -18,26 +18,23 @@
package org.apache.hadoop.hbase.master;
-import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.NavigableSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.NamespaceExistException;
-import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZKNamespaceManager;
import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -48,12 +45,12 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.constraint.ConstraintException;
+import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
import com.google.common.collect.Sets;
@@ -69,24 +66,44 @@ public class TableNamespaceManager {
private Configuration conf;
private MasterServices masterServices;
- private Table nsTable;
+ private Table nsTable = null;
private ZKNamespaceManager zkNamespaceManager;
private boolean initialized;
-
+
+ private final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
+
public static final String KEY_MAX_REGIONS = "hbase.namespace.quota.maxregions";
public static final String KEY_MAX_TABLES = "hbase.namespace.quota.maxtables";
-
static final String NS_INIT_TIMEOUT = "hbase.master.namespace.init.timeout";
static final int DEFAULT_NS_INIT_TIMEOUT = 300000;
+ /** Configuration key for time out for trying to acquire table locks */
+ private static final String TABLE_WRITE_LOCK_TIMEOUT_MS =
+ "hbase.table.write.lock.timeout.ms";
+ /** Configuration key for time out for trying to acquire table locks */
+ private static final String TABLE_READ_LOCK_TIMEOUT_MS =
+ "hbase.table.read.lock.timeout.ms";
+ private static final long DEFAULT_TABLE_WRITE_LOCK_TIMEOUT_MS = 600 * 1000; //10 min default
+ private static final long DEFAULT_TABLE_READ_LOCK_TIMEOUT_MS = 600 * 1000; //10 min default
+
+ private long exclusiveLockTimeoutMs;
+ private long sharedLockTimeoutMs;
+
public TableNamespaceManager(MasterServices masterServices) {
this.masterServices = masterServices;
this.conf = masterServices.getConfiguration();
+
+ this.exclusiveLockTimeoutMs = conf.getLong(
+ TABLE_WRITE_LOCK_TIMEOUT_MS,
+ DEFAULT_TABLE_WRITE_LOCK_TIMEOUT_MS);
+ this.sharedLockTimeoutMs = conf.getLong(
+ TABLE_READ_LOCK_TIMEOUT_MS,
+ DEFAULT_TABLE_READ_LOCK_TIMEOUT_MS);
}
public void start() throws IOException {
if (!MetaTableAccessor.tableExists(masterServices.getConnection(),
- TableName.NAMESPACE_TABLE_NAME)) {
+ TableName.NAMESPACE_TABLE_NAME)) {
LOG.info("Namespace table not found. Creating...");
createNamespaceTable(masterServices);
}
@@ -97,7 +114,7 @@ public class TableNamespaceManager {
// So that it should be initialized later on lazily.
long startTime = EnvironmentEdgeManager.currentTime();
int timeout = conf.getInt(NS_INIT_TIMEOUT, DEFAULT_NS_INIT_TIMEOUT);
- while (!isTableAssigned()) {
+ while (!isTableAvailableAndInitialized(false)) {
if (EnvironmentEdgeManager.currentTime() - startTime + 100 > timeout) {
// We can't do anything if ns is not online.
throw new IOException("Timedout " + timeout + "ms waiting for namespace table to " +
@@ -106,36 +123,56 @@ public class TableNamespaceManager {
Thread.sleep(100);
}
} catch (InterruptedException e) {
- throw (InterruptedIOException)new InterruptedIOException().initCause(e);
+ throw (InterruptedIOException) new InterruptedIOException().initCause(e);
}
-
- // initialize namespace table
- isTableAvailableAndInitialized();
}
private synchronized Table getNamespaceTable() throws IOException {
- if (!isTableAvailableAndInitialized()) {
+ if (!isTableNamespaceManagerInitialized()) {
throw new IOException(this.getClass().getName() + " isn't ready to serve");
}
return nsTable;
}
+ private synchronized boolean acquireSharedLock() throws IOException {
+ try {
+ return rwLock.readLock().tryLock(sharedLockTimeoutMs, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException e) {
+ throw (InterruptedIOException) new InterruptedIOException().initCause(e);
+ }
+ }
- public synchronized NamespaceDescriptor get(String name) throws IOException {
- if (!isTableAvailableAndInitialized()) return null;
- return zkNamespaceManager.get(name);
+ public synchronized void releaseSharedLock() {
+ rwLock.readLock().unlock();
}
- public synchronized void create(NamespaceDescriptor ns) throws IOException {
- create(getNamespaceTable(), ns);
+ public synchronized boolean acquireExclusiveLock() {
+ try {
+ return rwLock.writeLock().tryLock(exclusiveLockTimeoutMs, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException e) {
+ return false;
+ }
}
- public synchronized void update(NamespaceDescriptor ns) throws IOException {
- Table table = getNamespaceTable();
- if (get(table, ns.getName()) == null) {
- throw new NamespaceNotFoundException(ns.getName());
+ public synchronized void releaseExclusiveLock() {
+ rwLock.writeLock().unlock();
+ }
+
+ /*
+ * check whether a namespace has already existed.
+ */
+ public boolean doesNamespaceExist(final String namespaceName) throws IOException {
+ if (nsTable == null) {
+ throw new IOException(this.getClass().getName() + " isn't ready to serve");
}
- upsert(table, ns);
+ return (get(nsTable, namespaceName) != null);
+ }
+
+ public synchronized NamespaceDescriptor get(String name) throws IOException {
+ if (!isTableNamespaceManagerInitialized()) {
+ return null;
+ }
+ return zkNamespaceManager.get(name);
}
private NamespaceDescriptor get(Table table, String name) throws IOException {
@@ -150,78 +187,51 @@ public class TableNamespaceManager {
HBaseProtos.NamespaceDescriptor.parseFrom(val));
}
- private void create(Table table, NamespaceDescriptor ns) throws IOException {
- if (get(table, ns.getName()) != null) {
- throw new NamespaceExistException(ns.getName());
- }
- validateTableAndRegionCount(ns);
- FileSystem fs = masterServices.getMasterFileSystem().getFileSystem();
- fs.mkdirs(FSUtils.getNamespaceDir(
- masterServices.getMasterFileSystem().getRootDir(), ns.getName()));
- upsert(table, ns);
- if (this.masterServices.isInitialized()) {
- this.masterServices.getMasterQuotaManager().setNamespaceQuota(ns);
+ public void insertIntoNSTable(final NamespaceDescriptor ns) throws IOException {
+ if (nsTable == null) {
+ throw new IOException(this.getClass().getName() + " isn't ready to serve");
}
- }
-
- private void upsert(Table table, NamespaceDescriptor ns) throws IOException {
- validateTableAndRegionCount(ns);
Put p = new Put(Bytes.toBytes(ns.getName()));
p.addImmutable(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_BYTES,
ProtobufUtil.toProtoNamespaceDescriptor(ns).toByteArray());
- table.put(p);
+ nsTable.put(p);
+ }
+
+ public void updateZKNamespaceManager(final NamespaceDescriptor ns) throws IOException {
try {
zkNamespaceManager.update(ns);
- } catch(IOException ex) {
- String msg = "Failed to update namespace information in ZK. Aborting.";
- LOG.fatal(msg, ex);
- masterServices.abort(msg, ex);
+ } catch (IOException ex) {
+ String msg = "Failed to update namespace information in ZK.";
+ LOG.error(msg, ex);
+ throw new IOException(msg, ex);
}
}
- public synchronized void remove(String name) throws IOException {
- if (get(name) == null) {
- throw new NamespaceNotFoundException(name);
- }
- if (NamespaceDescriptor.RESERVED_NAMESPACES.contains(name)) {
- throw new ConstraintException("Reserved namespace "+name+" cannot be removed.");
- }
- int tableCount;
- try {
- tableCount = masterServices.listTableDescriptorsByNamespace(name).size();
- } catch (FileNotFoundException fnfe) {
- throw new NamespaceNotFoundException(name);
- }
- if (tableCount > 0) {
- throw new ConstraintException("Only empty namespaces can be removed. " +
- "Namespace "+name+" has "+tableCount+" tables");
- }
- Delete d = new Delete(Bytes.toBytes(name));
- getNamespaceTable().delete(d);
- //don't abort if cleanup isn't complete
- //it will be replaced on new namespace creation
- zkNamespaceManager.remove(name);
- FileSystem fs = masterServices.getMasterFileSystem().getFileSystem();
- for(FileStatus status :
- fs.listStatus(FSUtils.getNamespaceDir(
- masterServices.getMasterFileSystem().getRootDir(), name))) {
- if (!HConstants.HBASE_NON_TABLE_DIRS.contains(status.getPath().getName())) {
- throw new IOException("Namespace directory contains table dir: "+status.getPath());
- }
- }
- if (!fs.delete(FSUtils.getNamespaceDir(
- masterServices.getMasterFileSystem().getRootDir(), name), true)) {
- throw new IOException("Failed to remove namespace: "+name);
+ public void removeFromNSTable(final String namespaceName) throws IOException {
+ if (nsTable == null) {
+ throw new IOException(this.getClass().getName() + " isn't ready to serve");
}
- this.masterServices.getMasterQuotaManager().removeNamespaceQuota(name);
+ Delete d = new Delete(Bytes.toBytes(namespaceName));
+ nsTable.delete(d);
+ }
+
+ public void removeFromZKNamespaceManager(final String namespaceName) throws IOException {
+ zkNamespaceManager.remove(namespaceName);
}
public synchronized NavigableSet<NamespaceDescriptor> list() throws IOException {
NavigableSet<NamespaceDescriptor> ret =
Sets.newTreeSet(NamespaceDescriptor.NAMESPACE_DESCRIPTOR_COMPARATOR);
- ResultScanner scanner = getNamespaceTable().getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
+ ResultScanner scanner =
+ getNamespaceTable().getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
+ boolean locked = false;
try {
+ locked = acquireSharedLock();
+ if (!locked) {
+ throw new IOException(
+ "Fail to acquire lock to scan namespace list. Some namespace DDL is in progress.");
+ }
for(Result r : scanner) {
byte[] val = CellUtil.cloneValue(r.getColumnLatestCell(
HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
@@ -231,6 +241,9 @@ public class TableNamespaceManager {
}
} finally {
scanner.close();
+ if (locked) {
+ releaseSharedLock();
+ }
}
return ret;
}
@@ -247,6 +260,15 @@ public class TableNamespaceManager {
newRegions));
}
+ @SuppressWarnings("deprecation")
+ private boolean isTableNamespaceManagerInitialized() throws IOException {
+ if (initialized) {
+ this.nsTable = this.masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME);
+ return true;
+ }
+ return false;
+ }
+
/**
* This method checks if the namespace table is assigned and then
* tries to create its HTable. If it was already created before, it also makes
@@ -256,31 +278,61 @@ public class TableNamespaceManager {
* @throws IOException
*/
@SuppressWarnings("deprecation")
- public synchronized boolean isTableAvailableAndInitialized() throws IOException {
+ public synchronized boolean isTableAvailableAndInitialized(
+ final boolean createNamespaceAync) throws IOException {
// Did we already get a table? If so, still make sure it's available
- if (initialized) {
- this.nsTable = this.masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME);
+ if (isTableNamespaceManagerInitialized()) {
return true;
}
// Now check if the table is assigned, if not then fail fast
if (isTableAssigned()) {
try {
+ boolean initGoodSofar = true;
nsTable = this.masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME);
zkNamespaceManager = new ZKNamespaceManager(masterServices.getZooKeeper());
zkNamespaceManager.start();
if (get(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) {
- create(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE);
+ if (createNamespaceAync) {
+ masterServices.getMasterProcedureExecutor().submitProcedure(
+ new CreateNamespaceProcedure(
+ masterServices.getMasterProcedureExecutor().getEnvironment(),
+ NamespaceDescriptor.DEFAULT_NAMESPACE));
+ initGoodSofar = false;
+ }
+ else {
+ masterServices.createNamespaceSync(
+ NamespaceDescriptor.DEFAULT_NAMESPACE,
+ HConstants.NO_NONCE,
+ HConstants.NO_NONCE);
+ }
}
if (get(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == null) {
- create(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE);
+ if (createNamespaceAync) {
+ masterServices.getMasterProcedureExecutor().submitProcedure(
+ new CreateNamespaceProcedure(
+ masterServices.getMasterProcedureExecutor().getEnvironment(),
+ NamespaceDescriptor.SYSTEM_NAMESPACE));
+ initGoodSofar = false;
+ }
+ else {
+ masterServices.createNamespaceSync(
+ NamespaceDescriptor.SYSTEM_NAMESPACE,
+ HConstants.NO_NONCE,
+ HConstants.NO_NONCE);
+ }
+ }
+
+ if (!initGoodSofar) {
+ // some required namespace is created asynchronized. We should complete init later.
+ return false;
}
ResultScanner scanner = nsTable.getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
try {
for (Result result : scanner) {
- byte[] val = CellUtil.cloneValue(result.getColumnLatest(
+ byte[] val = CellUtil.cloneValue(result.getColumnLatestCell(
HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_BYTES));
NamespaceDescriptor ns =
@@ -306,10 +358,10 @@ public class TableNamespaceManager {
private boolean isTableAssigned() {
return !masterServices.getAssignmentManager().getRegionStates().
- getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME).isEmpty();
+ getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME).isEmpty();
}
-
- void validateTableAndRegionCount(NamespaceDescriptor desc) throws IOException {
+
+ public void validateTableAndRegionCount(NamespaceDescriptor desc) throws IOException {
if (getMaxRegions(desc) <= 0) {
throw new ConstraintException("The max region quota for " + desc.getName()
+ " is less than or equal to zero.");