HBASE-16618 Procedure v2 - Add base class for table and ns procedures
This commit is contained in:
parent
981200bf13
commit
4c6a98bd76
|
@ -0,0 +1,70 @@
|
||||||
|
/**
|
||||||
|
* 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 org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base class for all the Namespace procedures that want to use a StateMachineProcedure.
|
||||||
|
* It provide some basic helpers like basic locking and basic toStringClassDetails().
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public abstract class AbstractStateMachineNamespaceProcedure<TState>
|
||||||
|
extends StateMachineProcedure<MasterProcedureEnv, TState>
|
||||||
|
implements TableProcedureInterface {
|
||||||
|
|
||||||
|
protected AbstractStateMachineNamespaceProcedure() {
|
||||||
|
// Required by the Procedure framework to create the procedure on replay
|
||||||
|
}
|
||||||
|
|
||||||
|
protected AbstractStateMachineNamespaceProcedure(final MasterProcedureEnv env) {
|
||||||
|
this.setOwner(env.getRequestUser().getShortName());
|
||||||
|
}
|
||||||
|
|
||||||
|
protected abstract String getNamespaceName();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TableName getTableName() {
|
||||||
|
return TableName.NAMESPACE_TABLE_NAME;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public abstract TableOperationType getTableOperationType();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void toStringClassDetails(final StringBuilder sb) {
|
||||||
|
sb.append(getClass().getSimpleName());
|
||||||
|
sb.append(" (namespace=");
|
||||||
|
sb.append(getNamespaceName());
|
||||||
|
sb.append(")");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected boolean acquireLock(final MasterProcedureEnv env) {
|
||||||
|
if (env.waitInitialized(this)) return false;
|
||||||
|
return env.getProcedureQueue().tryAcquireNamespaceExclusiveLock(this, getNamespaceName());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void releaseLock(final MasterProcedureEnv env) {
|
||||||
|
env.getProcedureQueue().releaseNamespaceExclusiveLock(this, getNamespaceName());
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,111 @@
|
||||||
|
/**
|
||||||
|
* 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 org.apache.hadoop.hbase.MetaTableAccessor;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base class for all the Table procedures that want to use a StateMachineProcedure.
|
||||||
|
* It provide some basic helpers like basic locking, sync latch, and basic toStringClassDetails().
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public abstract class AbstractStateMachineTableProcedure<TState>
|
||||||
|
extends StateMachineProcedure<MasterProcedureEnv, TState>
|
||||||
|
implements TableProcedureInterface {
|
||||||
|
|
||||||
|
// used for compatibility with old clients
|
||||||
|
private final ProcedurePrepareLatch syncLatch;
|
||||||
|
|
||||||
|
private User user;
|
||||||
|
|
||||||
|
protected AbstractStateMachineTableProcedure() {
|
||||||
|
// Required by the Procedure framework to create the procedure on replay
|
||||||
|
syncLatch = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected AbstractStateMachineTableProcedure(final MasterProcedureEnv env) {
|
||||||
|
this(env, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected AbstractStateMachineTableProcedure(final MasterProcedureEnv env,
|
||||||
|
final ProcedurePrepareLatch latch) {
|
||||||
|
this.user = env.getRequestUser();
|
||||||
|
this.setOwner(user.getShortName());
|
||||||
|
|
||||||
|
// used for compatibility with clients without procedures
|
||||||
|
// they need a sync TableExistsException, TableNotFoundException, TableNotDisabledException, ...
|
||||||
|
this.syncLatch = latch;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public abstract TableName getTableName();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public abstract TableOperationType getTableOperationType();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void toStringClassDetails(final StringBuilder sb) {
|
||||||
|
sb.append(getClass().getSimpleName());
|
||||||
|
sb.append(" (table=");
|
||||||
|
sb.append(getTableName());
|
||||||
|
sb.append(")");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected boolean acquireLock(final MasterProcedureEnv env) {
|
||||||
|
if (env.waitInitialized(this)) return false;
|
||||||
|
return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, getTableName());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void releaseLock(final MasterProcedureEnv env) {
|
||||||
|
env.getProcedureQueue().releaseTableExclusiveLock(this, getTableName());
|
||||||
|
}
|
||||||
|
|
||||||
|
protected User getUser() {
|
||||||
|
return user;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void setUser(final User user) {
|
||||||
|
this.user = user;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void releaseSyncLatch() {
|
||||||
|
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check whether a table is modifiable - exists and either offline or online with config set
|
||||||
|
* @param env MasterProcedureEnv
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
protected void checkTableModifiable(final MasterProcedureEnv env) throws IOException {
|
||||||
|
// Checks whether the table exists
|
||||||
|
if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), getTableName())) {
|
||||||
|
throw new TableNotFoundException(getTableName());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -33,37 +33,30 @@ import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.TableState;
|
import org.apache.hadoop.hbase.client.TableState;
|
||||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyState;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The procedure to add a column family to an existing table.
|
* The procedure to add a column family to an existing table.
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class AddColumnFamilyProcedure
|
public class AddColumnFamilyProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, AddColumnFamilyState>
|
extends AbstractStateMachineTableProcedure<AddColumnFamilyState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(AddColumnFamilyProcedure.class);
|
private static final Log LOG = LogFactory.getLog(AddColumnFamilyProcedure.class);
|
||||||
|
|
||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
private HTableDescriptor unmodifiedHTableDescriptor;
|
private HTableDescriptor unmodifiedHTableDescriptor;
|
||||||
private HColumnDescriptor cfDescriptor;
|
private HColumnDescriptor cfDescriptor;
|
||||||
private User user;
|
|
||||||
|
|
||||||
private List<HRegionInfo> regionInfoList;
|
private List<HRegionInfo> regionInfoList;
|
||||||
private Boolean traceEnabled;
|
private Boolean traceEnabled;
|
||||||
|
|
||||||
// used for compatibility with old clients, until 2.0 the client had a sync behavior
|
|
||||||
private final ProcedurePrepareLatch syncLatch;
|
|
||||||
|
|
||||||
public AddColumnFamilyProcedure() {
|
public AddColumnFamilyProcedure() {
|
||||||
|
super();
|
||||||
this.unmodifiedHTableDescriptor = null;
|
this.unmodifiedHTableDescriptor = null;
|
||||||
this.regionInfoList = null;
|
this.regionInfoList = null;
|
||||||
this.traceEnabled = null;
|
this.traceEnabled = null;
|
||||||
this.syncLatch = null;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public AddColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
public AddColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||||
|
@ -73,14 +66,12 @@ public class AddColumnFamilyProcedure
|
||||||
|
|
||||||
public AddColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
public AddColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||||
final HColumnDescriptor cfDescriptor, final ProcedurePrepareLatch latch) {
|
final HColumnDescriptor cfDescriptor, final ProcedurePrepareLatch latch) {
|
||||||
|
super(env, latch);
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.cfDescriptor = cfDescriptor;
|
this.cfDescriptor = cfDescriptor;
|
||||||
this.user = env.getRequestUser();
|
|
||||||
this.setOwner(this.user.getShortName());
|
|
||||||
this.unmodifiedHTableDescriptor = null;
|
this.unmodifiedHTableDescriptor = null;
|
||||||
this.regionInfoList = null;
|
this.regionInfoList = null;
|
||||||
this.traceEnabled = null;
|
this.traceEnabled = null;
|
||||||
this.syncLatch = latch;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -153,7 +144,7 @@ public class AddColumnFamilyProcedure
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void completionCleanup(final MasterProcedureEnv env) {
|
protected void completionCleanup(final MasterProcedureEnv env) {
|
||||||
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
|
releaseSyncLatch();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -171,24 +162,13 @@ public class AddColumnFamilyProcedure
|
||||||
return AddColumnFamilyState.ADD_COLUMN_FAMILY_PREPARE;
|
return AddColumnFamilyState.ADD_COLUMN_FAMILY_PREPARE;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected boolean acquireLock(final MasterProcedureEnv env) {
|
|
||||||
if (env.waitInitialized(this)) return false;
|
|
||||||
return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void releaseLock(final MasterProcedureEnv env) {
|
|
||||||
env.getProcedureQueue().releaseTableExclusiveLock(this, tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void serializeStateData(final OutputStream stream) throws IOException {
|
public void serializeStateData(final OutputStream stream) throws IOException {
|
||||||
super.serializeStateData(stream);
|
super.serializeStateData(stream);
|
||||||
|
|
||||||
MasterProcedureProtos.AddColumnFamilyStateData.Builder addCFMsg =
|
MasterProcedureProtos.AddColumnFamilyStateData.Builder addCFMsg =
|
||||||
MasterProcedureProtos.AddColumnFamilyStateData.newBuilder()
|
MasterProcedureProtos.AddColumnFamilyStateData.newBuilder()
|
||||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
|
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||||
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
||||||
.setColumnfamilySchema(ProtobufUtil.convertToColumnFamilySchema(cfDescriptor));
|
.setColumnfamilySchema(ProtobufUtil.convertToColumnFamilySchema(cfDescriptor));
|
||||||
if (unmodifiedHTableDescriptor != null) {
|
if (unmodifiedHTableDescriptor != null) {
|
||||||
|
@ -205,7 +185,7 @@ public class AddColumnFamilyProcedure
|
||||||
|
|
||||||
MasterProcedureProtos.AddColumnFamilyStateData addCFMsg =
|
MasterProcedureProtos.AddColumnFamilyStateData addCFMsg =
|
||||||
MasterProcedureProtos.AddColumnFamilyStateData.parseDelimitedFrom(stream);
|
MasterProcedureProtos.AddColumnFamilyStateData.parseDelimitedFrom(stream);
|
||||||
user = MasterProcedureUtil.toUserInfo(addCFMsg.getUserInfo());
|
setUser(MasterProcedureUtil.toUserInfo(addCFMsg.getUserInfo()));
|
||||||
tableName = ProtobufUtil.toTableName(addCFMsg.getTableName());
|
tableName = ProtobufUtil.toTableName(addCFMsg.getTableName());
|
||||||
cfDescriptor = ProtobufUtil.convertToHColumnDesc(addCFMsg.getColumnfamilySchema());
|
cfDescriptor = ProtobufUtil.convertToHColumnDesc(addCFMsg.getColumnfamilySchema());
|
||||||
if (addCFMsg.hasUnmodifiedTableSchema()) {
|
if (addCFMsg.hasUnmodifiedTableSchema()) {
|
||||||
|
@ -244,7 +224,7 @@ public class AddColumnFamilyProcedure
|
||||||
*/
|
*/
|
||||||
private void prepareAdd(final MasterProcedureEnv env) throws IOException {
|
private void prepareAdd(final MasterProcedureEnv env) throws IOException {
|
||||||
// Checks whether the table is allowed to be modified.
|
// Checks whether the table is allowed to be modified.
|
||||||
MasterDDLOperationHelper.checkTableModifiable(env, tableName);
|
checkTableModifiable(env);
|
||||||
|
|
||||||
// In order to update the descriptor, we need to retrieve the old descriptor for comparison.
|
// In order to update the descriptor, we need to retrieve the old descriptor for comparison.
|
||||||
unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
|
unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||||
|
@ -369,10 +349,10 @@ public class AddColumnFamilyProcedure
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
switch (state) {
|
switch (state) {
|
||||||
case ADD_COLUMN_FAMILY_PRE_OPERATION:
|
case ADD_COLUMN_FAMILY_PRE_OPERATION:
|
||||||
cpHost.preAddColumnFamilyAction(tableName, cfDescriptor, user);
|
cpHost.preAddColumnFamilyAction(tableName, cfDescriptor, getUser());
|
||||||
break;
|
break;
|
||||||
case ADD_COLUMN_FAMILY_POST_OPERATION:
|
case ADD_COLUMN_FAMILY_POST_OPERATION:
|
||||||
cpHost.postCompletedAddColumnFamilyAction(tableName, cfDescriptor, user);
|
cpHost.postCompletedAddColumnFamilyAction(tableName, cfDescriptor, getUser());
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||||
|
|
|
@ -47,12 +47,10 @@ import org.apache.hadoop.hbase.master.MetricsSnapshot;
|
||||||
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure.CreateHdfsRegions;
|
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure.CreateHdfsRegions;
|
||||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CloneSnapshotState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CloneSnapshotState;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
|
||||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
|
@ -67,11 +65,9 @@ import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class CloneSnapshotProcedure
|
public class CloneSnapshotProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, CloneSnapshotState>
|
extends AbstractStateMachineTableProcedure<CloneSnapshotState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(CloneSnapshotProcedure.class);
|
private static final Log LOG = LogFactory.getLog(CloneSnapshotProcedure.class);
|
||||||
|
|
||||||
private User user;
|
|
||||||
private HTableDescriptor hTableDescriptor;
|
private HTableDescriptor hTableDescriptor;
|
||||||
private SnapshotDescription snapshot;
|
private SnapshotDescription snapshot;
|
||||||
private List<HRegionInfo> newRegions = null;
|
private List<HRegionInfo> newRegions = null;
|
||||||
|
@ -97,10 +93,9 @@ public class CloneSnapshotProcedure
|
||||||
*/
|
*/
|
||||||
public CloneSnapshotProcedure(final MasterProcedureEnv env,
|
public CloneSnapshotProcedure(final MasterProcedureEnv env,
|
||||||
final HTableDescriptor hTableDescriptor, final SnapshotDescription snapshot) {
|
final HTableDescriptor hTableDescriptor, final SnapshotDescription snapshot) {
|
||||||
|
super(env);
|
||||||
this.hTableDescriptor = hTableDescriptor;
|
this.hTableDescriptor = hTableDescriptor;
|
||||||
this.snapshot = snapshot;
|
this.snapshot = snapshot;
|
||||||
this.user = env.getRequestUser();
|
|
||||||
this.setOwner(this.user.getShortName());
|
|
||||||
|
|
||||||
getMonitorStatus();
|
getMonitorStatus();
|
||||||
}
|
}
|
||||||
|
@ -233,7 +228,7 @@ public class CloneSnapshotProcedure
|
||||||
|
|
||||||
MasterProcedureProtos.CloneSnapshotStateData.Builder cloneSnapshotMsg =
|
MasterProcedureProtos.CloneSnapshotStateData.Builder cloneSnapshotMsg =
|
||||||
MasterProcedureProtos.CloneSnapshotStateData.newBuilder()
|
MasterProcedureProtos.CloneSnapshotStateData.newBuilder()
|
||||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
|
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||||
.setSnapshot(this.snapshot)
|
.setSnapshot(this.snapshot)
|
||||||
.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDescriptor));
|
.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDescriptor));
|
||||||
if (newRegions != null) {
|
if (newRegions != null) {
|
||||||
|
@ -264,7 +259,7 @@ public class CloneSnapshotProcedure
|
||||||
|
|
||||||
MasterProcedureProtos.CloneSnapshotStateData cloneSnapshotMsg =
|
MasterProcedureProtos.CloneSnapshotStateData cloneSnapshotMsg =
|
||||||
MasterProcedureProtos.CloneSnapshotStateData.parseDelimitedFrom(stream);
|
MasterProcedureProtos.CloneSnapshotStateData.parseDelimitedFrom(stream);
|
||||||
user = MasterProcedureUtil.toUserInfo(cloneSnapshotMsg.getUserInfo());
|
setUser(MasterProcedureUtil.toUserInfo(cloneSnapshotMsg.getUserInfo()));
|
||||||
snapshot = cloneSnapshotMsg.getSnapshot();
|
snapshot = cloneSnapshotMsg.getSnapshot();
|
||||||
hTableDescriptor = ProtobufUtil.convertToHTableDesc(cloneSnapshotMsg.getTableSchema());
|
hTableDescriptor = ProtobufUtil.convertToHTableDesc(cloneSnapshotMsg.getTableSchema());
|
||||||
if (cloneSnapshotMsg.getRegionInfoCount() == 0) {
|
if (cloneSnapshotMsg.getRegionInfoCount() == 0) {
|
||||||
|
@ -290,19 +285,6 @@ public class CloneSnapshotProcedure
|
||||||
getMonitorStatus();
|
getMonitorStatus();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected boolean acquireLock(final MasterProcedureEnv env) {
|
|
||||||
if (env.waitInitialized(this)) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, getTableName());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void releaseLock(final MasterProcedureEnv env) {
|
|
||||||
env.getProcedureQueue().releaseTableExclusiveLock(this, getTableName());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Action before any real action of cloning from snapshot.
|
* Action before any real action of cloning from snapshot.
|
||||||
* @param env MasterProcedureEnv
|
* @param env MasterProcedureEnv
|
||||||
|
@ -339,7 +321,7 @@ public class CloneSnapshotProcedure
|
||||||
|
|
||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
cpHost.preCreateTableAction(hTableDescriptor, null, user);
|
cpHost.preCreateTableAction(hTableDescriptor, null, getUser());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -355,7 +337,7 @@ public class CloneSnapshotProcedure
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
final HRegionInfo[] regions = (newRegions == null) ? null :
|
final HRegionInfo[] regions = (newRegions == null) ? null :
|
||||||
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
||||||
cpHost.postCompletedCreateTableAction(hTableDescriptor, regions, user);
|
cpHost.postCompletedCreateTableAction(hTableDescriptor, regions, getUser());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -26,11 +26,9 @@ import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||||
import org.apache.hadoop.hbase.NamespaceExistException;
|
import org.apache.hadoop.hbase.NamespaceExistException;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||||
import org.apache.hadoop.hbase.master.TableNamespaceManager;
|
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.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceState;
|
||||||
|
@ -41,8 +39,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class CreateNamespaceProcedure
|
public class CreateNamespaceProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, CreateNamespaceState>
|
extends AbstractStateMachineNamespaceProcedure<CreateNamespaceState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(CreateNamespaceProcedure.class);
|
private static final Log LOG = LogFactory.getLog(CreateNamespaceProcedure.class);
|
||||||
|
|
||||||
private NamespaceDescriptor nsDescriptor;
|
private NamespaceDescriptor nsDescriptor;
|
||||||
|
@ -54,9 +51,9 @@ public class CreateNamespaceProcedure
|
||||||
|
|
||||||
public CreateNamespaceProcedure(final MasterProcedureEnv env,
|
public CreateNamespaceProcedure(final MasterProcedureEnv env,
|
||||||
final NamespaceDescriptor nsDescriptor) {
|
final NamespaceDescriptor nsDescriptor) {
|
||||||
|
super(env);
|
||||||
this.nsDescriptor = nsDescriptor;
|
this.nsDescriptor = nsDescriptor;
|
||||||
this.traceEnabled = null;
|
this.traceEnabled = null;
|
||||||
this.setOwner(env.getRequestUser().getUGI().getShortUserName());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -157,14 +154,6 @@ public class CreateNamespaceProcedure
|
||||||
nsDescriptor = ProtobufUtil.toNamespaceDescriptor(createNamespaceMsg.getNamespaceDescriptor());
|
nsDescriptor = ProtobufUtil.toNamespaceDescriptor(createNamespaceMsg.getNamespaceDescriptor());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void toStringClassDetails(StringBuilder sb) {
|
|
||||||
sb.append(getClass().getSimpleName());
|
|
||||||
sb.append(" (Namespace=");
|
|
||||||
sb.append(nsDescriptor.getName());
|
|
||||||
sb.append(")");
|
|
||||||
}
|
|
||||||
|
|
||||||
private boolean isBootstrapNamespace() {
|
private boolean isBootstrapNamespace() {
|
||||||
return nsDescriptor.equals(NamespaceDescriptor.DEFAULT_NAMESPACE) ||
|
return nsDescriptor.equals(NamespaceDescriptor.DEFAULT_NAMESPACE) ||
|
||||||
nsDescriptor.equals(NamespaceDescriptor.SYSTEM_NAMESPACE);
|
nsDescriptor.equals(NamespaceDescriptor.SYSTEM_NAMESPACE);
|
||||||
|
@ -183,22 +172,13 @@ public class CreateNamespaceProcedure
|
||||||
return env.getProcedureQueue().tryAcquireNamespaceExclusiveLock(this, getNamespaceName());
|
return env.getProcedureQueue().tryAcquireNamespaceExclusiveLock(this, getNamespaceName());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void releaseLock(final MasterProcedureEnv env) {
|
|
||||||
env.getProcedureQueue().releaseNamespaceExclusiveLock(this, getNamespaceName());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TableName getTableName() {
|
|
||||||
return TableName.NAMESPACE_TABLE_NAME;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TableOperationType getTableOperationType() {
|
public TableOperationType getTableOperationType() {
|
||||||
return TableOperationType.EDIT;
|
return TableOperationType.EDIT;
|
||||||
}
|
}
|
||||||
|
|
||||||
private String getNamespaceName() {
|
@Override
|
||||||
|
protected String getNamespaceName() {
|
||||||
return nsDescriptor.getName();
|
return nsDescriptor.getName();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -41,12 +41,10 @@ import org.apache.hadoop.hbase.client.TableState;
|
||||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
|
||||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
||||||
|
@ -56,20 +54,15 @@ import com.google.common.collect.Lists;
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class CreateTableProcedure
|
public class CreateTableProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, CreateTableState>
|
extends AbstractStateMachineTableProcedure<CreateTableState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(CreateTableProcedure.class);
|
private static final Log LOG = LogFactory.getLog(CreateTableProcedure.class);
|
||||||
|
|
||||||
// used for compatibility with old clients
|
|
||||||
private final ProcedurePrepareLatch syncLatch;
|
|
||||||
|
|
||||||
private HTableDescriptor hTableDescriptor;
|
private HTableDescriptor hTableDescriptor;
|
||||||
private List<HRegionInfo> newRegions;
|
private List<HRegionInfo> newRegions;
|
||||||
private User user;
|
|
||||||
|
|
||||||
public CreateTableProcedure() {
|
public CreateTableProcedure() {
|
||||||
// Required by the Procedure framework to create the procedure on replay
|
// Required by the Procedure framework to create the procedure on replay
|
||||||
syncLatch = null;
|
super();
|
||||||
}
|
}
|
||||||
|
|
||||||
public CreateTableProcedure(final MasterProcedureEnv env,
|
public CreateTableProcedure(final MasterProcedureEnv env,
|
||||||
|
@ -80,14 +73,9 @@ public class CreateTableProcedure
|
||||||
public CreateTableProcedure(final MasterProcedureEnv env,
|
public CreateTableProcedure(final MasterProcedureEnv env,
|
||||||
final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions,
|
final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions,
|
||||||
final ProcedurePrepareLatch syncLatch) {
|
final ProcedurePrepareLatch syncLatch) {
|
||||||
|
super(env, syncLatch);
|
||||||
this.hTableDescriptor = hTableDescriptor;
|
this.hTableDescriptor = hTableDescriptor;
|
||||||
this.newRegions = newRegions != null ? Lists.newArrayList(newRegions) : null;
|
this.newRegions = newRegions != null ? Lists.newArrayList(newRegions) : null;
|
||||||
this.user = env.getRequestUser();
|
|
||||||
this.setOwner(this.user.getShortName());
|
|
||||||
|
|
||||||
// used for compatibility with clients without procedures
|
|
||||||
// they need a sync TableExistsException
|
|
||||||
this.syncLatch = syncLatch;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -101,7 +89,7 @@ public class CreateTableProcedure
|
||||||
case CREATE_TABLE_PRE_OPERATION:
|
case CREATE_TABLE_PRE_OPERATION:
|
||||||
// Verify if we can create the table
|
// Verify if we can create the table
|
||||||
boolean exists = !prepareCreate(env);
|
boolean exists = !prepareCreate(env);
|
||||||
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
|
releaseSyncLatch();
|
||||||
|
|
||||||
if (exists) {
|
if (exists) {
|
||||||
assert isFailed() : "the delete should have an exception here";
|
assert isFailed() : "the delete should have an exception here";
|
||||||
|
@ -151,7 +139,7 @@ public class CreateTableProcedure
|
||||||
// We can fail if the table does exist or the descriptor is malformed.
|
// We can fail if the table does exist or the descriptor is malformed.
|
||||||
// TODO: coprocessor rollback semantic is still undefined.
|
// TODO: coprocessor rollback semantic is still undefined.
|
||||||
DeleteTableProcedure.deleteTableStates(env, getTableName());
|
DeleteTableProcedure.deleteTableStates(env, getTableName());
|
||||||
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
|
releaseSyncLatch();
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -194,21 +182,13 @@ public class CreateTableProcedure
|
||||||
return TableOperationType.CREATE;
|
return TableOperationType.CREATE;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void toStringClassDetails(StringBuilder sb) {
|
|
||||||
sb.append(getClass().getSimpleName());
|
|
||||||
sb.append(" (table=");
|
|
||||||
sb.append(getTableName());
|
|
||||||
sb.append(")");
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void serializeStateData(final OutputStream stream) throws IOException {
|
public void serializeStateData(final OutputStream stream) throws IOException {
|
||||||
super.serializeStateData(stream);
|
super.serializeStateData(stream);
|
||||||
|
|
||||||
MasterProcedureProtos.CreateTableStateData.Builder state =
|
MasterProcedureProtos.CreateTableStateData.Builder state =
|
||||||
MasterProcedureProtos.CreateTableStateData.newBuilder()
|
MasterProcedureProtos.CreateTableStateData.newBuilder()
|
||||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
|
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||||
.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDescriptor));
|
.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDescriptor));
|
||||||
if (newRegions != null) {
|
if (newRegions != null) {
|
||||||
for (HRegionInfo hri: newRegions) {
|
for (HRegionInfo hri: newRegions) {
|
||||||
|
@ -224,7 +204,7 @@ public class CreateTableProcedure
|
||||||
|
|
||||||
MasterProcedureProtos.CreateTableStateData state =
|
MasterProcedureProtos.CreateTableStateData state =
|
||||||
MasterProcedureProtos.CreateTableStateData.parseDelimitedFrom(stream);
|
MasterProcedureProtos.CreateTableStateData.parseDelimitedFrom(stream);
|
||||||
user = MasterProcedureUtil.toUserInfo(state.getUserInfo());
|
setUser(MasterProcedureUtil.toUserInfo(state.getUserInfo()));
|
||||||
hTableDescriptor = ProtobufUtil.convertToHTableDesc(state.getTableSchema());
|
hTableDescriptor = ProtobufUtil.convertToHTableDesc(state.getTableSchema());
|
||||||
if (state.getRegionInfoCount() == 0) {
|
if (state.getRegionInfoCount() == 0) {
|
||||||
newRegions = null;
|
newRegions = null;
|
||||||
|
@ -244,11 +224,6 @@ public class CreateTableProcedure
|
||||||
return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, getTableName());
|
return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, getTableName());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void releaseLock(final MasterProcedureEnv env) {
|
|
||||||
env.getProcedureQueue().releaseTableExclusiveLock(this, getTableName());
|
|
||||||
}
|
|
||||||
|
|
||||||
private boolean prepareCreate(final MasterProcedureEnv env) throws IOException {
|
private boolean prepareCreate(final MasterProcedureEnv env) throws IOException {
|
||||||
final TableName tableName = getTableName();
|
final TableName tableName = getTableName();
|
||||||
if (MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
|
if (MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
|
||||||
|
@ -278,7 +253,7 @@ public class CreateTableProcedure
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
final HRegionInfo[] regions = newRegions == null ? null :
|
final HRegionInfo[] regions = newRegions == null ? null :
|
||||||
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
||||||
cpHost.preCreateTableAction(hTableDescriptor, regions, user);
|
cpHost.preCreateTableAction(hTableDescriptor, regions, getUser());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -288,7 +263,7 @@ public class CreateTableProcedure
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
final HRegionInfo[] regions = (newRegions == null) ? null :
|
final HRegionInfo[] regions = (newRegions == null) ? null :
|
||||||
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
newRegions.toArray(new HRegionInfo[newRegions.size()]);
|
||||||
cpHost.postCompletedCreateTableAction(hTableDescriptor, regions, user);
|
cpHost.postCompletedCreateTableAction(hTableDescriptor, regions, getUser());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -33,11 +33,9 @@ import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.TableState;
|
import org.apache.hadoop.hbase.client.TableState;
|
||||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyState;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
|
||||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
|
||||||
|
@ -46,27 +44,22 @@ import org.apache.hadoop.hbase.util.Bytes;
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class DeleteColumnFamilyProcedure
|
public class DeleteColumnFamilyProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, DeleteColumnFamilyState>
|
extends AbstractStateMachineTableProcedure<DeleteColumnFamilyState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(DeleteColumnFamilyProcedure.class);
|
private static final Log LOG = LogFactory.getLog(DeleteColumnFamilyProcedure.class);
|
||||||
|
|
||||||
private HTableDescriptor unmodifiedHTableDescriptor;
|
private HTableDescriptor unmodifiedHTableDescriptor;
|
||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
private byte [] familyName;
|
private byte [] familyName;
|
||||||
private boolean hasMob;
|
private boolean hasMob;
|
||||||
private User user;
|
|
||||||
|
|
||||||
private List<HRegionInfo> regionInfoList;
|
private List<HRegionInfo> regionInfoList;
|
||||||
private Boolean traceEnabled;
|
private Boolean traceEnabled;
|
||||||
|
|
||||||
// used for compatibility with old clients, until 2.0 the client had a sync behavior
|
|
||||||
private final ProcedurePrepareLatch syncLatch;
|
|
||||||
|
|
||||||
public DeleteColumnFamilyProcedure() {
|
public DeleteColumnFamilyProcedure() {
|
||||||
|
super();
|
||||||
this.unmodifiedHTableDescriptor = null;
|
this.unmodifiedHTableDescriptor = null;
|
||||||
this.regionInfoList = null;
|
this.regionInfoList = null;
|
||||||
this.traceEnabled = null;
|
this.traceEnabled = null;
|
||||||
this.syncLatch = null;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public DeleteColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
public DeleteColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||||
|
@ -76,14 +69,12 @@ public class DeleteColumnFamilyProcedure
|
||||||
|
|
||||||
public DeleteColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
public DeleteColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||||
final byte[] familyName, final ProcedurePrepareLatch latch) {
|
final byte[] familyName, final ProcedurePrepareLatch latch) {
|
||||||
|
super(env, latch);
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.familyName = familyName;
|
this.familyName = familyName;
|
||||||
this.user = env.getRequestUser();
|
|
||||||
this.setOwner(this.user.getShortName());
|
|
||||||
this.unmodifiedHTableDescriptor = null;
|
this.unmodifiedHTableDescriptor = null;
|
||||||
this.regionInfoList = null;
|
this.regionInfoList = null;
|
||||||
this.traceEnabled = null;
|
this.traceEnabled = null;
|
||||||
this.syncLatch = latch;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -160,7 +151,7 @@ public class DeleteColumnFamilyProcedure
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void completionCleanup(final MasterProcedureEnv env) {
|
protected void completionCleanup(final MasterProcedureEnv env) {
|
||||||
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
|
releaseSyncLatch();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -178,24 +169,13 @@ public class DeleteColumnFamilyProcedure
|
||||||
return DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_PREPARE;
|
return DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_PREPARE;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected boolean acquireLock(final MasterProcedureEnv env) {
|
|
||||||
if (env.waitInitialized(this)) return false;
|
|
||||||
return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void releaseLock(final MasterProcedureEnv env) {
|
|
||||||
env.getProcedureQueue().releaseTableExclusiveLock(this, tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void serializeStateData(final OutputStream stream) throws IOException {
|
public void serializeStateData(final OutputStream stream) throws IOException {
|
||||||
super.serializeStateData(stream);
|
super.serializeStateData(stream);
|
||||||
|
|
||||||
MasterProcedureProtos.DeleteColumnFamilyStateData.Builder deleteCFMsg =
|
MasterProcedureProtos.DeleteColumnFamilyStateData.Builder deleteCFMsg =
|
||||||
MasterProcedureProtos.DeleteColumnFamilyStateData.newBuilder()
|
MasterProcedureProtos.DeleteColumnFamilyStateData.newBuilder()
|
||||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
|
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||||
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
||||||
.setColumnfamilyName(ByteStringer.wrap(familyName));
|
.setColumnfamilyName(ByteStringer.wrap(familyName));
|
||||||
if (unmodifiedHTableDescriptor != null) {
|
if (unmodifiedHTableDescriptor != null) {
|
||||||
|
@ -211,7 +191,7 @@ public class DeleteColumnFamilyProcedure
|
||||||
super.deserializeStateData(stream);
|
super.deserializeStateData(stream);
|
||||||
MasterProcedureProtos.DeleteColumnFamilyStateData deleteCFMsg =
|
MasterProcedureProtos.DeleteColumnFamilyStateData deleteCFMsg =
|
||||||
MasterProcedureProtos.DeleteColumnFamilyStateData.parseDelimitedFrom(stream);
|
MasterProcedureProtos.DeleteColumnFamilyStateData.parseDelimitedFrom(stream);
|
||||||
user = MasterProcedureUtil.toUserInfo(deleteCFMsg.getUserInfo());
|
setUser(MasterProcedureUtil.toUserInfo(deleteCFMsg.getUserInfo()));
|
||||||
tableName = ProtobufUtil.toTableName(deleteCFMsg.getTableName());
|
tableName = ProtobufUtil.toTableName(deleteCFMsg.getTableName());
|
||||||
familyName = deleteCFMsg.getColumnfamilyName().toByteArray();
|
familyName = deleteCFMsg.getColumnfamilyName().toByteArray();
|
||||||
|
|
||||||
|
@ -251,7 +231,7 @@ public class DeleteColumnFamilyProcedure
|
||||||
*/
|
*/
|
||||||
private void prepareDelete(final MasterProcedureEnv env) throws IOException {
|
private void prepareDelete(final MasterProcedureEnv env) throws IOException {
|
||||||
// Checks whether the table is allowed to be modified.
|
// Checks whether the table is allowed to be modified.
|
||||||
MasterDDLOperationHelper.checkTableModifiable(env, tableName);
|
checkTableModifiable(env);
|
||||||
|
|
||||||
// In order to update the descriptor, we need to retrieve the old descriptor for comparison.
|
// In order to update the descriptor, we need to retrieve the old descriptor for comparison.
|
||||||
unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
|
unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||||
|
@ -384,10 +364,10 @@ public class DeleteColumnFamilyProcedure
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
switch (state) {
|
switch (state) {
|
||||||
case DELETE_COLUMN_FAMILY_PRE_OPERATION:
|
case DELETE_COLUMN_FAMILY_PRE_OPERATION:
|
||||||
cpHost.preDeleteColumnFamilyAction(tableName, familyName, user);
|
cpHost.preDeleteColumnFamilyAction(tableName, familyName, getUser());
|
||||||
break;
|
break;
|
||||||
case DELETE_COLUMN_FAMILY_POST_OPERATION:
|
case DELETE_COLUMN_FAMILY_POST_OPERATION:
|
||||||
cpHost.postCompletedDeleteColumnFamilyAction(tableName, familyName, user);
|
cpHost.postCompletedDeleteColumnFamilyAction(tableName, familyName, getUser());
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||||
|
|
|
@ -31,12 +31,10 @@ import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||||
import org.apache.hadoop.hbase.NamespaceNotFoundException;
|
import org.apache.hadoop.hbase.NamespaceNotFoundException;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.constraint.ConstraintException;
|
import org.apache.hadoop.hbase.constraint.ConstraintException;
|
||||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||||
import org.apache.hadoop.hbase.master.TableNamespaceManager;
|
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.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteNamespaceState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteNamespaceState;
|
||||||
|
@ -47,8 +45,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class DeleteNamespaceProcedure
|
public class DeleteNamespaceProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, DeleteNamespaceState>
|
extends AbstractStateMachineNamespaceProcedure<DeleteNamespaceState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(DeleteNamespaceProcedure.class);
|
private static final Log LOG = LogFactory.getLog(DeleteNamespaceProcedure.class);
|
||||||
|
|
||||||
private NamespaceDescriptor nsDescriptor;
|
private NamespaceDescriptor nsDescriptor;
|
||||||
|
@ -61,10 +58,10 @@ public class DeleteNamespaceProcedure
|
||||||
}
|
}
|
||||||
|
|
||||||
public DeleteNamespaceProcedure(final MasterProcedureEnv env, final String namespaceName) {
|
public DeleteNamespaceProcedure(final MasterProcedureEnv env, final String namespaceName) {
|
||||||
|
super(env);
|
||||||
this.namespaceName = namespaceName;
|
this.namespaceName = namespaceName;
|
||||||
this.nsDescriptor = null;
|
this.nsDescriptor = null;
|
||||||
this.traceEnabled = null;
|
this.traceEnabled = null;
|
||||||
this.setOwner(env.getRequestUser().getUGI().getShortUserName());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -175,36 +172,13 @@ public class DeleteNamespaceProcedure
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@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) {
|
|
||||||
if (env.waitInitialized(this)) return false;
|
|
||||||
return env.getProcedureQueue().tryAcquireNamespaceExclusiveLock(this, getNamespaceName());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void releaseLock(final MasterProcedureEnv env) {
|
|
||||||
env.getProcedureQueue().releaseNamespaceExclusiveLock(this, getNamespaceName());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TableName getTableName() {
|
|
||||||
return TableName.NAMESPACE_TABLE_NAME;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TableOperationType getTableOperationType() {
|
public TableOperationType getTableOperationType() {
|
||||||
return TableOperationType.EDIT;
|
return TableOperationType.EDIT;
|
||||||
}
|
}
|
||||||
|
|
||||||
private String getNamespaceName() {
|
@Override
|
||||||
|
protected String getNamespaceName() {
|
||||||
return namespaceName;
|
return namespaceName;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -49,31 +49,24 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class DeleteTableProcedure
|
public class DeleteTableProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, DeleteTableState>
|
extends AbstractStateMachineTableProcedure<DeleteTableState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(DeleteTableProcedure.class);
|
private static final Log LOG = LogFactory.getLog(DeleteTableProcedure.class);
|
||||||
|
|
||||||
private List<HRegionInfo> regions;
|
private List<HRegionInfo> regions;
|
||||||
private User user;
|
|
||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
|
|
||||||
// used for compatibility with old clients
|
|
||||||
private final ProcedurePrepareLatch syncLatch;
|
|
||||||
|
|
||||||
public DeleteTableProcedure() {
|
public DeleteTableProcedure() {
|
||||||
// Required by the Procedure framework to create the procedure on replay
|
// Required by the Procedure framework to create the procedure on replay
|
||||||
syncLatch = null;
|
super();
|
||||||
}
|
}
|
||||||
|
|
||||||
public DeleteTableProcedure(final MasterProcedureEnv env, final TableName tableName) {
|
public DeleteTableProcedure(final MasterProcedureEnv env, final TableName tableName) {
|
||||||
|
@ -82,13 +75,8 @@ public class DeleteTableProcedure
|
||||||
|
|
||||||
public DeleteTableProcedure(final MasterProcedureEnv env, final TableName tableName,
|
public DeleteTableProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||||
final ProcedurePrepareLatch syncLatch) {
|
final ProcedurePrepareLatch syncLatch) {
|
||||||
|
super(env, syncLatch);
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.user = env.getRequestUser();
|
|
||||||
this.setOwner(this.user.getShortName());
|
|
||||||
|
|
||||||
// used for compatibility with clients without procedures
|
|
||||||
// they need a sync TableNotFoundException, TableNotDisabledException, ...
|
|
||||||
this.syncLatch = syncLatch;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -102,7 +90,7 @@ public class DeleteTableProcedure
|
||||||
case DELETE_TABLE_PRE_OPERATION:
|
case DELETE_TABLE_PRE_OPERATION:
|
||||||
// Verify if we can delete the table
|
// Verify if we can delete the table
|
||||||
boolean deletable = prepareDelete(env);
|
boolean deletable = prepareDelete(env);
|
||||||
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
|
releaseSyncLatch();
|
||||||
if (!deletable) {
|
if (!deletable) {
|
||||||
assert isFailed() : "the delete should have an exception here";
|
assert isFailed() : "the delete should have an exception here";
|
||||||
return Flow.NO_MORE_STATE;
|
return Flow.NO_MORE_STATE;
|
||||||
|
@ -163,7 +151,7 @@ public class DeleteTableProcedure
|
||||||
// nothing to rollback, pre-delete is just table-state checks.
|
// nothing to rollback, pre-delete is just table-state checks.
|
||||||
// We can fail if the table does not exist or is not disabled.
|
// We can fail if the table does not exist or is not disabled.
|
||||||
// TODO: coprocessor rollback semantic is still undefined.
|
// TODO: coprocessor rollback semantic is still undefined.
|
||||||
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
|
releaseSyncLatch();
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -206,32 +194,13 @@ public class DeleteTableProcedure
|
||||||
return TableOperationType.DELETE;
|
return TableOperationType.DELETE;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected boolean acquireLock(final MasterProcedureEnv env) {
|
|
||||||
if (env.waitInitialized(this)) return false;
|
|
||||||
return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, getTableName());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void releaseLock(final MasterProcedureEnv env) {
|
|
||||||
env.getProcedureQueue().releaseTableExclusiveLock(this, getTableName());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void toStringClassDetails(StringBuilder sb) {
|
|
||||||
sb.append(getClass().getSimpleName());
|
|
||||||
sb.append(" (table=");
|
|
||||||
sb.append(getTableName());
|
|
||||||
sb.append(")");
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void serializeStateData(final OutputStream stream) throws IOException {
|
public void serializeStateData(final OutputStream stream) throws IOException {
|
||||||
super.serializeStateData(stream);
|
super.serializeStateData(stream);
|
||||||
|
|
||||||
MasterProcedureProtos.DeleteTableStateData.Builder state =
|
MasterProcedureProtos.DeleteTableStateData.Builder state =
|
||||||
MasterProcedureProtos.DeleteTableStateData.newBuilder()
|
MasterProcedureProtos.DeleteTableStateData.newBuilder()
|
||||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
|
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||||
.setTableName(ProtobufUtil.toProtoTableName(tableName));
|
.setTableName(ProtobufUtil.toProtoTableName(tableName));
|
||||||
if (regions != null) {
|
if (regions != null) {
|
||||||
for (HRegionInfo hri: regions) {
|
for (HRegionInfo hri: regions) {
|
||||||
|
@ -247,7 +216,7 @@ public class DeleteTableProcedure
|
||||||
|
|
||||||
MasterProcedureProtos.DeleteTableStateData state =
|
MasterProcedureProtos.DeleteTableStateData state =
|
||||||
MasterProcedureProtos.DeleteTableStateData.parseDelimitedFrom(stream);
|
MasterProcedureProtos.DeleteTableStateData.parseDelimitedFrom(stream);
|
||||||
user = MasterProcedureUtil.toUserInfo(state.getUserInfo());
|
setUser(MasterProcedureUtil.toUserInfo(state.getUserInfo()));
|
||||||
tableName = ProtobufUtil.toTableName(state.getTableName());
|
tableName = ProtobufUtil.toTableName(state.getTableName());
|
||||||
if (state.getRegionInfoCount() == 0) {
|
if (state.getRegionInfoCount() == 0) {
|
||||||
regions = null;
|
regions = null;
|
||||||
|
@ -274,7 +243,7 @@ public class DeleteTableProcedure
|
||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
final TableName tableName = this.tableName;
|
final TableName tableName = this.tableName;
|
||||||
cpHost.preDeleteTableAction(tableName, user);
|
cpHost.preDeleteTableAction(tableName, getUser());
|
||||||
}
|
}
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -286,7 +255,7 @@ public class DeleteTableProcedure
|
||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
final TableName tableName = this.tableName;
|
final TableName tableName = this.tableName;
|
||||||
cpHost.postCompletedDeleteTableAction(tableName, user);
|
cpHost.postCompletedDeleteTableAction(tableName, getUser());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -41,27 +41,20 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.master.RegionState;
|
import org.apache.hadoop.hbase.master.RegionState;
|
||||||
import org.apache.hadoop.hbase.master.RegionStates;
|
import org.apache.hadoop.hbase.master.RegionStates;
|
||||||
import org.apache.hadoop.hbase.master.TableStateManager;
|
import org.apache.hadoop.hbase.master.TableStateManager;
|
||||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.htrace.Trace;
|
import org.apache.htrace.Trace;
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class DisableTableProcedure
|
public class DisableTableProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, DisableTableState>
|
extends AbstractStateMachineTableProcedure<DisableTableState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(DisableTableProcedure.class);
|
private static final Log LOG = LogFactory.getLog(DisableTableProcedure.class);
|
||||||
|
|
||||||
// This is for back compatible with 1.0 asynchronized operations.
|
|
||||||
private final ProcedurePrepareLatch syncLatch;
|
|
||||||
|
|
||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
private boolean skipTableStateCheck;
|
private boolean skipTableStateCheck;
|
||||||
private User user;
|
|
||||||
|
|
||||||
private Boolean traceEnabled = null;
|
private Boolean traceEnabled = null;
|
||||||
|
|
||||||
|
@ -72,7 +65,7 @@ public class DisableTableProcedure
|
||||||
}
|
}
|
||||||
|
|
||||||
public DisableTableProcedure() {
|
public DisableTableProcedure() {
|
||||||
syncLatch = null;
|
super();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -94,20 +87,9 @@ public class DisableTableProcedure
|
||||||
*/
|
*/
|
||||||
public DisableTableProcedure(final MasterProcedureEnv env, final TableName tableName,
|
public DisableTableProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||||
final boolean skipTableStateCheck, final ProcedurePrepareLatch syncLatch) {
|
final boolean skipTableStateCheck, final ProcedurePrepareLatch syncLatch) {
|
||||||
|
super(env, syncLatch);
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.skipTableStateCheck = skipTableStateCheck;
|
this.skipTableStateCheck = skipTableStateCheck;
|
||||||
this.user = env.getRequestUser();
|
|
||||||
this.setOwner(this.user.getShortName());
|
|
||||||
|
|
||||||
// Compatible with 1.0: We use latch to make sure that this procedure implementation is
|
|
||||||
// compatible with 1.0 asynchronized operations. We need to lock the table and check
|
|
||||||
// whether the Disable operation could be performed (table exists and online; table state
|
|
||||||
// is ENABLED). Once it is done, we are good to release the latch and the client can
|
|
||||||
// start asynchronously wait for the operation.
|
|
||||||
//
|
|
||||||
// Note: the member syncLatch could be null if we are in failover or recovery scenario.
|
|
||||||
// This is ok for backward compatible, as 1.0 client would not able to peek at procedure.
|
|
||||||
this.syncLatch = syncLatch;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -173,7 +155,7 @@ public class DisableTableProcedure
|
||||||
case DISABLE_TABLE_PRE_OPERATION:
|
case DISABLE_TABLE_PRE_OPERATION:
|
||||||
return;
|
return;
|
||||||
case DISABLE_TABLE_PREPARE:
|
case DISABLE_TABLE_PREPARE:
|
||||||
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
|
releaseSyncLatch();
|
||||||
return;
|
return;
|
||||||
default:
|
default:
|
||||||
break;
|
break;
|
||||||
|
@ -209,24 +191,13 @@ public class DisableTableProcedure
|
||||||
return DisableTableState.DISABLE_TABLE_PREPARE;
|
return DisableTableState.DISABLE_TABLE_PREPARE;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected boolean acquireLock(final MasterProcedureEnv env) {
|
|
||||||
if (env.waitInitialized(this)) return false;
|
|
||||||
return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void releaseLock(final MasterProcedureEnv env) {
|
|
||||||
env.getProcedureQueue().releaseTableExclusiveLock(this, tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void serializeStateData(final OutputStream stream) throws IOException {
|
public void serializeStateData(final OutputStream stream) throws IOException {
|
||||||
super.serializeStateData(stream);
|
super.serializeStateData(stream);
|
||||||
|
|
||||||
MasterProcedureProtos.DisableTableStateData.Builder disableTableMsg =
|
MasterProcedureProtos.DisableTableStateData.Builder disableTableMsg =
|
||||||
MasterProcedureProtos.DisableTableStateData.newBuilder()
|
MasterProcedureProtos.DisableTableStateData.newBuilder()
|
||||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
|
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||||
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
||||||
.setSkipTableStateCheck(skipTableStateCheck);
|
.setSkipTableStateCheck(skipTableStateCheck);
|
||||||
|
|
||||||
|
@ -239,19 +210,11 @@ public class DisableTableProcedure
|
||||||
|
|
||||||
MasterProcedureProtos.DisableTableStateData disableTableMsg =
|
MasterProcedureProtos.DisableTableStateData disableTableMsg =
|
||||||
MasterProcedureProtos.DisableTableStateData.parseDelimitedFrom(stream);
|
MasterProcedureProtos.DisableTableStateData.parseDelimitedFrom(stream);
|
||||||
user = MasterProcedureUtil.toUserInfo(disableTableMsg.getUserInfo());
|
setUser(MasterProcedureUtil.toUserInfo(disableTableMsg.getUserInfo()));
|
||||||
tableName = ProtobufUtil.toTableName(disableTableMsg.getTableName());
|
tableName = ProtobufUtil.toTableName(disableTableMsg.getTableName());
|
||||||
skipTableStateCheck = disableTableMsg.getSkipTableStateCheck();
|
skipTableStateCheck = disableTableMsg.getSkipTableStateCheck();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void toStringClassDetails(StringBuilder sb) {
|
|
||||||
sb.append(getClass().getSimpleName());
|
|
||||||
sb.append(" (table=");
|
|
||||||
sb.append(tableName);
|
|
||||||
sb.append(")");
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TableName getTableName() {
|
public TableName getTableName() {
|
||||||
return tableName;
|
return tableName;
|
||||||
|
@ -297,7 +260,7 @@ public class DisableTableProcedure
|
||||||
}
|
}
|
||||||
|
|
||||||
// We are done the check. Future actions in this procedure could be done asynchronously.
|
// We are done the check. Future actions in this procedure could be done asynchronously.
|
||||||
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
|
releaseSyncLatch();
|
||||||
|
|
||||||
return canTableBeDisabled;
|
return canTableBeDisabled;
|
||||||
}
|
}
|
||||||
|
@ -457,10 +420,10 @@ public class DisableTableProcedure
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
switch (state) {
|
switch (state) {
|
||||||
case DISABLE_TABLE_PRE_OPERATION:
|
case DISABLE_TABLE_PRE_OPERATION:
|
||||||
cpHost.preDisableTableAction(tableName, user);
|
cpHost.preDisableTableAction(tableName, getUser());
|
||||||
break;
|
break;
|
||||||
case DISABLE_TABLE_POST_OPERATION:
|
case DISABLE_TABLE_POST_OPERATION:
|
||||||
cpHost.postCompletedDisableTableAction(tableName, user);
|
cpHost.postCompletedDisableTableAction(tableName, getUser());
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||||
|
|
|
@ -42,11 +42,9 @@ import org.apache.hadoop.hbase.master.RegionPlan;
|
||||||
import org.apache.hadoop.hbase.master.RegionState;
|
import org.apache.hadoop.hbase.master.RegionState;
|
||||||
import org.apache.hadoop.hbase.master.RegionStates;
|
import org.apache.hadoop.hbase.master.RegionStates;
|
||||||
import org.apache.hadoop.hbase.master.ServerManager;
|
import org.apache.hadoop.hbase.master.ServerManager;
|
||||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsState;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -54,8 +52,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class DispatchMergingRegionsProcedure
|
public class DispatchMergingRegionsProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, DispatchMergingRegionsState>
|
extends AbstractStateMachineTableProcedure<DispatchMergingRegionsState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(DispatchMergingRegionsProcedure.class);
|
private static final Log LOG = LogFactory.getLog(DispatchMergingRegionsProcedure.class);
|
||||||
|
|
||||||
private final AtomicBoolean aborted = new AtomicBoolean(false);
|
private final AtomicBoolean aborted = new AtomicBoolean(false);
|
||||||
|
@ -66,7 +63,6 @@ implements TableProcedureInterface {
|
||||||
private String regionsToMergeListFullName;
|
private String regionsToMergeListFullName;
|
||||||
private String regionsToMergeListEncodedName;
|
private String regionsToMergeListEncodedName;
|
||||||
|
|
||||||
private User user;
|
|
||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
private HRegionInfo [] regionsToMerge;
|
private HRegionInfo [] regionsToMerge;
|
||||||
private boolean forcible;
|
private boolean forcible;
|
||||||
|
@ -85,6 +81,7 @@ implements TableProcedureInterface {
|
||||||
final TableName tableName,
|
final TableName tableName,
|
||||||
final HRegionInfo [] regionsToMerge,
|
final HRegionInfo [] regionsToMerge,
|
||||||
final boolean forcible) {
|
final boolean forcible) {
|
||||||
|
super(env);
|
||||||
this.traceEnabled = isTraceEnabled();
|
this.traceEnabled = isTraceEnabled();
|
||||||
this.assignmentManager = getAssignmentManager(env);
|
this.assignmentManager = getAssignmentManager(env);
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
|
@ -94,9 +91,6 @@ implements TableProcedureInterface {
|
||||||
this.regionsToMerge = regionsToMerge;
|
this.regionsToMerge = regionsToMerge;
|
||||||
this.forcible = forcible;
|
this.forcible = forcible;
|
||||||
|
|
||||||
this.user = env.getRequestUser();
|
|
||||||
this.setOwner(this.user.getShortName());
|
|
||||||
|
|
||||||
this.timeout = -1;
|
this.timeout = -1;
|
||||||
this.regionsToMergeListFullName = getRegionsToMergeListFullNameString();
|
this.regionsToMergeListFullName = getRegionsToMergeListFullNameString();
|
||||||
this.regionsToMergeListEncodedName = getRegionsToMergeListEncodedNameString();
|
this.regionsToMergeListEncodedName = getRegionsToMergeListEncodedNameString();
|
||||||
|
@ -220,7 +214,7 @@ implements TableProcedureInterface {
|
||||||
|
|
||||||
MasterProcedureProtos.DispatchMergingRegionsStateData.Builder dispatchMergingRegionsMsg =
|
MasterProcedureProtos.DispatchMergingRegionsStateData.Builder dispatchMergingRegionsMsg =
|
||||||
MasterProcedureProtos.DispatchMergingRegionsStateData.newBuilder()
|
MasterProcedureProtos.DispatchMergingRegionsStateData.newBuilder()
|
||||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
|
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||||
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
||||||
.setForcible(forcible);
|
.setForcible(forcible);
|
||||||
for (HRegionInfo hri: regionsToMerge) {
|
for (HRegionInfo hri: regionsToMerge) {
|
||||||
|
@ -235,7 +229,7 @@ implements TableProcedureInterface {
|
||||||
|
|
||||||
MasterProcedureProtos.DispatchMergingRegionsStateData dispatchMergingRegionsMsg =
|
MasterProcedureProtos.DispatchMergingRegionsStateData dispatchMergingRegionsMsg =
|
||||||
MasterProcedureProtos.DispatchMergingRegionsStateData.parseDelimitedFrom(stream);
|
MasterProcedureProtos.DispatchMergingRegionsStateData.parseDelimitedFrom(stream);
|
||||||
user = MasterProcedureUtil.toUserInfo(dispatchMergingRegionsMsg.getUserInfo());
|
setUser(MasterProcedureUtil.toUserInfo(dispatchMergingRegionsMsg.getUserInfo()));
|
||||||
tableName = ProtobufUtil.toTableName(dispatchMergingRegionsMsg.getTableName());
|
tableName = ProtobufUtil.toTableName(dispatchMergingRegionsMsg.getTableName());
|
||||||
|
|
||||||
assert(dispatchMergingRegionsMsg.getRegionInfoCount() == 2);
|
assert(dispatchMergingRegionsMsg.getRegionInfoCount() == 2);
|
||||||
|
@ -419,7 +413,7 @@ implements TableProcedureInterface {
|
||||||
regionsToMerge[0],
|
regionsToMerge[0],
|
||||||
regionsToMerge[1],
|
regionsToMerge[1],
|
||||||
forcible,
|
forcible,
|
||||||
user);
|
getUser());
|
||||||
LOG.info("Sent merge to server " + getServerName(env) + " for region " +
|
LOG.info("Sent merge to server " + getServerName(env) + " for region " +
|
||||||
getRegionsToMergeListEncodedNameString() + ", focible=" + forcible);
|
getRegionsToMergeListEncodedNameString() + ", focible=" + forcible);
|
||||||
return;
|
return;
|
||||||
|
|
|
@ -45,31 +45,24 @@ import org.apache.hadoop.hbase.master.MasterServices;
|
||||||
import org.apache.hadoop.hbase.master.RegionStates;
|
import org.apache.hadoop.hbase.master.RegionStates;
|
||||||
import org.apache.hadoop.hbase.master.ServerManager;
|
import org.apache.hadoop.hbase.master.ServerManager;
|
||||||
import org.apache.hadoop.hbase.master.TableStateManager;
|
import org.apache.hadoop.hbase.master.TableStateManager;
|
||||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class EnableTableProcedure
|
public class EnableTableProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, EnableTableState>
|
extends AbstractStateMachineTableProcedure<EnableTableState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(EnableTableProcedure.class);
|
private static final Log LOG = LogFactory.getLog(EnableTableProcedure.class);
|
||||||
|
|
||||||
// This is for back compatible with 1.0 asynchronized operations.
|
|
||||||
private final ProcedurePrepareLatch syncLatch;
|
|
||||||
|
|
||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
private boolean skipTableStateCheck;
|
private boolean skipTableStateCheck;
|
||||||
private User user;
|
|
||||||
|
|
||||||
private Boolean traceEnabled = null;
|
private Boolean traceEnabled = null;
|
||||||
|
|
||||||
public EnableTableProcedure() {
|
public EnableTableProcedure() {
|
||||||
syncLatch = null;
|
super();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -91,20 +84,9 @@ public class EnableTableProcedure
|
||||||
*/
|
*/
|
||||||
public EnableTableProcedure(final MasterProcedureEnv env, final TableName tableName,
|
public EnableTableProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||||
final boolean skipTableStateCheck, final ProcedurePrepareLatch syncLatch) {
|
final boolean skipTableStateCheck, final ProcedurePrepareLatch syncLatch) {
|
||||||
|
super(env, syncLatch);
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.skipTableStateCheck = skipTableStateCheck;
|
this.skipTableStateCheck = skipTableStateCheck;
|
||||||
this.user = env.getRequestUser();
|
|
||||||
this.setOwner(this.user.getShortName());
|
|
||||||
|
|
||||||
// Compatible with 1.0: We use latch to make sure that this procedure implementation is
|
|
||||||
// compatible with 1.0 asynchronized operations. We need to lock the table and check
|
|
||||||
// whether the Enable operation could be performed (table exists and offline; table state
|
|
||||||
// is DISABLED). Once it is done, we are good to release the latch and the client can
|
|
||||||
// start asynchronously wait for the operation.
|
|
||||||
//
|
|
||||||
// Note: the member syncLatch could be null if we are in failover or recovery scenario.
|
|
||||||
// This is ok for backward compatible, as 1.0 client would not able to peek at procedure.
|
|
||||||
this.syncLatch = syncLatch;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -166,7 +148,7 @@ public class EnableTableProcedure
|
||||||
case ENABLE_TABLE_PRE_OPERATION:
|
case ENABLE_TABLE_PRE_OPERATION:
|
||||||
return;
|
return;
|
||||||
case ENABLE_TABLE_PREPARE:
|
case ENABLE_TABLE_PREPARE:
|
||||||
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
|
releaseSyncLatch();
|
||||||
return;
|
return;
|
||||||
default:
|
default:
|
||||||
break;
|
break;
|
||||||
|
@ -202,24 +184,13 @@ public class EnableTableProcedure
|
||||||
return EnableTableState.ENABLE_TABLE_PREPARE;
|
return EnableTableState.ENABLE_TABLE_PREPARE;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected boolean acquireLock(final MasterProcedureEnv env) {
|
|
||||||
if (env.waitInitialized(this)) return false;
|
|
||||||
return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void releaseLock(final MasterProcedureEnv env) {
|
|
||||||
env.getProcedureQueue().releaseTableExclusiveLock(this, tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void serializeStateData(final OutputStream stream) throws IOException {
|
public void serializeStateData(final OutputStream stream) throws IOException {
|
||||||
super.serializeStateData(stream);
|
super.serializeStateData(stream);
|
||||||
|
|
||||||
MasterProcedureProtos.EnableTableStateData.Builder enableTableMsg =
|
MasterProcedureProtos.EnableTableStateData.Builder enableTableMsg =
|
||||||
MasterProcedureProtos.EnableTableStateData.newBuilder()
|
MasterProcedureProtos.EnableTableStateData.newBuilder()
|
||||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
|
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||||
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
||||||
.setSkipTableStateCheck(skipTableStateCheck);
|
.setSkipTableStateCheck(skipTableStateCheck);
|
||||||
|
|
||||||
|
@ -232,19 +203,11 @@ public class EnableTableProcedure
|
||||||
|
|
||||||
MasterProcedureProtos.EnableTableStateData enableTableMsg =
|
MasterProcedureProtos.EnableTableStateData enableTableMsg =
|
||||||
MasterProcedureProtos.EnableTableStateData.parseDelimitedFrom(stream);
|
MasterProcedureProtos.EnableTableStateData.parseDelimitedFrom(stream);
|
||||||
user = MasterProcedureUtil.toUserInfo(enableTableMsg.getUserInfo());
|
setUser(MasterProcedureUtil.toUserInfo(enableTableMsg.getUserInfo()));
|
||||||
tableName = ProtobufUtil.toTableName(enableTableMsg.getTableName());
|
tableName = ProtobufUtil.toTableName(enableTableMsg.getTableName());
|
||||||
skipTableStateCheck = enableTableMsg.getSkipTableStateCheck();
|
skipTableStateCheck = enableTableMsg.getSkipTableStateCheck();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void toStringClassDetails(StringBuilder sb) {
|
|
||||||
sb.append(getClass().getSimpleName());
|
|
||||||
sb.append(" (table=");
|
|
||||||
sb.append(tableName);
|
|
||||||
sb.append(")");
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TableName getTableName() {
|
public TableName getTableName() {
|
||||||
return tableName;
|
return tableName;
|
||||||
|
@ -291,7 +254,7 @@ public class EnableTableProcedure
|
||||||
}
|
}
|
||||||
|
|
||||||
// We are done the check. Future actions in this procedure could be done asynchronously.
|
// We are done the check. Future actions in this procedure could be done asynchronously.
|
||||||
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
|
releaseSyncLatch();
|
||||||
|
|
||||||
return canTableBeEnabled;
|
return canTableBeEnabled;
|
||||||
}
|
}
|
||||||
|
@ -533,10 +496,10 @@ public class EnableTableProcedure
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
switch (state) {
|
switch (state) {
|
||||||
case ENABLE_TABLE_PRE_OPERATION:
|
case ENABLE_TABLE_PRE_OPERATION:
|
||||||
cpHost.preEnableTableAction(getTableName(), user);
|
cpHost.preEnableTableAction(getTableName(), getUser());
|
||||||
break;
|
break;
|
||||||
case ENABLE_TABLE_POST_OPERATION:
|
case ENABLE_TABLE_POST_OPERATION:
|
||||||
cpHost.postCompletedEnableTableAction(getTableName(), user);
|
cpHost.postCompletedEnableTableAction(getTableName(), getUser());
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||||
|
|
|
@ -29,11 +29,9 @@ import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.HRegionLocation;
|
import org.apache.hadoop.hbase.HRegionLocation;
|
||||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.TableNotDisabledException;
|
import org.apache.hadoop.hbase.TableNotDisabledException;
|
||||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.Connection;
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||||
|
@ -55,20 +53,6 @@ public final class MasterDDLOperationHelper {
|
||||||
|
|
||||||
private MasterDDLOperationHelper() {}
|
private MasterDDLOperationHelper() {}
|
||||||
|
|
||||||
/**
|
|
||||||
* Check whether a table is modifiable - exists and either offline or online with config set
|
|
||||||
* @param env MasterProcedureEnv
|
|
||||||
* @param tableName name of the table
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public static void checkTableModifiable(final MasterProcedureEnv env, final TableName tableName)
|
|
||||||
throws IOException {
|
|
||||||
// Checks whether the table exists
|
|
||||||
if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
|
|
||||||
throw new TableNotFoundException(tableName);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Remove the column family from the file system
|
* Remove the column family from the file system
|
||||||
**/
|
**/
|
||||||
|
|
|
@ -34,35 +34,28 @@ import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.TableState;
|
import org.apache.hadoop.hbase.client.TableState;
|
||||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyState;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The procedure to modify a column family from an existing table.
|
* The procedure to modify a column family from an existing table.
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class ModifyColumnFamilyProcedure
|
public class ModifyColumnFamilyProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, ModifyColumnFamilyState>
|
extends AbstractStateMachineTableProcedure<ModifyColumnFamilyState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(ModifyColumnFamilyProcedure.class);
|
private static final Log LOG = LogFactory.getLog(ModifyColumnFamilyProcedure.class);
|
||||||
|
|
||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
private HTableDescriptor unmodifiedHTableDescriptor;
|
private HTableDescriptor unmodifiedHTableDescriptor;
|
||||||
private HColumnDescriptor cfDescriptor;
|
private HColumnDescriptor cfDescriptor;
|
||||||
private User user;
|
|
||||||
|
|
||||||
private Boolean traceEnabled;
|
private Boolean traceEnabled;
|
||||||
|
|
||||||
// used for compatibility with old clients, until 2.0 the client had a sync behavior
|
|
||||||
private final ProcedurePrepareLatch syncLatch;
|
|
||||||
|
|
||||||
public ModifyColumnFamilyProcedure() {
|
public ModifyColumnFamilyProcedure() {
|
||||||
|
super();
|
||||||
this.unmodifiedHTableDescriptor = null;
|
this.unmodifiedHTableDescriptor = null;
|
||||||
this.traceEnabled = null;
|
this.traceEnabled = null;
|
||||||
this.syncLatch = null;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public ModifyColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
public ModifyColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||||
|
@ -72,13 +65,11 @@ public class ModifyColumnFamilyProcedure
|
||||||
|
|
||||||
public ModifyColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
public ModifyColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||||
final HColumnDescriptor cfDescriptor, final ProcedurePrepareLatch latch) {
|
final HColumnDescriptor cfDescriptor, final ProcedurePrepareLatch latch) {
|
||||||
|
super(env, latch);
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.cfDescriptor = cfDescriptor;
|
this.cfDescriptor = cfDescriptor;
|
||||||
this.user = env.getRequestUser();
|
|
||||||
this.setOwner(this.user.getShortName());
|
|
||||||
this.unmodifiedHTableDescriptor = null;
|
this.unmodifiedHTableDescriptor = null;
|
||||||
this.traceEnabled = null;
|
this.traceEnabled = null;
|
||||||
this.syncLatch = latch;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -150,7 +141,7 @@ public class ModifyColumnFamilyProcedure
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void completionCleanup(final MasterProcedureEnv env) {
|
protected void completionCleanup(final MasterProcedureEnv env) {
|
||||||
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
|
releaseSyncLatch();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -168,24 +159,13 @@ public class ModifyColumnFamilyProcedure
|
||||||
return ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_PREPARE;
|
return ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_PREPARE;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected boolean acquireLock(final MasterProcedureEnv env) {
|
|
||||||
if (env.waitInitialized(this)) return false;
|
|
||||||
return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void releaseLock(final MasterProcedureEnv env) {
|
|
||||||
env.getProcedureQueue().releaseTableExclusiveLock(this, tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void serializeStateData(final OutputStream stream) throws IOException {
|
public void serializeStateData(final OutputStream stream) throws IOException {
|
||||||
super.serializeStateData(stream);
|
super.serializeStateData(stream);
|
||||||
|
|
||||||
MasterProcedureProtos.ModifyColumnFamilyStateData.Builder modifyCFMsg =
|
MasterProcedureProtos.ModifyColumnFamilyStateData.Builder modifyCFMsg =
|
||||||
MasterProcedureProtos.ModifyColumnFamilyStateData.newBuilder()
|
MasterProcedureProtos.ModifyColumnFamilyStateData.newBuilder()
|
||||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
|
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||||
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
||||||
.setColumnfamilySchema(ProtobufUtil.convertToColumnFamilySchema(cfDescriptor));
|
.setColumnfamilySchema(ProtobufUtil.convertToColumnFamilySchema(cfDescriptor));
|
||||||
if (unmodifiedHTableDescriptor != null) {
|
if (unmodifiedHTableDescriptor != null) {
|
||||||
|
@ -202,7 +182,7 @@ public class ModifyColumnFamilyProcedure
|
||||||
|
|
||||||
MasterProcedureProtos.ModifyColumnFamilyStateData modifyCFMsg =
|
MasterProcedureProtos.ModifyColumnFamilyStateData modifyCFMsg =
|
||||||
MasterProcedureProtos.ModifyColumnFamilyStateData.parseDelimitedFrom(stream);
|
MasterProcedureProtos.ModifyColumnFamilyStateData.parseDelimitedFrom(stream);
|
||||||
user = MasterProcedureUtil.toUserInfo(modifyCFMsg.getUserInfo());
|
setUser(MasterProcedureUtil.toUserInfo(modifyCFMsg.getUserInfo()));
|
||||||
tableName = ProtobufUtil.toTableName(modifyCFMsg.getTableName());
|
tableName = ProtobufUtil.toTableName(modifyCFMsg.getTableName());
|
||||||
cfDescriptor = ProtobufUtil.convertToHColumnDesc(modifyCFMsg.getColumnfamilySchema());
|
cfDescriptor = ProtobufUtil.convertToHColumnDesc(modifyCFMsg.getColumnfamilySchema());
|
||||||
if (modifyCFMsg.hasUnmodifiedTableSchema()) {
|
if (modifyCFMsg.hasUnmodifiedTableSchema()) {
|
||||||
|
@ -241,7 +221,7 @@ public class ModifyColumnFamilyProcedure
|
||||||
*/
|
*/
|
||||||
private void prepareModify(final MasterProcedureEnv env) throws IOException {
|
private void prepareModify(final MasterProcedureEnv env) throws IOException {
|
||||||
// Checks whether the table is allowed to be modified.
|
// Checks whether the table is allowed to be modified.
|
||||||
MasterDDLOperationHelper.checkTableModifiable(env, tableName);
|
checkTableModifiable(env);
|
||||||
|
|
||||||
unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
|
unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
|
||||||
if (unmodifiedHTableDescriptor == null) {
|
if (unmodifiedHTableDescriptor == null) {
|
||||||
|
@ -350,10 +330,10 @@ public class ModifyColumnFamilyProcedure
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
switch (state) {
|
switch (state) {
|
||||||
case MODIFY_COLUMN_FAMILY_PRE_OPERATION:
|
case MODIFY_COLUMN_FAMILY_PRE_OPERATION:
|
||||||
cpHost.preModifyColumnFamilyAction(tableName, cfDescriptor, user);
|
cpHost.preModifyColumnFamilyAction(tableName, cfDescriptor, getUser());
|
||||||
break;
|
break;
|
||||||
case MODIFY_COLUMN_FAMILY_POST_OPERATION:
|
case MODIFY_COLUMN_FAMILY_POST_OPERATION:
|
||||||
cpHost.postCompletedModifyColumnFamilyAction(tableName, cfDescriptor, user);
|
cpHost.postCompletedModifyColumnFamilyAction(tableName, cfDescriptor, getUser());
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||||
|
|
|
@ -26,10 +26,8 @@ import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||||
import org.apache.hadoop.hbase.NamespaceNotFoundException;
|
import org.apache.hadoop.hbase.NamespaceNotFoundException;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.master.TableNamespaceManager;
|
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.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceState;
|
||||||
|
@ -39,8 +37,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNa
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class ModifyNamespaceProcedure
|
public class ModifyNamespaceProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, ModifyNamespaceState>
|
extends AbstractStateMachineNamespaceProcedure<ModifyNamespaceState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(ModifyNamespaceProcedure.class);
|
private static final Log LOG = LogFactory.getLog(ModifyNamespaceProcedure.class);
|
||||||
|
|
||||||
private NamespaceDescriptor oldNsDescriptor;
|
private NamespaceDescriptor oldNsDescriptor;
|
||||||
|
@ -54,10 +51,10 @@ public class ModifyNamespaceProcedure
|
||||||
|
|
||||||
public ModifyNamespaceProcedure(final MasterProcedureEnv env,
|
public ModifyNamespaceProcedure(final MasterProcedureEnv env,
|
||||||
final NamespaceDescriptor newNsDescriptor) {
|
final NamespaceDescriptor newNsDescriptor) {
|
||||||
|
super(env);
|
||||||
this.oldNsDescriptor = null;
|
this.oldNsDescriptor = null;
|
||||||
this.newNsDescriptor = newNsDescriptor;
|
this.newNsDescriptor = newNsDescriptor;
|
||||||
this.traceEnabled = null;
|
this.traceEnabled = null;
|
||||||
this.setOwner(env.getRequestUser().getUGI().getShortUserName());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -160,36 +157,13 @@ public class ModifyNamespaceProcedure
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@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) {
|
|
||||||
if (env.waitInitialized(this)) return false;
|
|
||||||
return env.getProcedureQueue().tryAcquireNamespaceExclusiveLock(this, getNamespaceName());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void releaseLock(final MasterProcedureEnv env) {
|
|
||||||
env.getProcedureQueue().releaseNamespaceExclusiveLock(this, getNamespaceName());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TableName getTableName() {
|
|
||||||
return TableName.NAMESPACE_TABLE_NAME;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TableOperationType getTableOperationType() {
|
public TableOperationType getTableOperationType() {
|
||||||
return TableOperationType.EDIT;
|
return TableOperationType.EDIT;
|
||||||
}
|
}
|
||||||
|
|
||||||
private String getNamespaceName() {
|
@Override
|
||||||
|
protected String getNamespaceName() {
|
||||||
return newNsDescriptor.getName();
|
return newNsDescriptor.getName();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -44,33 +44,26 @@ import org.apache.hadoop.hbase.client.Scan;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
import org.apache.hadoop.hbase.client.TableState;
|
import org.apache.hadoop.hbase.client.TableState;
|
||||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableState;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
|
||||||
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
|
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class ModifyTableProcedure
|
public class ModifyTableProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, ModifyTableState>
|
extends AbstractStateMachineTableProcedure<ModifyTableState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(ModifyTableProcedure.class);
|
private static final Log LOG = LogFactory.getLog(ModifyTableProcedure.class);
|
||||||
|
|
||||||
private HTableDescriptor unmodifiedHTableDescriptor = null;
|
private HTableDescriptor unmodifiedHTableDescriptor = null;
|
||||||
private HTableDescriptor modifiedHTableDescriptor;
|
private HTableDescriptor modifiedHTableDescriptor;
|
||||||
private User user;
|
|
||||||
private boolean deleteColumnFamilyInModify;
|
private boolean deleteColumnFamilyInModify;
|
||||||
|
|
||||||
private List<HRegionInfo> regionInfoList;
|
private List<HRegionInfo> regionInfoList;
|
||||||
private Boolean traceEnabled = null;
|
private Boolean traceEnabled = null;
|
||||||
|
|
||||||
// used for compatibility with old clients, until 2.0 the client had a sync behavior
|
|
||||||
private final ProcedurePrepareLatch syncLatch;
|
|
||||||
|
|
||||||
public ModifyTableProcedure() {
|
public ModifyTableProcedure() {
|
||||||
|
super();
|
||||||
initilize();
|
initilize();
|
||||||
this.syncLatch = null;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public ModifyTableProcedure(final MasterProcedureEnv env, final HTableDescriptor htd) {
|
public ModifyTableProcedure(final MasterProcedureEnv env, final HTableDescriptor htd) {
|
||||||
|
@ -79,11 +72,9 @@ public class ModifyTableProcedure
|
||||||
|
|
||||||
public ModifyTableProcedure(final MasterProcedureEnv env, final HTableDescriptor htd,
|
public ModifyTableProcedure(final MasterProcedureEnv env, final HTableDescriptor htd,
|
||||||
final ProcedurePrepareLatch latch) {
|
final ProcedurePrepareLatch latch) {
|
||||||
|
super(env, latch);
|
||||||
initilize();
|
initilize();
|
||||||
this.modifiedHTableDescriptor = htd;
|
this.modifiedHTableDescriptor = htd;
|
||||||
this.user = env.getRequestUser();
|
|
||||||
this.setOwner(this.user.getShortName());
|
|
||||||
this.syncLatch = latch;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void initilize() {
|
private void initilize() {
|
||||||
|
@ -174,7 +165,7 @@ public class ModifyTableProcedure
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void completionCleanup(final MasterProcedureEnv env) {
|
protected void completionCleanup(final MasterProcedureEnv env) {
|
||||||
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
|
releaseSyncLatch();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -192,24 +183,13 @@ public class ModifyTableProcedure
|
||||||
return ModifyTableState.MODIFY_TABLE_PREPARE;
|
return ModifyTableState.MODIFY_TABLE_PREPARE;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected boolean acquireLock(final MasterProcedureEnv env) {
|
|
||||||
if (env.waitInitialized(this)) return false;
|
|
||||||
return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, getTableName());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void releaseLock(final MasterProcedureEnv env) {
|
|
||||||
env.getProcedureQueue().releaseTableExclusiveLock(this, getTableName());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void serializeStateData(final OutputStream stream) throws IOException {
|
public void serializeStateData(final OutputStream stream) throws IOException {
|
||||||
super.serializeStateData(stream);
|
super.serializeStateData(stream);
|
||||||
|
|
||||||
MasterProcedureProtos.ModifyTableStateData.Builder modifyTableMsg =
|
MasterProcedureProtos.ModifyTableStateData.Builder modifyTableMsg =
|
||||||
MasterProcedureProtos.ModifyTableStateData.newBuilder()
|
MasterProcedureProtos.ModifyTableStateData.newBuilder()
|
||||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
|
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||||
.setModifiedTableSchema(ProtobufUtil.convertToTableSchema(modifiedHTableDescriptor))
|
.setModifiedTableSchema(ProtobufUtil.convertToTableSchema(modifiedHTableDescriptor))
|
||||||
.setDeleteColumnFamilyInModify(deleteColumnFamilyInModify);
|
.setDeleteColumnFamilyInModify(deleteColumnFamilyInModify);
|
||||||
|
|
||||||
|
@ -227,7 +207,7 @@ public class ModifyTableProcedure
|
||||||
|
|
||||||
MasterProcedureProtos.ModifyTableStateData modifyTableMsg =
|
MasterProcedureProtos.ModifyTableStateData modifyTableMsg =
|
||||||
MasterProcedureProtos.ModifyTableStateData.parseDelimitedFrom(stream);
|
MasterProcedureProtos.ModifyTableStateData.parseDelimitedFrom(stream);
|
||||||
user = MasterProcedureUtil.toUserInfo(modifyTableMsg.getUserInfo());
|
setUser(MasterProcedureUtil.toUserInfo(modifyTableMsg.getUserInfo()));
|
||||||
modifiedHTableDescriptor = ProtobufUtil.convertToHTableDesc(modifyTableMsg.getModifiedTableSchema());
|
modifiedHTableDescriptor = ProtobufUtil.convertToHTableDesc(modifyTableMsg.getModifiedTableSchema());
|
||||||
deleteColumnFamilyInModify = modifyTableMsg.getDeleteColumnFamilyInModify();
|
deleteColumnFamilyInModify = modifyTableMsg.getDeleteColumnFamilyInModify();
|
||||||
|
|
||||||
|
@ -237,14 +217,6 @@ public class ModifyTableProcedure
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void toStringClassDetails(StringBuilder sb) {
|
|
||||||
sb.append(getClass().getSimpleName());
|
|
||||||
sb.append(" (table=");
|
|
||||||
sb.append(getTableName());
|
|
||||||
sb.append(")");
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TableName getTableName() {
|
public TableName getTableName() {
|
||||||
return modifiedHTableDescriptor.getTableName();
|
return modifiedHTableDescriptor.getTableName();
|
||||||
|
@ -447,10 +419,10 @@ public class ModifyTableProcedure
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
switch (state) {
|
switch (state) {
|
||||||
case MODIFY_TABLE_PRE_OPERATION:
|
case MODIFY_TABLE_PRE_OPERATION:
|
||||||
cpHost.preModifyTableAction(getTableName(), modifiedHTableDescriptor, user);
|
cpHost.preModifyTableAction(getTableName(), modifiedHTableDescriptor, getUser());
|
||||||
break;
|
break;
|
||||||
case MODIFY_TABLE_POST_OPERATION:
|
case MODIFY_TABLE_POST_OPERATION:
|
||||||
cpHost.postCompletedModifyTableAction(getTableName(), modifiedHTableDescriptor, user);
|
cpHost.postCompletedModifyTableAction(getTableName(), modifiedHTableDescriptor,getUser());
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||||
|
|
|
@ -46,13 +46,11 @@ import org.apache.hadoop.hbase.master.MetricsSnapshot;
|
||||||
import org.apache.hadoop.hbase.master.RegionStates;
|
import org.apache.hadoop.hbase.master.RegionStates;
|
||||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.RestoreSnapshotState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.RestoreSnapshotState;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
|
||||||
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
||||||
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
|
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
|
||||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||||
|
@ -61,8 +59,7 @@ import org.apache.hadoop.hbase.util.Pair;
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class RestoreSnapshotProcedure
|
public class RestoreSnapshotProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, RestoreSnapshotState>
|
extends AbstractStateMachineTableProcedure<RestoreSnapshotState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(RestoreSnapshotProcedure.class);
|
private static final Log LOG = LogFactory.getLog(RestoreSnapshotProcedure.class);
|
||||||
|
|
||||||
private HTableDescriptor modifiedHTableDescriptor;
|
private HTableDescriptor modifiedHTableDescriptor;
|
||||||
|
@ -72,7 +69,6 @@ public class RestoreSnapshotProcedure
|
||||||
private Map<String, Pair<String, String>> parentsToChildrenPairMap =
|
private Map<String, Pair<String, String>> parentsToChildrenPairMap =
|
||||||
new HashMap<String, Pair<String, String>>();
|
new HashMap<String, Pair<String, String>>();
|
||||||
|
|
||||||
private User user;
|
|
||||||
private SnapshotDescription snapshot;
|
private SnapshotDescription snapshot;
|
||||||
|
|
||||||
// Monitor
|
// Monitor
|
||||||
|
@ -97,13 +93,11 @@ public class RestoreSnapshotProcedure
|
||||||
final MasterProcedureEnv env,
|
final MasterProcedureEnv env,
|
||||||
final HTableDescriptor hTableDescriptor,
|
final HTableDescriptor hTableDescriptor,
|
||||||
final SnapshotDescription snapshot) {
|
final SnapshotDescription snapshot) {
|
||||||
|
super(env);
|
||||||
// This is the new schema we are going to write out as this modification.
|
// This is the new schema we are going to write out as this modification.
|
||||||
this.modifiedHTableDescriptor = hTableDescriptor;
|
this.modifiedHTableDescriptor = hTableDescriptor;
|
||||||
// Snapshot information
|
// Snapshot information
|
||||||
this.snapshot = snapshot;
|
this.snapshot = snapshot;
|
||||||
// User and owner information
|
|
||||||
this.user = env.getRequestUser();
|
|
||||||
this.setOwner(this.user.getShortName());
|
|
||||||
|
|
||||||
// Monitor
|
// Monitor
|
||||||
getMonitorStatus();
|
getMonitorStatus();
|
||||||
|
@ -231,7 +225,7 @@ public class RestoreSnapshotProcedure
|
||||||
|
|
||||||
MasterProcedureProtos.RestoreSnapshotStateData.Builder restoreSnapshotMsg =
|
MasterProcedureProtos.RestoreSnapshotStateData.Builder restoreSnapshotMsg =
|
||||||
MasterProcedureProtos.RestoreSnapshotStateData.newBuilder()
|
MasterProcedureProtos.RestoreSnapshotStateData.newBuilder()
|
||||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
|
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||||
.setSnapshot(this.snapshot)
|
.setSnapshot(this.snapshot)
|
||||||
.setModifiedTableSchema(ProtobufUtil.convertToTableSchema(modifiedHTableDescriptor));
|
.setModifiedTableSchema(ProtobufUtil.convertToTableSchema(modifiedHTableDescriptor));
|
||||||
|
|
||||||
|
@ -273,7 +267,7 @@ public class RestoreSnapshotProcedure
|
||||||
|
|
||||||
MasterProcedureProtos.RestoreSnapshotStateData restoreSnapshotMsg =
|
MasterProcedureProtos.RestoreSnapshotStateData restoreSnapshotMsg =
|
||||||
MasterProcedureProtos.RestoreSnapshotStateData.parseDelimitedFrom(stream);
|
MasterProcedureProtos.RestoreSnapshotStateData.parseDelimitedFrom(stream);
|
||||||
user = MasterProcedureUtil.toUserInfo(restoreSnapshotMsg.getUserInfo());
|
setUser(MasterProcedureUtil.toUserInfo(restoreSnapshotMsg.getUserInfo()));
|
||||||
snapshot = restoreSnapshotMsg.getSnapshot();
|
snapshot = restoreSnapshotMsg.getSnapshot();
|
||||||
modifiedHTableDescriptor =
|
modifiedHTableDescriptor =
|
||||||
ProtobufUtil.convertToHTableDesc(restoreSnapshotMsg.getModifiedTableSchema());
|
ProtobufUtil.convertToHTableDesc(restoreSnapshotMsg.getModifiedTableSchema());
|
||||||
|
@ -316,19 +310,6 @@ public class RestoreSnapshotProcedure
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected boolean acquireLock(final MasterProcedureEnv env) {
|
|
||||||
if (env.waitInitialized(this)) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, getTableName());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void releaseLock(final MasterProcedureEnv env) {
|
|
||||||
env.getProcedureQueue().releaseTableExclusiveLock(this, getTableName());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Action before any real action of restoring from snapshot.
|
* Action before any real action of restoring from snapshot.
|
||||||
* @param env MasterProcedureEnv
|
* @param env MasterProcedureEnv
|
||||||
|
|
|
@ -40,28 +40,21 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
|
||||||
import org.apache.hadoop.hbase.security.User;
|
|
||||||
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class TruncateTableProcedure
|
public class TruncateTableProcedure
|
||||||
extends StateMachineProcedure<MasterProcedureEnv, TruncateTableState>
|
extends AbstractStateMachineTableProcedure<TruncateTableState> {
|
||||||
implements TableProcedureInterface {
|
|
||||||
private static final Log LOG = LogFactory.getLog(TruncateTableProcedure.class);
|
private static final Log LOG = LogFactory.getLog(TruncateTableProcedure.class);
|
||||||
|
|
||||||
private boolean preserveSplits;
|
private boolean preserveSplits;
|
||||||
private List<HRegionInfo> regions;
|
private List<HRegionInfo> regions;
|
||||||
private User user;
|
|
||||||
private HTableDescriptor hTableDescriptor;
|
private HTableDescriptor hTableDescriptor;
|
||||||
private TableName tableName;
|
private TableName tableName;
|
||||||
|
|
||||||
// used for compatibility with old clients, until 2.0 the client had a sync behavior
|
|
||||||
private final ProcedurePrepareLatch syncLatch;
|
|
||||||
|
|
||||||
public TruncateTableProcedure() {
|
public TruncateTableProcedure() {
|
||||||
// Required by the Procedure framework to create the procedure on replay
|
// Required by the Procedure framework to create the procedure on replay
|
||||||
syncLatch = null;
|
super();
|
||||||
}
|
}
|
||||||
|
|
||||||
public TruncateTableProcedure(final MasterProcedureEnv env, final TableName tableName,
|
public TruncateTableProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||||
|
@ -71,11 +64,9 @@ public class TruncateTableProcedure
|
||||||
|
|
||||||
public TruncateTableProcedure(final MasterProcedureEnv env, final TableName tableName,
|
public TruncateTableProcedure(final MasterProcedureEnv env, final TableName tableName,
|
||||||
boolean preserveSplits, ProcedurePrepareLatch latch) {
|
boolean preserveSplits, ProcedurePrepareLatch latch) {
|
||||||
|
super(env, latch);
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.preserveSplits = preserveSplits;
|
this.preserveSplits = preserveSplits;
|
||||||
this.user = env.getRequestUser();
|
|
||||||
this.setOwner(this.user.getShortName());
|
|
||||||
this.syncLatch = latch;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -166,7 +157,7 @@ public class TruncateTableProcedure
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void completionCleanup(final MasterProcedureEnv env) {
|
protected void completionCleanup(final MasterProcedureEnv env) {
|
||||||
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
|
releaseSyncLatch();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -210,17 +201,6 @@ public class TruncateTableProcedure
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected boolean acquireLock(final MasterProcedureEnv env) {
|
|
||||||
if (env.waitInitialized(this)) return false;
|
|
||||||
return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, getTableName());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void releaseLock(final MasterProcedureEnv env) {
|
|
||||||
env.getProcedureQueue().releaseTableExclusiveLock(this, getTableName());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void toStringClassDetails(StringBuilder sb) {
|
public void toStringClassDetails(StringBuilder sb) {
|
||||||
sb.append(getClass().getSimpleName());
|
sb.append(getClass().getSimpleName());
|
||||||
|
@ -237,7 +217,7 @@ public class TruncateTableProcedure
|
||||||
|
|
||||||
MasterProcedureProtos.TruncateTableStateData.Builder state =
|
MasterProcedureProtos.TruncateTableStateData.Builder state =
|
||||||
MasterProcedureProtos.TruncateTableStateData.newBuilder()
|
MasterProcedureProtos.TruncateTableStateData.newBuilder()
|
||||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
|
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||||
.setPreserveSplits(preserveSplits);
|
.setPreserveSplits(preserveSplits);
|
||||||
if (hTableDescriptor != null) {
|
if (hTableDescriptor != null) {
|
||||||
state.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDescriptor));
|
state.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDescriptor));
|
||||||
|
@ -258,7 +238,7 @@ public class TruncateTableProcedure
|
||||||
|
|
||||||
MasterProcedureProtos.TruncateTableStateData state =
|
MasterProcedureProtos.TruncateTableStateData state =
|
||||||
MasterProcedureProtos.TruncateTableStateData.parseDelimitedFrom(stream);
|
MasterProcedureProtos.TruncateTableStateData.parseDelimitedFrom(stream);
|
||||||
user = MasterProcedureUtil.toUserInfo(state.getUserInfo());
|
setUser(MasterProcedureUtil.toUserInfo(state.getUserInfo()));
|
||||||
if (state.hasTableSchema()) {
|
if (state.hasTableSchema()) {
|
||||||
hTableDescriptor = ProtobufUtil.convertToHTableDesc(state.getTableSchema());
|
hTableDescriptor = ProtobufUtil.convertToHTableDesc(state.getTableSchema());
|
||||||
tableName = hTableDescriptor.getTableName();
|
tableName = hTableDescriptor.getTableName();
|
||||||
|
@ -291,7 +271,7 @@ public class TruncateTableProcedure
|
||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
final TableName tableName = getTableName();
|
final TableName tableName = getTableName();
|
||||||
cpHost.preTruncateTableAction(tableName, user);
|
cpHost.preTruncateTableAction(tableName, getUser());
|
||||||
}
|
}
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -301,7 +281,7 @@ public class TruncateTableProcedure
|
||||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
final TableName tableName = getTableName();
|
final TableName tableName = getTableName();
|
||||||
cpHost.postCompletedTruncateTableAction(tableName, user);
|
cpHost.postCompletedTruncateTableAction(tableName, getUser());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
Loading…
Reference in New Issue