HBASE-15406 Split / merge switch left disabled after early termination of hbck

This commit is contained in:
chenheng 2016-04-07 14:25:18 +08:00
parent 7efb9edecb
commit a9f1c4a871
15 changed files with 2235 additions and 567 deletions

View File

@ -1716,10 +1716,15 @@ public interface Admin extends Abortable, Closeable {
*
* @param enabled enabled or not
* @param synchronous If true, it waits until current split() call, if outstanding, to return.
* @param skipLock if false, we will do lock before change switch.
* with the lock, other requests to change the switch will be rejected!
* And when you set it to be false,
* you should call {@link #releaseSplitOrMergeLockAndRollback()} by yourself
* @param switchTypes switchType list {@link MasterSwitchType}
* @return Previous switch value array
*/
boolean[] setSplitOrMergeEnabled(final boolean enabled, final boolean synchronous,
final boolean skipLock,
final MasterSwitchType... switchTypes) throws IOException;
/**
@ -1729,6 +1734,14 @@ public interface Admin extends Abortable, Closeable {
*/
boolean isSplitOrMergeEnabled(final MasterSwitchType switchType) throws IOException;
/**
* You should call this method after you call
* {@link #setSplitOrMergeEnabled(boolean, boolean, boolean, MasterSwitchType...)}
* with skipLock be false, this method will release the lock created by above method
* and rollback the switch state to be original state before you change switch
* */
void releaseSplitOrMergeLockAndRollback() throws IOException;
/**
* Currently, there are only two compact types:
* {@code NORMAL} means do store files compaction;

View File

@ -1750,6 +1750,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
return stub.isSplitOrMergeEnabled(controller, request);
}
@Override
public MasterProtos.ReleaseSplitOrMergeLockAndRollbackResponse
releaseSplitOrMergeLockAndRollback(RpcController controller,
MasterProtos.ReleaseSplitOrMergeLockAndRollbackRequest request) throws ServiceException {
return stub.releaseSplitOrMergeLockAndRollback(controller, request);
}
@Override
public IsNormalizerEnabledResponse isNormalizerEnabled(RpcController controller,
IsNormalizerEnabledRequest request) throws ServiceException {

View File

@ -3570,13 +3570,13 @@ public class HBaseAdmin implements Admin {
@Override
public boolean[] setSplitOrMergeEnabled(final boolean enabled, final boolean synchronous,
final MasterSwitchType... switchTypes)
throws IOException {
final boolean skipLock, final MasterSwitchType... switchTypes) throws IOException {
return executeCallable(new MasterCallable<boolean[]>(getConnection()) {
@Override
public boolean[] call(int callTimeout) throws ServiceException {
MasterProtos.SetSplitOrMergeEnabledResponse response = master.setSplitOrMergeEnabled(null,
RequestConverter.buildSetSplitOrMergeEnabledRequest(enabled, synchronous, switchTypes));
RequestConverter.buildSetSplitOrMergeEnabledRequest(enabled, synchronous,
skipLock, switchTypes));
boolean[] result = new boolean[switchTypes.length];
int i = 0;
for (Boolean prevValue : response.getPrevValueList()) {
@ -3598,6 +3598,18 @@ public class HBaseAdmin implements Admin {
});
}
@Override
public void releaseSplitOrMergeLockAndRollback() throws IOException {
executeCallable(new MasterCallable<Void>(getConnection()) {
@Override
public Void call(int callTimeout) throws ServiceException {
master.releaseSplitOrMergeLockAndRollback(null,
RequestConverter.buildReleaseSplitOrMergeLockAndRollbackRequest());
return null;
}
});
}
private HRegionInfo getMobRegionInfo(TableName tableName) {
return new HRegionInfo(tableName, Bytes.toBytes(".mob"),
HConstants.EMPTY_END_ROW, false, 0);

View File

@ -104,6 +104,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ReleaseSplitOrMergeLockAndRollbackRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
@ -1710,6 +1711,13 @@ public final class RequestConverter {
return builder.build();
}
public static ReleaseSplitOrMergeLockAndRollbackRequest
buildReleaseSplitOrMergeLockAndRollbackRequest() {
ReleaseSplitOrMergeLockAndRollbackRequest.Builder builder =
ReleaseSplitOrMergeLockAndRollbackRequest.newBuilder();
return builder.build();
}
/**
* Creates a protocol buffer SetSplitOrMergeEnabledRequest
*
@ -1720,10 +1728,11 @@ public final class RequestConverter {
* @return a SetSplitOrMergeEnabledRequest
*/
public static SetSplitOrMergeEnabledRequest buildSetSplitOrMergeEnabledRequest(boolean enabled,
boolean synchronous, Admin.MasterSwitchType... switchTypes) {
boolean synchronous, boolean skipLock, Admin.MasterSwitchType... switchTypes) {
SetSplitOrMergeEnabledRequest.Builder builder = SetSplitOrMergeEnabledRequest.newBuilder();
builder.setEnabled(enabled);
builder.setSynchronous(synchronous);
builder.setSkipLock(skipLock);
for (Admin.MasterSwitchType switchType : switchTypes) {
builder.addSwitchTypes(convert(switchType));
}

View File

@ -117,6 +117,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
private String regionNormalizerZNode;
// znode containing the state of all switches, currently there are split and merge child node.
private String switchZNode;
// znode containing the lock for the switches
private String switchLockZNode;
// znode containing the lock for the tables
public String tableLockZNode;
// znode containing the state of recovering regions
@ -433,6 +435,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
regionNormalizerZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.regionNormalizer", "normalizer"));
switchZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.switch", "switch"));
switchLockZNode = ZKUtil.joinZNode(switchZNode, "locks");
tableLockZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.tableLock", "table-lock"));
recoveringRegionsZNode = ZKUtil.joinZNode(baseZNode,
@ -799,4 +802,11 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
public String getSwitchZNode() {
return switchZNode;
}
/**
* @return ZK node for switchLock node.
* */
public String getSwitchLockZNode() {
return switchLockZNode;
}
}

View File

@ -9733,6 +9733,540 @@ public final class ZooKeeperProtos {
// @@protoc_insertion_point(class_scope:hbase.pb.SwitchState)
}
public interface SplitAndMergeStateOrBuilder
extends com.google.protobuf.MessageOrBuilder {
// optional bool split_enabled = 1;
/**
* <code>optional bool split_enabled = 1;</code>
*/
boolean hasSplitEnabled();
/**
* <code>optional bool split_enabled = 1;</code>
*/
boolean getSplitEnabled();
// optional bool merge_enabled = 2;
/**
* <code>optional bool merge_enabled = 2;</code>
*/
boolean hasMergeEnabled();
/**
* <code>optional bool merge_enabled = 2;</code>
*/
boolean getMergeEnabled();
}
/**
* Protobuf type {@code hbase.pb.SplitAndMergeState}
*
* <pre>
**
* State for split and merge, used in hbck
* </pre>
*/
public static final class SplitAndMergeState extends
com.google.protobuf.GeneratedMessage
implements SplitAndMergeStateOrBuilder {
// Use SplitAndMergeState.newBuilder() to construct.
private SplitAndMergeState(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
super(builder);
this.unknownFields = builder.getUnknownFields();
}
private SplitAndMergeState(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
private static final SplitAndMergeState defaultInstance;
public static SplitAndMergeState getDefaultInstance() {
return defaultInstance;
}
public SplitAndMergeState getDefaultInstanceForType() {
return defaultInstance;
}
private final com.google.protobuf.UnknownFieldSet unknownFields;
@java.lang.Override
public final com.google.protobuf.UnknownFieldSet
getUnknownFields() {
return this.unknownFields;
}
private SplitAndMergeState(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
initFields();
int mutable_bitField0_ = 0;
com.google.protobuf.UnknownFieldSet.Builder unknownFields =
com.google.protobuf.UnknownFieldSet.newBuilder();
try {
boolean done = false;
while (!done) {
int tag = input.readTag();
switch (tag) {
case 0:
done = true;
break;
default: {
if (!parseUnknownField(input, unknownFields,
extensionRegistry, tag)) {
done = true;
}
break;
}
case 8: {
bitField0_ |= 0x00000001;
splitEnabled_ = input.readBool();
break;
}
case 16: {
bitField0_ |= 0x00000002;
mergeEnabled_ = input.readBool();
break;
}
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
throw e.setUnfinishedMessage(this);
} catch (java.io.IOException e) {
throw new com.google.protobuf.InvalidProtocolBufferException(
e.getMessage()).setUnfinishedMessage(this);
} finally {
this.unknownFields = unknownFields.build();
makeExtensionsImmutable();
}
}
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SplitAndMergeState_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SplitAndMergeState_fieldAccessorTable
.ensureFieldAccessorsInitialized(
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState.Builder.class);
}
public static com.google.protobuf.Parser<SplitAndMergeState> PARSER =
new com.google.protobuf.AbstractParser<SplitAndMergeState>() {
public SplitAndMergeState parsePartialFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
return new SplitAndMergeState(input, extensionRegistry);
}
};
@java.lang.Override
public com.google.protobuf.Parser<SplitAndMergeState> getParserForType() {
return PARSER;
}
private int bitField0_;
// optional bool split_enabled = 1;
public static final int SPLIT_ENABLED_FIELD_NUMBER = 1;
private boolean splitEnabled_;
/**
* <code>optional bool split_enabled = 1;</code>
*/
public boolean hasSplitEnabled() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
/**
* <code>optional bool split_enabled = 1;</code>
*/
public boolean getSplitEnabled() {
return splitEnabled_;
}
// optional bool merge_enabled = 2;
public static final int MERGE_ENABLED_FIELD_NUMBER = 2;
private boolean mergeEnabled_;
/**
* <code>optional bool merge_enabled = 2;</code>
*/
public boolean hasMergeEnabled() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
/**
* <code>optional bool merge_enabled = 2;</code>
*/
public boolean getMergeEnabled() {
return mergeEnabled_;
}
private void initFields() {
splitEnabled_ = false;
mergeEnabled_ = false;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
byte isInitialized = memoizedIsInitialized;
if (isInitialized != -1) return isInitialized == 1;
memoizedIsInitialized = 1;
return true;
}
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
if (((bitField0_ & 0x00000001) == 0x00000001)) {
output.writeBool(1, splitEnabled_);
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
output.writeBool(2, mergeEnabled_);
}
getUnknownFields().writeTo(output);
}
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
size = 0;
if (((bitField0_ & 0x00000001) == 0x00000001)) {
size += com.google.protobuf.CodedOutputStream
.computeBoolSize(1, splitEnabled_);
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
size += com.google.protobuf.CodedOutputStream
.computeBoolSize(2, mergeEnabled_);
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState)) {
return super.equals(obj);
}
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState) obj;
boolean result = true;
result = result && (hasSplitEnabled() == other.hasSplitEnabled());
if (hasSplitEnabled()) {
result = result && (getSplitEnabled()
== other.getSplitEnabled());
}
result = result && (hasMergeEnabled() == other.hasMergeEnabled());
if (hasMergeEnabled()) {
result = result && (getMergeEnabled()
== other.getMergeEnabled());
}
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
}
private int memoizedHashCode = 0;
@java.lang.Override
public int hashCode() {
if (memoizedHashCode != 0) {
return memoizedHashCode;
}
int hash = 41;
hash = (19 * hash) + getDescriptorForType().hashCode();
if (hasSplitEnabled()) {
hash = (37 * hash) + SPLIT_ENABLED_FIELD_NUMBER;
hash = (53 * hash) + hashBoolean(getSplitEnabled());
}
if (hasMergeEnabled()) {
hash = (37 * hash) + MERGE_ENABLED_FIELD_NUMBER;
hash = (53 * hash) + hashBoolean(getMergeEnabled());
}
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data);
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState parseFrom(
com.google.protobuf.ByteString data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data, extensionRegistry);
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState parseFrom(byte[] data)
throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data);
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState parseFrom(
byte[] data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data, extensionRegistry);
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState parseFrom(java.io.InputStream input)
throws java.io.IOException {
return PARSER.parseFrom(input);
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState parseFrom(
java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
return PARSER.parseFrom(input, extensionRegistry);
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState parseDelimitedFrom(java.io.InputStream input)
throws java.io.IOException {
return PARSER.parseDelimitedFrom(input);
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState parseDelimitedFrom(
java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
return PARSER.parseDelimitedFrom(input, extensionRegistry);
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState parseFrom(
com.google.protobuf.CodedInputStream input)
throws java.io.IOException {
return PARSER.parseFrom(input);
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState parseFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
return PARSER.parseFrom(input, extensionRegistry);
}
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
@java.lang.Override
protected Builder newBuilderForType(
com.google.protobuf.GeneratedMessage.BuilderParent parent) {
Builder builder = new Builder(parent);
return builder;
}
/**
* Protobuf type {@code hbase.pb.SplitAndMergeState}
*
* <pre>
**
* State for split and merge, used in hbck
* </pre>
*/
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder<Builder>
implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeStateOrBuilder {
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SplitAndMergeState_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SplitAndMergeState_fieldAccessorTable
.ensureFieldAccessorsInitialized(
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState.Builder.class);
}
// Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState.newBuilder()
private Builder() {
maybeForceBuilderInitialization();
}
private Builder(
com.google.protobuf.GeneratedMessage.BuilderParent parent) {
super(parent);
maybeForceBuilderInitialization();
}
private void maybeForceBuilderInitialization() {
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
}
}
private static Builder create() {
return new Builder();
}
public Builder clear() {
super.clear();
splitEnabled_ = false;
bitField0_ = (bitField0_ & ~0x00000001);
mergeEnabled_ = false;
bitField0_ = (bitField0_ & ~0x00000002);
return this;
}
public Builder clone() {
return create().mergeFrom(buildPartial());
}
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SplitAndMergeState_descriptor;
}
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState getDefaultInstanceForType() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState.getDefaultInstance();
}
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState build() {
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(result);
}
return result;
}
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState buildPartial() {
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState(this);
int from_bitField0_ = bitField0_;
int to_bitField0_ = 0;
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
to_bitField0_ |= 0x00000001;
}
result.splitEnabled_ = splitEnabled_;
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
to_bitField0_ |= 0x00000002;
}
result.mergeEnabled_ = mergeEnabled_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
}
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState) {
return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState)other);
} else {
super.mergeFrom(other);
return this;
}
}
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState other) {
if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState.getDefaultInstance()) return this;
if (other.hasSplitEnabled()) {
setSplitEnabled(other.getSplitEnabled());
}
if (other.hasMergeEnabled()) {
setMergeEnabled(other.getMergeEnabled());
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
public final boolean isInitialized() {
return true;
}
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState parsedMessage = null;
try {
parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitAndMergeState) e.getUnfinishedMessage();
throw e;
} finally {
if (parsedMessage != null) {
mergeFrom(parsedMessage);
}
}
return this;
}
private int bitField0_;
// optional bool split_enabled = 1;
private boolean splitEnabled_ ;
/**
* <code>optional bool split_enabled = 1;</code>
*/
public boolean hasSplitEnabled() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
/**
* <code>optional bool split_enabled = 1;</code>
*/
public boolean getSplitEnabled() {
return splitEnabled_;
}
/**
* <code>optional bool split_enabled = 1;</code>
*/
public Builder setSplitEnabled(boolean value) {
bitField0_ |= 0x00000001;
splitEnabled_ = value;
onChanged();
return this;
}
/**
* <code>optional bool split_enabled = 1;</code>
*/
public Builder clearSplitEnabled() {
bitField0_ = (bitField0_ & ~0x00000001);
splitEnabled_ = false;
onChanged();
return this;
}
// optional bool merge_enabled = 2;
private boolean mergeEnabled_ ;
/**
* <code>optional bool merge_enabled = 2;</code>
*/
public boolean hasMergeEnabled() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
/**
* <code>optional bool merge_enabled = 2;</code>
*/
public boolean getMergeEnabled() {
return mergeEnabled_;
}
/**
* <code>optional bool merge_enabled = 2;</code>
*/
public Builder setMergeEnabled(boolean value) {
bitField0_ |= 0x00000002;
mergeEnabled_ = value;
onChanged();
return this;
}
/**
* <code>optional bool merge_enabled = 2;</code>
*/
public Builder clearMergeEnabled() {
bitField0_ = (bitField0_ & ~0x00000002);
mergeEnabled_ = false;
onChanged();
return this;
}
// @@protoc_insertion_point(builder_scope:hbase.pb.SplitAndMergeState)
}
static {
defaultInstance = new SplitAndMergeState(true);
defaultInstance.initFields();
}
// @@protoc_insertion_point(class_scope:hbase.pb.SplitAndMergeState)
}
private static com.google.protobuf.Descriptors.Descriptor
internal_static_hbase_pb_MetaRegionServer_descriptor;
private static
@ -9793,6 +10327,11 @@ public final class ZooKeeperProtos {
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_hbase_pb_SwitchState_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor
internal_static_hbase_pb_SplitAndMergeState_descriptor;
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_hbase_pb_SplitAndMergeState_fieldAccessorTable;
public static com.google.protobuf.Descriptors.FileDescriptor
getDescriptor() {
@ -9837,9 +10376,10 @@ public final class ZooKeeperProtos {
"ner\030\002 \001(\0132\024.hbase.pb.ServerName\022\021\n\tthrea" +
"d_id\030\003 \001(\003\022\021\n\tis_shared\030\004 \001(\010\022\017\n\007purpose" +
"\030\005 \001(\t\022\023\n\013create_time\030\006 \001(\003\"\036\n\013SwitchSta" +
"te\022\017\n\007enabled\030\001 \001(\010BE\n*org.apache.hadoop" +
".hbase.protobuf.generatedB\017ZooKeeperProt" +
"osH\001\210\001\001\240\001\001"
"te\022\017\n\007enabled\030\001 \001(\010\"B\n\022SplitAndMergeStat" +
"e\022\025\n\rsplit_enabled\030\001 \001(\010\022\025\n\rmerge_enable" +
"d\030\002 \001(\010BE\n*org.apache.hadoop.hbase.proto" +
"buf.generatedB\017ZooKeeperProtosH\001\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -9918,6 +10458,12 @@ public final class ZooKeeperProtos {
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_SwitchState_descriptor,
new java.lang.String[] { "Enabled", });
internal_static_hbase_pb_SplitAndMergeState_descriptor =
getDescriptor().getMessageTypes().get(12);
internal_static_hbase_pb_SplitAndMergeState_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_SplitAndMergeState_descriptor,
new java.lang.String[] { "SplitEnabled", "MergeEnabled", });
return null;
}
};

View File

@ -288,6 +288,7 @@ message SetSplitOrMergeEnabledRequest {
required bool enabled = 1;
optional bool synchronous = 2;
repeated MasterSwitchType switch_types = 3;
optional bool skip_lock = 4;
}
message SetSplitOrMergeEnabledResponse {
@ -302,6 +303,12 @@ message IsSplitOrMergeEnabledResponse {
required bool enabled = 1;
}
message ReleaseSplitOrMergeLockAndRollbackRequest {
}
message ReleaseSplitOrMergeLockAndRollbackResponse {
}
message NormalizeRequest {
}
@ -671,6 +678,12 @@ service MasterService {
rpc IsSplitOrMergeEnabled(IsSplitOrMergeEnabledRequest)
returns(IsSplitOrMergeEnabledResponse);
/**
* Release lock and rollback state.
*/
rpc ReleaseSplitOrMergeLockAndRollback(ReleaseSplitOrMergeLockAndRollbackRequest)
returns(ReleaseSplitOrMergeLockAndRollbackResponse);
/**
* Run region normalizer. Can NOT run for various reasons. Check logs.
*/

View File

@ -166,3 +166,11 @@ message TableLock {
message SwitchState {
optional bool enabled = 1;
}
/**
* State for split and merge, used in hbck
*/
message SplitAndMergeState {
optional bool split_enabled = 1;
optional bool merge_enabled = 2;
}

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.exceptions.MergeRegionException;
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
import org.apache.hadoop.hbase.ipc.PriorityFunction;
@ -1495,6 +1496,10 @@ public class MasterRpcServices extends RSRpcServices
try {
master.checkInitialized();
boolean newValue = request.getEnabled();
boolean skipLock = request.getSkipLock();
if (!master.getSplitOrMergeTracker().lock(skipLock)) {
throw new DoNotRetryIOException("can't set splitOrMerge switch due to lock");
}
for (MasterSwitchType masterSwitchType : request.getSwitchTypesList()) {
Admin.MasterSwitchType switchType = convert(masterSwitchType);
boolean oldValue = master.isSplitOrMergeEnabled(switchType);
@ -1526,6 +1531,24 @@ public class MasterRpcServices extends RSRpcServices
return response.build();
}
@Override
public ReleaseSplitOrMergeLockAndRollbackResponse
releaseSplitOrMergeLockAndRollback(RpcController controller,
ReleaseSplitOrMergeLockAndRollbackRequest request) throws ServiceException {
try {
master.getSplitOrMergeTracker().releaseLockAndRollback();
} catch (KeeperException e) {
throw new ServiceException(e);
} catch (DeserializationException e) {
throw new ServiceException(e);
} catch (InterruptedException e) {
throw new ServiceException(e);
}
ReleaseSplitOrMergeLockAndRollbackResponse.Builder builder =
ReleaseSplitOrMergeLockAndRollbackResponse.newBuilder();
return builder.build();
}
@Override
public NormalizeResponse normalize(RpcController controller,
NormalizeRequest request) throws ServiceException {

View File

@ -54,6 +54,7 @@ import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
@ -307,6 +308,7 @@ public class HBaseFsck extends Configured implements Closeable {
private Map<TableName, Set<String>> skippedRegions = new HashMap<TableName, Set<String>>();
ZooKeeperWatcher zkw = null;
/**
* Constructor
*
@ -345,6 +347,7 @@ public class HBaseFsck extends Configured implements Closeable {
"hbase.hbck.lockfile.attempt.sleep.interval", DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL),
getConf().getInt(
"hbase.hbck.lockfile.attempt.maxsleeptime", DEFAULT_LOCK_FILE_ATTEMPT_MAX_SLEEP_TIME));
zkw = createZooKeeperWatcher();
}
private class FileLockCallable implements Callable<FSDataOutputStream> {
@ -686,7 +689,8 @@ public class HBaseFsck extends Configured implements Closeable {
}
boolean[] oldSplitAndMerge = null;
if (shouldDisableSplitAndMerge()) {
oldSplitAndMerge = admin.setSplitOrMergeEnabled(false, false,
admin.releaseSplitOrMergeLockAndRollback();
oldSplitAndMerge = admin.setSplitOrMergeEnabled(false, false, false,
Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE);
}
@ -703,14 +707,7 @@ public class HBaseFsck extends Configured implements Closeable {
if (shouldDisableSplitAndMerge()) {
if (oldSplitAndMerge != null) {
if (oldSplitAndMerge[0] && oldSplitAndMerge[1]) {
admin.setSplitOrMergeEnabled(true, false,
Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE);
} else if (oldSplitAndMerge[0]) {
admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.SPLIT);
} else if (oldSplitAndMerge[1]) {
admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.MERGE);
}
admin.releaseSplitOrMergeLockAndRollback();
}
}
}
@ -749,6 +746,10 @@ public class HBaseFsck extends Configured implements Closeable {
} catch (Exception io) {
LOG.warn(io);
} finally {
if (zkw != null) {
zkw.close();
zkw = null;
}
IOUtils.closeQuietly(admin);
IOUtils.closeQuietly(meta);
IOUtils.closeQuietly(connection);
@ -1789,14 +1790,7 @@ public class HBaseFsck extends Configured implements Closeable {
private ServerName getMetaRegionServerName(int replicaId)
throws IOException, KeeperException {
ZooKeeperWatcher zkw = createZooKeeperWatcher();
ServerName sn = null;
try {
sn = new MetaTableLocator().getMetaRegionLocation(zkw, replicaId);
} finally {
zkw.close();
}
return sn;
return new MetaTableLocator().getMetaRegionLocation(zkw, replicaId);
}
/**
@ -3281,19 +3275,15 @@ public class HBaseFsck extends Configured implements Closeable {
}
private void checkAndFixTableLocks() throws IOException {
ZooKeeperWatcher zkw = createZooKeeperWatcher();
TableLockChecker checker = new TableLockChecker(zkw, errors);
checker.checkTableLocks();
if (this.fixTableLocks) {
checker.fixExpiredTableLocks();
}
zkw.close();
}
private void checkAndFixReplication() throws IOException {
ZooKeeperWatcher zkw = createZooKeeperWatcher();
try {
ReplicationChecker checker = new ReplicationChecker(getConf(), zkw, connection, errors);
checker.checkUnDeletedQueues();
@ -3301,9 +3291,6 @@ public class HBaseFsck extends Configured implements Closeable {
checker.fixUnDeletedQueues();
setShouldRerun();
}
} finally {
zkw.close();
}
}
/**
@ -3372,12 +3359,7 @@ public class HBaseFsck extends Configured implements Closeable {
private void unassignMetaReplica(HbckInfo hi) throws IOException, InterruptedException,
KeeperException {
undeployRegions(hi);
ZooKeeperWatcher zkw = createZooKeeperWatcher();
try {
ZKUtil.deleteNode(zkw, zkw.getZNodeForReplica(hi.metaEntry.getReplicaId()));
} finally {
zkw.close();
}
}
private void assignMetaReplica(int replicaId)
@ -4206,7 +4188,12 @@ public class HBaseFsck extends Configured implements Closeable {
* Disable the split and merge
*/
public static void setDisableSplitAndMerge() {
disableSplitAndMerge = true;
setDisableSplitAndMerge(true);
}
@VisibleForTesting
public static void setDisableSplitAndMerge(boolean flag) {
disableSplitAndMerge = flag;
}
/**

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.hbase.zookeeper;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
@ -25,6 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.zookeeper.KeeperException;
@ -37,8 +40,13 @@ import org.apache.zookeeper.KeeperException;
@InterfaceAudience.Private
public class SplitOrMergeTracker {
public static final String LOCK = "splitOrMergeLock";
public static final String STATE = "splitOrMergeState";
private String splitZnode;
private String mergeZnode;
private String splitOrMergeLock;
private ZooKeeperWatcher watcher;
private SwitchStateTracker splitStateTracker;
private SwitchStateTracker mergeStateTracker;
@ -49,6 +57,9 @@ public class SplitOrMergeTracker {
if (ZKUtil.checkExists(watcher, watcher.getSwitchZNode()) < 0) {
ZKUtil.createAndFailSilent(watcher, watcher.getSwitchZNode());
}
if (ZKUtil.checkExists(watcher, watcher.getSwitchLockZNode()) < 0) {
ZKUtil.createAndFailSilent(watcher, watcher.getSwitchLockZNode());
}
} catch (KeeperException e) {
throw new RuntimeException(e);
}
@ -56,8 +67,12 @@ public class SplitOrMergeTracker {
conf.get("zookeeper.znode.switch.split", "split"));
mergeZnode = ZKUtil.joinZNode(watcher.getSwitchZNode(),
conf.get("zookeeper.znode.switch.merge", "merge"));
splitOrMergeLock = ZKUtil.joinZNode(watcher.getSwitchLockZNode(), LOCK);
splitStateTracker = new SwitchStateTracker(watcher, splitZnode, abortable);
mergeStateTracker = new SwitchStateTracker(watcher, mergeZnode, abortable);
this.watcher = watcher;
}
public void start() {
@ -91,6 +106,76 @@ public class SplitOrMergeTracker {
}
}
/**
* rollback the original state and delete lock node.
* */
public void releaseLockAndRollback()
throws KeeperException, DeserializationException, InterruptedException {
if (ZKUtil.checkExists(watcher, splitOrMergeLock) != -1) {
List<ZKUtil.ZKUtilOp> ops = new ArrayList<>();
rollback(ops);
ops.add(ZKUtil.ZKUtilOp.deleteNodeFailSilent(splitOrMergeLock));
ZKUtil.multiOrSequential(watcher, ops, false);
}
}
// If there is old states of switch on zk, do rollback
private void rollback(List<ZKUtil.ZKUtilOp> ops) throws KeeperException, InterruptedException, DeserializationException {
String splitOrMergeState = ZKUtil.joinZNode(watcher.getSwitchLockZNode(),
SplitOrMergeTracker.STATE);
if (ZKUtil.checkExists(watcher, splitOrMergeState) != -1) {
byte[] bytes = ZKUtil.getData(watcher, splitOrMergeState);
ProtobufUtil.expectPBMagicPrefix(bytes);
ZooKeeperProtos.SplitAndMergeState.Builder builder =
ZooKeeperProtos.SplitAndMergeState.newBuilder();
try {
int magicLen = ProtobufUtil.lengthOfPBMagic();
ProtobufUtil.mergeFrom(builder, bytes, magicLen, bytes.length - magicLen);
} catch (IOException e) {
throw new DeserializationException(e);
}
ZooKeeperProtos.SplitAndMergeState splitAndMergeState = builder.build();
splitStateTracker.setSwitchEnabled(splitAndMergeState.hasSplitEnabled());
mergeStateTracker.setSwitchEnabled(splitAndMergeState.hasMergeEnabled());
ops.add(ZKUtil.ZKUtilOp.deleteNodeFailSilent(splitOrMergeState));
}
}
/**
* If there is no lock, you could acquire the lock.
* After we create lock on zk, we save original splitOrMerge switches on zk.
* @param skipLock if true, it means we will skip the lock action
* but we still need to check whether the lock exists or not.
* @return true, lock successfully. otherwise, false
* */
public boolean lock(boolean skipLock) throws KeeperException {
if (ZKUtil.checkExists(watcher, splitOrMergeLock) != -1) {
return false;
}
if (skipLock) {
return true;
}
ZKUtil.createAndFailSilent(watcher, splitOrMergeLock);
if (ZKUtil.checkExists(watcher, splitOrMergeLock) != -1) {
saveOriginalState();
return true;
}
return false;
}
private void saveOriginalState() throws KeeperException {
boolean splitEnabled = isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT);
boolean mergeEnabled = isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE);
String splitOrMergeStates = ZKUtil.joinZNode(watcher.getSwitchLockZNode(),
SplitOrMergeTracker.STATE);
ZooKeeperProtos.SplitAndMergeState.Builder builder
= ZooKeeperProtos.SplitAndMergeState.newBuilder();
builder.setSplitEnabled(splitEnabled);
builder.setMergeEnabled(mergeEnabled);
ZKUtil.createSetData(watcher, splitOrMergeStates,
ProtobufUtil.prependPBMagic(builder.build().toByteArray()));
}
private static class SwitchStateTracker extends ZooKeeperNodeTracker {
public SwitchStateTracker(ZooKeeperWatcher watcher, String node, Abortable abortable) {

View File

@ -41,6 +41,7 @@ import java.util.List;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@Category({MediumTests.class, ClientTests.class})
public class TestSplitOrMergeStatus {
@ -76,14 +77,15 @@ public class TestSplitOrMergeStatus {
Admin admin = TEST_UTIL.getAdmin();
initSwitchStatus(admin);
boolean[] results = admin.setSplitOrMergeEnabled(false, false, Admin.MasterSwitchType.SPLIT);
boolean[] results = admin.setSplitOrMergeEnabled(false, false,
true, Admin.MasterSwitchType.SPLIT);
assertEquals(results.length, 1);
assertTrue(results[0]);
admin.split(t.getName());
int count = waitOnSplitOrMerge(t).size();
assertTrue(orignalCount == count);
results = admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.SPLIT);
results = admin.setSplitOrMergeEnabled(true, false, true, Admin.MasterSwitchType.SPLIT);
assertEquals(results.length, 1);
assertFalse(results[0]);
admin.split(t.getName());
@ -108,7 +110,8 @@ public class TestSplitOrMergeStatus {
waitForMergable(admin, name);
int orignalCount = locator.getAllRegionLocations().size();
boolean[] results = admin.setSplitOrMergeEnabled(false, false, Admin.MasterSwitchType.MERGE);
boolean[] results = admin.setSplitOrMergeEnabled(false, false,
true, Admin.MasterSwitchType.MERGE);
assertEquals(results.length, 1);
assertTrue(results[0]);
List<HRegionInfo> regions = admin.getTableRegions(t.getName());
@ -119,7 +122,7 @@ public class TestSplitOrMergeStatus {
assertTrue(orignalCount == count);
waitForMergable(admin, name);
results = admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.MERGE);
results = admin.setSplitOrMergeEnabled(true, false, true, Admin.MasterSwitchType.MERGE);
assertEquals(results.length, 1);
assertFalse(results[0]);
admin.mergeRegions(regions.get(0).getEncodedNameAsBytes(),
@ -132,7 +135,7 @@ public class TestSplitOrMergeStatus {
@Test
public void testMultiSwitches() throws IOException {
Admin admin = TEST_UTIL.getAdmin();
boolean[] switches = admin.setSplitOrMergeEnabled(false, false,
boolean[] switches = admin.setSplitOrMergeEnabled(false, false, true,
Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE);
for (boolean s : switches){
assertTrue(s);
@ -142,12 +145,34 @@ public class TestSplitOrMergeStatus {
admin.close();
}
@Test
public void testSwitchLock() throws IOException {
Admin admin = TEST_UTIL.getAdmin();
admin.setSplitOrMergeEnabled(false, false, false,
Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE);
try {
admin.setSplitOrMergeEnabled(false, false, true,
Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE);
fail();
} catch (IOException e) {
LOG.info("", e);
}
admin.releaseSplitOrMergeLockAndRollback();
try {
admin.setSplitOrMergeEnabled(true, false, true,
Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE);
} catch (IOException e) {
fail();
}
admin.close();
}
private void initSwitchStatus(Admin admin) throws IOException {
if (!admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT)) {
admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.SPLIT);
admin.setSplitOrMergeEnabled(true, false, true, Admin.MasterSwitchType.SPLIT);
}
if (!admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE)) {
admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.MERGE);
admin.setSplitOrMergeEnabled(true, false, true, Admin.MasterSwitchType.MERGE);
}
assertTrue(admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT));
assertTrue(admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE));

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
@ -61,6 +62,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.Assert;
@ -69,6 +71,8 @@ import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.IOException;
import java.util.ArrayList;
@ -89,6 +93,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.*;
import static org.junit.Assert.*;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.spy;
@Category({MiscTests.class, LargeTests.class})
public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
@ -1843,4 +1849,53 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
doQuarantineTest(table, hbck, 3, 0, 0, 0, 1);
hbck.close();
}
/**
* See HBASE-15406
* */
@Test
public void testSplitOrMergeStatWhenHBCKAbort() throws Exception {
admin.setSplitOrMergeEnabled(true, false, true,
Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE);
boolean oldSplit = admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT);
boolean oldMerge = admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE);
assertTrue(oldSplit);
assertTrue(oldMerge);
ExecutorService exec = new ScheduledThreadPoolExecutor(10);
HBaseFsck hbck = new HBaseFsck(conf, exec);
HBaseFsck.setDisplayFullReport(); // i.e. -details
final HBaseFsck spiedHbck = spy(hbck);
doAnswer(new Answer() {
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
// we close splitOrMerge flag in hbck, so in finally hbck will not set splitOrMerge back.
spiedHbck.setDisableSplitAndMerge(false);
return null;
}
}).when(spiedHbck).onlineConsistencyRepair();
spiedHbck.setDisableSplitAndMerge();
spiedHbck.connect();
spiedHbck.onlineHbck();
spiedHbck.close();
boolean split = admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT);
boolean merge = admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE);
assertFalse(split);
assertFalse(merge);
// rerun hbck to repair the switches state
hbck = new HBaseFsck(conf, exec);
hbck.setDisableSplitAndMerge();
hbck.connect();
hbck.onlineHbck();
hbck.close();
split = admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT);
merge = admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE);
assertTrue(split);
assertTrue(merge);
}
}

View File

@ -146,7 +146,7 @@ module Hbase
end
@admin.setSplitOrMergeEnabled(
java.lang.Boolean.valueOf(enabled), java.lang.Boolean.valueOf(false),
switch_type)[0]
java.lang.Boolean.valueOf(true), switch_type)[0]
end
#----------------------------------------------------------------------------------------------