HBASE-7290 / HBASE-6055 Online and Offline table snapshots. Merged to trunk. (Jesse Yates, Matteo Bertozzi, Jonathan Hsieh, Ted Yu)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1448506 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jonathan Hsieh 2013-02-21 03:38:05 +00:00
commit 10e9c4eb39
144 changed files with 28724 additions and 342 deletions

View File

@ -748,11 +748,21 @@ public final class HConstants {
/** Directory under /hbase where archived hfiles are stored */
public static final String HFILE_ARCHIVE_DIRECTORY = ".archive";
/**
* Name of the directory to store all snapshots. See SnapshotDescriptionUtils for
* remaining snapshot constants; this is here to keep HConstants dependencies at a minimum and
* uni-directional.
*/
public static final String SNAPSHOT_DIR_NAME = ".snapshot";
/** Temporary directory used for table creation and deletion */
public static final String HBASE_TEMP_DIRECTORY = ".tmp";
/** Directories that are not HBase table directories */
public static final List<String> HBASE_NON_TABLE_DIRS =
Collections.unmodifiableList(Arrays.asList(new String[] { HREGION_LOGDIR_NAME,
HREGION_OLDLOGDIR_NAME, CORRUPT_DIR_NAME, SPLIT_LOGDIR_NAME,
HBCK_SIDELINEDIR_NAME, HFILE_ARCHIVE_DIRECTORY }));
HBCK_SIDELINEDIR_NAME, HFILE_ARCHIVE_DIRECTORY, SNAPSHOT_DIR_NAME, HBASE_TEMP_DIRECTORY }));
/** Directories that are not HBase user table directories */
public static final List<String> HBASE_NON_USER_TABLE_DIRS =

View File

@ -10967,6 +10967,801 @@ public final class HBaseProtos {
// @@protoc_insertion_point(class_scope:NameInt64Pair)
}
public interface SnapshotDescriptionOrBuilder
extends com.google.protobuf.MessageOrBuilder {
// required string name = 1;
boolean hasName();
String getName();
// optional string table = 2;
boolean hasTable();
String getTable();
// optional int64 creationTime = 3 [default = 0];
boolean hasCreationTime();
long getCreationTime();
// optional .SnapshotDescription.Type type = 4 [default = FLUSH];
boolean hasType();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type getType();
// optional int32 version = 5;
boolean hasVersion();
int getVersion();
}
public static final class SnapshotDescription extends
com.google.protobuf.GeneratedMessage
implements SnapshotDescriptionOrBuilder {
// Use SnapshotDescription.newBuilder() to construct.
private SnapshotDescription(Builder builder) {
super(builder);
}
private SnapshotDescription(boolean noInit) {}
private static final SnapshotDescription defaultInstance;
public static SnapshotDescription getDefaultInstance() {
return defaultInstance;
}
public SnapshotDescription getDefaultInstanceForType() {
return defaultInstance;
}
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_SnapshotDescription_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_SnapshotDescription_fieldAccessorTable;
}
public enum Type
implements com.google.protobuf.ProtocolMessageEnum {
DISABLED(0, 0),
FLUSH(1, 1),
;
public static final int DISABLED_VALUE = 0;
public static final int FLUSH_VALUE = 1;
public final int getNumber() { return value; }
public static Type valueOf(int value) {
switch (value) {
case 0: return DISABLED;
case 1: return FLUSH;
default: return null;
}
}
public static com.google.protobuf.Internal.EnumLiteMap<Type>
internalGetValueMap() {
return internalValueMap;
}
private static com.google.protobuf.Internal.EnumLiteMap<Type>
internalValueMap =
new com.google.protobuf.Internal.EnumLiteMap<Type>() {
public Type findValueByNumber(int number) {
return Type.valueOf(number);
}
};
public final com.google.protobuf.Descriptors.EnumValueDescriptor
getValueDescriptor() {
return getDescriptor().getValues().get(index);
}
public final com.google.protobuf.Descriptors.EnumDescriptor
getDescriptorForType() {
return getDescriptor();
}
public static final com.google.protobuf.Descriptors.EnumDescriptor
getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.getDescriptor().getEnumTypes().get(0);
}
private static final Type[] VALUES = {
DISABLED, FLUSH,
};
public static Type valueOf(
com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
if (desc.getType() != getDescriptor()) {
throw new java.lang.IllegalArgumentException(
"EnumValueDescriptor is not for this type.");
}
return VALUES[desc.getIndex()];
}
private final int index;
private final int value;
private Type(int index, int value) {
this.index = index;
this.value = value;
}
// @@protoc_insertion_point(enum_scope:SnapshotDescription.Type)
}
private int bitField0_;
// required string name = 1;
public static final int NAME_FIELD_NUMBER = 1;
private java.lang.Object name_;
public boolean hasName() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getName() {
java.lang.Object ref = name_;
if (ref instanceof String) {
return (String) ref;
} else {
com.google.protobuf.ByteString bs =
(com.google.protobuf.ByteString) ref;
String s = bs.toStringUtf8();
if (com.google.protobuf.Internal.isValidUtf8(bs)) {
name_ = s;
}
return s;
}
}
private com.google.protobuf.ByteString getNameBytes() {
java.lang.Object ref = name_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
name_ = b;
return b;
} else {
return (com.google.protobuf.ByteString) ref;
}
}
// optional string table = 2;
public static final int TABLE_FIELD_NUMBER = 2;
private java.lang.Object table_;
public boolean hasTable() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getTable() {
java.lang.Object ref = table_;
if (ref instanceof String) {
return (String) ref;
} else {
com.google.protobuf.ByteString bs =
(com.google.protobuf.ByteString) ref;
String s = bs.toStringUtf8();
if (com.google.protobuf.Internal.isValidUtf8(bs)) {
table_ = s;
}
return s;
}
}
private com.google.protobuf.ByteString getTableBytes() {
java.lang.Object ref = table_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
table_ = b;
return b;
} else {
return (com.google.protobuf.ByteString) ref;
}
}
// optional int64 creationTime = 3 [default = 0];
public static final int CREATIONTIME_FIELD_NUMBER = 3;
private long creationTime_;
public boolean hasCreationTime() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
public long getCreationTime() {
return creationTime_;
}
// optional .SnapshotDescription.Type type = 4 [default = FLUSH];
public static final int TYPE_FIELD_NUMBER = 4;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type type_;
public boolean hasType() {
return ((bitField0_ & 0x00000008) == 0x00000008);
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type getType() {
return type_;
}
// optional int32 version = 5;
public static final int VERSION_FIELD_NUMBER = 5;
private int version_;
public boolean hasVersion() {
return ((bitField0_ & 0x00000010) == 0x00000010);
}
public int getVersion() {
return version_;
}
private void initFields() {
name_ = "";
table_ = "";
creationTime_ = 0L;
type_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type.FLUSH;
version_ = 0;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
byte isInitialized = memoizedIsInitialized;
if (isInitialized != -1) return isInitialized == 1;
if (!hasName()) {
memoizedIsInitialized = 0;
return false;
}
memoizedIsInitialized = 1;
return true;
}
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
if (((bitField0_ & 0x00000001) == 0x00000001)) {
output.writeBytes(1, getNameBytes());
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
output.writeBytes(2, getTableBytes());
}
if (((bitField0_ & 0x00000004) == 0x00000004)) {
output.writeInt64(3, creationTime_);
}
if (((bitField0_ & 0x00000008) == 0x00000008)) {
output.writeEnum(4, type_.getNumber());
}
if (((bitField0_ & 0x00000010) == 0x00000010)) {
output.writeInt32(5, version_);
}
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
.computeBytesSize(1, getNameBytes());
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
size += com.google.protobuf.CodedOutputStream
.computeBytesSize(2, getTableBytes());
}
if (((bitField0_ & 0x00000004) == 0x00000004)) {
size += com.google.protobuf.CodedOutputStream
.computeInt64Size(3, creationTime_);
}
if (((bitField0_ & 0x00000008) == 0x00000008)) {
size += com.google.protobuf.CodedOutputStream
.computeEnumSize(4, type_.getNumber());
}
if (((bitField0_ & 0x00000010) == 0x00000010)) {
size += com.google.protobuf.CodedOutputStream
.computeInt32Size(5, version_);
}
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.HBaseProtos.SnapshotDescription)) {
return super.equals(obj);
}
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription) obj;
boolean result = true;
result = result && (hasName() == other.hasName());
if (hasName()) {
result = result && getName()
.equals(other.getName());
}
result = result && (hasTable() == other.hasTable());
if (hasTable()) {
result = result && getTable()
.equals(other.getTable());
}
result = result && (hasCreationTime() == other.hasCreationTime());
if (hasCreationTime()) {
result = result && (getCreationTime()
== other.getCreationTime());
}
result = result && (hasType() == other.hasType());
if (hasType()) {
result = result &&
(getType() == other.getType());
}
result = result && (hasVersion() == other.hasVersion());
if (hasVersion()) {
result = result && (getVersion()
== other.getVersion());
}
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
}
@java.lang.Override
public int hashCode() {
int hash = 41;
hash = (19 * hash) + getDescriptorForType().hashCode();
if (hasName()) {
hash = (37 * hash) + NAME_FIELD_NUMBER;
hash = (53 * hash) + getName().hashCode();
}
if (hasTable()) {
hash = (37 * hash) + TABLE_FIELD_NUMBER;
hash = (53 * hash) + getTable().hashCode();
}
if (hasCreationTime()) {
hash = (37 * hash) + CREATIONTIME_FIELD_NUMBER;
hash = (53 * hash) + hashLong(getCreationTime());
}
if (hasType()) {
hash = (37 * hash) + TYPE_FIELD_NUMBER;
hash = (53 * hash) + hashEnum(getType());
}
if (hasVersion()) {
hash = (37 * hash) + VERSION_FIELD_NUMBER;
hash = (53 * hash) + getVersion();
}
hash = (29 * hash) + getUnknownFields().hashCode();
return hash;
}
public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data).buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
com.google.protobuf.ByteString data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data, extensionRegistry)
.buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(byte[] data)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data).buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
byte[] data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data, extensionRegistry)
.buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(java.io.InputStream input)
throws java.io.IOException {
return newBuilder().mergeFrom(input).buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseDelimitedFrom(java.io.InputStream input)
throws java.io.IOException {
Builder builder = newBuilder();
if (builder.mergeDelimitedFrom(input)) {
return builder.buildParsed();
} else {
return null;
}
}
public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseDelimitedFrom(
java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
Builder builder = newBuilder();
if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
return builder.buildParsed();
} else {
return null;
}
}
public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
com.google.protobuf.CodedInputStream input)
throws java.io.IOException {
return newBuilder().mergeFrom(input).buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription 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;
}
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder<Builder>
implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder {
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_SnapshotDescription_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_SnapshotDescription_fieldAccessorTable;
}
// Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.newBuilder()
private Builder() {
maybeForceBuilderInitialization();
}
private Builder(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();
name_ = "";
bitField0_ = (bitField0_ & ~0x00000001);
table_ = "";
bitField0_ = (bitField0_ & ~0x00000002);
creationTime_ = 0L;
bitField0_ = (bitField0_ & ~0x00000004);
type_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type.FLUSH;
bitField0_ = (bitField0_ & ~0x00000008);
version_ = 0;
bitField0_ = (bitField0_ & ~0x00000010);
return this;
}
public Builder clone() {
return create().mergeFrom(buildPartial());
}
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.getDescriptor();
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription getDefaultInstanceForType() {
return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.getDefaultInstance();
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription build() {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(result);
}
return result;
}
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(
result).asInvalidProtocolBufferException();
}
return result;
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription buildPartial() {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription(this);
int from_bitField0_ = bitField0_;
int to_bitField0_ = 0;
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
to_bitField0_ |= 0x00000001;
}
result.name_ = name_;
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
to_bitField0_ |= 0x00000002;
}
result.table_ = table_;
if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
to_bitField0_ |= 0x00000004;
}
result.creationTime_ = creationTime_;
if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
to_bitField0_ |= 0x00000008;
}
result.type_ = type_;
if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
to_bitField0_ |= 0x00000010;
}
result.version_ = version_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
}
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription) {
return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription)other);
} else {
super.mergeFrom(other);
return this;
}
}
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription other) {
if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.getDefaultInstance()) return this;
if (other.hasName()) {
setName(other.getName());
}
if (other.hasTable()) {
setTable(other.getTable());
}
if (other.hasCreationTime()) {
setCreationTime(other.getCreationTime());
}
if (other.hasType()) {
setType(other.getType());
}
if (other.hasVersion()) {
setVersion(other.getVersion());
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
public final boolean isInitialized() {
if (!hasName()) {
return false;
}
return true;
}
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
com.google.protobuf.UnknownFieldSet.Builder unknownFields =
com.google.protobuf.UnknownFieldSet.newBuilder(
this.getUnknownFields());
while (true) {
int tag = input.readTag();
switch (tag) {
case 0:
this.setUnknownFields(unknownFields.build());
onChanged();
return this;
default: {
if (!parseUnknownField(input, unknownFields,
extensionRegistry, tag)) {
this.setUnknownFields(unknownFields.build());
onChanged();
return this;
}
break;
}
case 10: {
bitField0_ |= 0x00000001;
name_ = input.readBytes();
break;
}
case 18: {
bitField0_ |= 0x00000002;
table_ = input.readBytes();
break;
}
case 24: {
bitField0_ |= 0x00000004;
creationTime_ = input.readInt64();
break;
}
case 32: {
int rawValue = input.readEnum();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type value = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type.valueOf(rawValue);
if (value == null) {
unknownFields.mergeVarintField(4, rawValue);
} else {
bitField0_ |= 0x00000008;
type_ = value;
}
break;
}
case 40: {
bitField0_ |= 0x00000010;
version_ = input.readInt32();
break;
}
}
}
}
private int bitField0_;
// required string name = 1;
private java.lang.Object name_ = "";
public boolean hasName() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getName() {
java.lang.Object ref = name_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
name_ = s;
return s;
} else {
return (String) ref;
}
}
public Builder setName(String value) {
if (value == null) {
throw new NullPointerException();
}
bitField0_ |= 0x00000001;
name_ = value;
onChanged();
return this;
}
public Builder clearName() {
bitField0_ = (bitField0_ & ~0x00000001);
name_ = getDefaultInstance().getName();
onChanged();
return this;
}
void setName(com.google.protobuf.ByteString value) {
bitField0_ |= 0x00000001;
name_ = value;
onChanged();
}
// optional string table = 2;
private java.lang.Object table_ = "";
public boolean hasTable() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getTable() {
java.lang.Object ref = table_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
table_ = s;
return s;
} else {
return (String) ref;
}
}
public Builder setTable(String value) {
if (value == null) {
throw new NullPointerException();
}
bitField0_ |= 0x00000002;
table_ = value;
onChanged();
return this;
}
public Builder clearTable() {
bitField0_ = (bitField0_ & ~0x00000002);
table_ = getDefaultInstance().getTable();
onChanged();
return this;
}
void setTable(com.google.protobuf.ByteString value) {
bitField0_ |= 0x00000002;
table_ = value;
onChanged();
}
// optional int64 creationTime = 3 [default = 0];
private long creationTime_ ;
public boolean hasCreationTime() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
public long getCreationTime() {
return creationTime_;
}
public Builder setCreationTime(long value) {
bitField0_ |= 0x00000004;
creationTime_ = value;
onChanged();
return this;
}
public Builder clearCreationTime() {
bitField0_ = (bitField0_ & ~0x00000004);
creationTime_ = 0L;
onChanged();
return this;
}
// optional .SnapshotDescription.Type type = 4 [default = FLUSH];
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type type_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type.FLUSH;
public boolean hasType() {
return ((bitField0_ & 0x00000008) == 0x00000008);
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type getType() {
return type_;
}
public Builder setType(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type value) {
if (value == null) {
throw new NullPointerException();
}
bitField0_ |= 0x00000008;
type_ = value;
onChanged();
return this;
}
public Builder clearType() {
bitField0_ = (bitField0_ & ~0x00000008);
type_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type.FLUSH;
onChanged();
return this;
}
// optional int32 version = 5;
private int version_ ;
public boolean hasVersion() {
return ((bitField0_ & 0x00000010) == 0x00000010);
}
public int getVersion() {
return version_;
}
public Builder setVersion(int value) {
bitField0_ |= 0x00000010;
version_ = value;
onChanged();
return this;
}
public Builder clearVersion() {
bitField0_ = (bitField0_ & ~0x00000010);
version_ = 0;
onChanged();
return this;
}
// @@protoc_insertion_point(builder_scope:SnapshotDescription)
}
static {
defaultInstance = new SnapshotDescription(true);
defaultInstance.initFields();
}
// @@protoc_insertion_point(class_scope:SnapshotDescription)
}
public interface EmptyMsgOrBuilder
extends com.google.protobuf.MessageOrBuilder {
}
@ -11721,6 +12516,11 @@ public final class HBaseProtos {
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_NameInt64Pair_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor
internal_static_SnapshotDescription_descriptor;
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_SnapshotDescription_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor
internal_static_EmptyMsg_descriptor;
private static
@ -11783,15 +12583,19 @@ public final class HBaseProtos {
"air\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016Byte" +
"sBytesPair\022\r\n\005first\030\001 \002(\014\022\016\n\006second\030\002 \002(" +
"\014\",\n\rNameInt64Pair\022\014\n\004name\030\001 \001(\t\022\r\n\005valu" +
"e\030\002 \001(\003\"\n\n\010EmptyMsg\"\032\n\007LongMsg\022\017\n\007longMs" +
"g\030\001 \002(\003*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS" +
"_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n" +
"\020GREATER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_O" +
"P\020\006*_\n\007KeyType\022\013\n\007MINIMUM\020\000\022\007\n\003PUT\020\004\022\n\n\006" +
"DELETE\020\010\022\021\n\rDELETE_COLUMN\020\014\022\021\n\rDELETE_FA" +
"MILY\020\016\022\014\n\007MAXIMUM\020\377\001B>\n*org.apache.hadoo",
"p.hbase.protobuf.generatedB\013HBaseProtosH" +
"\001\240\001\001"
"e\030\002 \001(\003\"\255\001\n\023SnapshotDescription\022\014\n\004name\030" +
"\001 \002(\t\022\r\n\005table\030\002 \001(\t\022\027\n\014creationTime\030\003 \001" +
"(\003:\0010\022.\n\004type\030\004 \001(\0162\031.SnapshotDescriptio" +
"n.Type:\005FLUSH\022\017\n\007version\030\005 \001(\005\"\037\n\004Type\022\014" +
"\n\010DISABLED\020\000\022\t\n\005FLUSH\020\001\"\n\n\010EmptyMsg\"\032\n\007L" +
"ongMsg\022\017\n\007longMsg\030\001 \002(\003*r\n\013CompareType\022\010" +
"\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r",
"\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007G" +
"REATER\020\005\022\t\n\005NO_OP\020\006*_\n\007KeyType\022\013\n\007MINIMU" +
"M\020\000\022\007\n\003PUT\020\004\022\n\n\006DELETE\020\010\022\021\n\rDELETE_COLUM" +
"N\020\014\022\021\n\rDELETE_FAMILY\020\016\022\014\n\007MAXIMUM\020\377\001B>\n*" +
"org.apache.hadoop.hbase.protobuf.generat" +
"edB\013HBaseProtosH\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -11918,8 +12722,16 @@ public final class HBaseProtos {
new java.lang.String[] { "Name", "Value", },
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameInt64Pair.class,
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameInt64Pair.Builder.class);
internal_static_EmptyMsg_descriptor =
internal_static_SnapshotDescription_descriptor =
getDescriptor().getMessageTypes().get(15);
internal_static_SnapshotDescription_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_SnapshotDescription_descriptor,
new java.lang.String[] { "Name", "Table", "CreationTime", "Type", "Version", },
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.class,
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Builder.class);
internal_static_EmptyMsg_descriptor =
getDescriptor().getMessageTypes().get(16);
internal_static_EmptyMsg_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_EmptyMsg_descriptor,
@ -11927,7 +12739,7 @@ public final class HBaseProtos {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg.class,
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg.Builder.class);
internal_static_LongMsg_descriptor =
getDescriptor().getMessageTypes().get(16);
getDescriptor().getMessageTypes().get(17);
internal_static_LongMsg_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_LongMsg_descriptor,

View File

@ -0,0 +1,58 @@
/**
* 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.
*/
// This file contains protocol buffers that are used for error handling
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
option java_outer_classname = "ErrorHandlingProtos";
option java_generate_equals_and_hash = true;
option optimize_for = SPEED;
/**
* Protobuf version of a java.lang.StackTraceElement
* so we can serialize exceptions.
*/
message StackTraceElementMessage {
optional string declaringClass = 1;
optional string methodName = 2;
optional string fileName = 3;
optional int32 lineNumber = 4;
}
/**
* Cause of a remote failure for a generic exception. Contains
* all the information for a generic exception as well as
* optional info about the error for generic info passing
* (which should be another protobuffed class).
*/
message GenericExceptionMessage {
optional string className = 1;
optional string message = 2;
optional bytes errorInfo = 3;
repeated StackTraceElementMessage trace = 4;
}
/**
* Exception sent across the wire when a remote task needs
* to notify other tasks that it failed and why
*/
message ForeignExceptionMessage {
optional string source = 1;
optional GenericExceptionMessage genericException = 2;
}

View File

@ -177,6 +177,55 @@ message IsCatalogJanitorEnabledResponse {
required bool value = 1;
}
message TakeSnapshotRequest{
required SnapshotDescription snapshot = 1;
}
message TakeSnapshotResponse{
required int64 expectedTimeout = 1;
}
message ListSnapshotRequest{
}
message ListSnapshotResponse{
repeated SnapshotDescription snapshots = 1;
}
message DeleteSnapshotRequest{
required SnapshotDescription snapshot = 1;
}
message DeleteSnapshotResponse{
}
message RestoreSnapshotRequest {
required SnapshotDescription snapshot = 1;
}
message RestoreSnapshotResponse {
}
/* if you don't send the snapshot, then you will get it back
* in the response (if the snapshot is done) so you can check the snapshot
*/
message IsSnapshotDoneRequest{
optional SnapshotDescription snapshot = 1;
}
message IsSnapshotDoneResponse{
optional bool done = 1 [default = false];
optional SnapshotDescription snapshot = 2;
}
message IsRestoreSnapshotDoneRequest {
optional SnapshotDescription snapshot = 1;
}
message IsRestoreSnapshotDoneResponse {
optional bool done = 1 [default = true];
}
service MasterAdminService {
/** Adds a column to the specified table. */
rpc addColumn(AddColumnRequest)
@ -280,4 +329,38 @@ service MasterAdminService {
*/
rpc execMasterService(CoprocessorServiceRequest)
returns(CoprocessorServiceResponse);
/**
* Create a snapshot for the given table.
* @param snapshot description of the snapshot to take
*/
rpc snapshot(TakeSnapshotRequest) returns(TakeSnapshotResponse);
/**
* List completed snapshots.
* @return a list of snapshot descriptors for completed snapshots
*/
rpc getCompletedSnapshots(ListSnapshotRequest) returns(ListSnapshotResponse);
/**
* Delete an existing snapshot. This method can also be used to clean up an aborted snapshot.
* @param snapshotName snapshot to delete
*/
rpc deleteSnapshot(DeleteSnapshotRequest) returns(DeleteSnapshotResponse);
/**
* Determine if the snapshot is done yet.
*/
rpc isSnapshotDone(IsSnapshotDoneRequest) returns(IsSnapshotDoneResponse);
/**
* Restore a snapshot
* @param snapshot description of the snapshot to restore
*/
rpc restoreSnapshot(RestoreSnapshotRequest) returns(RestoreSnapshotResponse);
/**
* Determine if the snapshot restore is done yet.
*/
rpc isRestoreSnapshotDone(IsRestoreSnapshotDoneRequest) returns(IsRestoreSnapshotDoneResponse);
}

View File

@ -263,9 +263,25 @@ message NameInt64Pair {
optional int64 value = 2;
}
/**
* Description of the snapshot to take
*/
message SnapshotDescription {
required string name = 1;
optional string table = 2; // not needed for delete, but checked for in taking snapshot
optional int64 creationTime = 3 [default = 0];
enum Type {
DISABLED = 0;
FLUSH = 1;
}
optional Type type = 4 [default = FLUSH];
optional int32 version = 5;
}
message EmptyMsg {
}
message LongMsg {
required int64 longMsg = 1;
}

View File

@ -43,6 +43,7 @@ org.apache.hadoop.hbase.client.HBaseAdmin;
org.apache.hadoop.hbase.client.HConnectionManager;
org.apache.hadoop.hbase.HTableDescriptor;
org.apache.hadoop.hbase.HBaseConfiguration;
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
</%import>
<%if format.equals("json") %>
<& ../common/TaskMonitorTmpl; filter = filter; format = "json" &>
@ -147,6 +148,9 @@ org.apache.hadoop.hbase.HBaseConfiguration;
<li class="">
<a href="#tab_catalogTables" data-toggle="tab">Catalog Tables</a>
</li>
<li class="">
<a href="#tab_userSnapshots" data-toggle="tab">Snapshots</a>
</li>
</ul>
<div class="tab-content" style="padding-bottom: 9px; border-bottom: 1px solid #ddd;">
<div class="tab-pane active" id="tab_userTables">
@ -159,6 +163,9 @@ org.apache.hadoop.hbase.HBaseConfiguration;
<& catalogTables &>
</%if>
</div>
<div class="tab-pane" id="userSnapshots">
<& userSnapshots &>
</div>
</div>
</div>
</section>
@ -310,6 +317,28 @@ org.apache.hadoop.hbase.HBaseConfiguration;
</%if>
</%def>
<%def userSnapshots>
<%java>
List<SnapshotDescription> snapshots = admin.getCompletedSnapshots();
</%java>
<%if (snapshots != null && snapshots.size() > 0)%>
<table class="table table-striped">
<tr>
<th>Snapshot Name</th>
<th>Table</th>
<th>Creation Time</th>
</tr>
<%for SnapshotDescription snapshotDesc : snapshots%>
<tr>
<td><a href="snapshot.jsp?name=<% snapshotDesc.getName() %>"><% snapshotDesc.getName() %></a> </td>
<td><a href="table.jsp?name=<% snapshotDesc.getTable() %>"><% snapshotDesc.getTable() %></a></td>
<td><% new Date(snapshotDesc.getCreationTime()) %></td>
</tr>
</%for>
<p><% snapshots.size() %> snapshot(s) in set.</p>
</table>
</%if>
</%def>
<%def deadRegionServers>

View File

@ -92,6 +92,14 @@ public abstract class Chore extends HasThread {
this.sleeper.skipSleepCycle();
}
/*
* Exposed for TESTING!
* calls directly the chore method, from the current thread.
*/
public void choreForTesting() {
chore();
}
/**
* Override to run a task before we start looping.
* @return true if initial chore was successful

View File

@ -29,6 +29,8 @@ import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.DeserializationException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
@ -38,6 +40,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.WritableComparable;
@ -433,7 +436,7 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
* @throws IllegalArgumentException If not null and not a legitimate family
* name: i.e. 'printable' and ends in a ':' (Null passes are allowed because
* <code>b</code> can be null when deserializing). Cannot start with a '.'
* either. Also Family can not be an empty value.
* either. Also Family can not be an empty value or equal "recovered.edits".
*/
public static byte [] isLegalFamilyName(final byte [] b) {
if (b == null) {
@ -451,6 +454,11 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
Bytes.toString(b));
}
}
byte[] recoveredEdit = Bytes.toBytes(HLog.RECOVERED_EDITS_DIR);
if (Bytes.equals(recoveredEdit, b)) {
throw new IllegalArgumentException("Family name cannot be: " +
HLog.RECOVERED_EDITS_DIR);
}
return b;
}

View File

@ -110,6 +110,9 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
private static final int ENC_SEPARATOR = '.';
public static final int MD5_HEX_LENGTH = 32;
/** A non-capture group so that this can be embedded. */
public static final String ENCODED_REGION_NAME_REGEX = "(?:[a-f0-9]+)";
/**
* Does region name contain its encoded name?
* @param regionName region name

View File

@ -411,6 +411,9 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
Bytes.equals(tableName, HConstants.META_TABLE_NAME);
}
// A non-capture group so that this can be embedded.
public static final String VALID_USER_TABLE_REGEX = "(?:[a-zA-Z_0-9][a-zA-Z_0-9.-]*)";
/**
* Check passed byte buffer, "tableName", is legal user-space table name.
* @return Returns passed <code>tableName</code> param

View File

@ -33,4 +33,4 @@ import org.apache.hadoop.hbase.security.TokenInfo;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface MasterAdminProtocol
extends MasterAdminService.BlockingInterface, MasterProtocol {}
extends MasterAdminService.BlockingInterface, MasterProtocol {}

View File

@ -73,14 +73,12 @@ public class HFileArchiver {
public static void archiveRegion(Configuration conf, FileSystem fs, HRegionInfo info)
throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
archiveRegion(conf, fs, rootDir, HTableDescriptor.getTableDir(rootDir, info.getTableName()),
archiveRegion(fs, rootDir, HTableDescriptor.getTableDir(rootDir, info.getTableName()),
HRegion.getRegionDir(rootDir, info));
}
/**
* Remove an entire region from the table directory via archiving the region's hfiles.
* @param conf the configuration to use
* @param fs {@link FileSystem} from which to remove the region
* @param rootdir {@link Path} to the root directory where hbase files are stored (for building
* the archive path)
@ -90,8 +88,7 @@ public class HFileArchiver {
* operations could not complete.
* @throws IOException if the request cannot be completed
*/
public static boolean archiveRegion(Configuration conf, FileSystem fs,
Path rootdir, Path tableDir, Path regionDir)
public static boolean archiveRegion(FileSystem fs, Path rootdir, Path tableDir, Path regionDir)
throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("ARCHIVING region " + regionDir.toString());
@ -110,7 +107,7 @@ public class HFileArchiver {
// make sure the regiondir lives under the tabledir
Preconditions.checkArgument(regionDir.toString().startsWith(tableDir.toString()));
Path regionArchiveDir = HFileArchiveUtil.getRegionArchiveDir(conf, tableDir, regionDir);
Path regionArchiveDir = HFileArchiveUtil.getRegionArchiveDir(rootdir, tableDir, regionDir);
LOG.debug("Have an archive directory, preparing to move files");
FileStatusConverter getAsFile = new FileStatusConverter(fs);
@ -239,6 +236,35 @@ public class HFileArchiver {
}
}
/**
* Archive the store file
* @param fs the filesystem where the store files live
* @param regionInfo region hosting the store files
* @param conf {@link Configuration} to examine to determine the archive directory
* @param tableDir {@link Path} to where the table is being stored (for building the archive path)
* @param family the family hosting the store files
* @param storeFile file to be archived
* @throws IOException if the files could not be correctly disposed.
*/
public static void archiveStoreFile(FileSystem fs, HRegionInfo regionInfo,
Configuration conf, Path tableDir, byte[] family, Path storeFile) throws IOException {
Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, regionInfo, tableDir, family);
// make sure we don't archive if we can't and that the archive dir exists
if (!fs.mkdirs(storeArchiveDir)) {
throw new IOException("Could not make archive directory (" + storeArchiveDir + ") for store:"
+ Bytes.toString(family) + ", deleting compacted files instead.");
}
// do the actual archive
long start = EnvironmentEdgeManager.currentTimeMillis();
File file = new FileablePath(fs, storeFile);
if (!resolveAndArchiveFile(storeArchiveDir, file, Long.toString(start))) {
throw new IOException("Failed to archive/delete the file for region:"
+ regionInfo.getRegionNameAsString() + ", family:" + Bytes.toString(family)
+ " into " + storeArchiveDir + ". Something is probably awry on the filesystem.");
}
}
/**
* Archive the given files and resolve any conflicts with existing files via appending the time
* archiving started (so all conflicts in the same group have the same timestamp appended).

View File

@ -18,8 +18,10 @@
package org.apache.hadoop.hbase.catalog;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.ConnectException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.commons.logging.Log;
@ -31,6 +33,7 @@ import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.util.Bytes;
@ -138,11 +141,44 @@ public class MetaEditor {
* @param d Delete to add to .META.
* @throws IOException
*/
static void deleteMetaTable(final CatalogTracker ct, final Delete d)
throws IOException {
static void deleteFromMetaTable(final CatalogTracker ct, final Delete d)
throws IOException {
List<Delete> dels = new ArrayList<Delete>(1);
dels.add(d);
deleteFromMetaTable(ct, dels);
}
/**
* Delete the passed <code>deletes</code> from the <code>.META.</code> table.
* @param ct CatalogTracker on whose back we will ride the edit.
* @param deletes Deletes to add to .META. This list should support #remove.
* @throws IOException
*/
public static void deleteFromMetaTable(final CatalogTracker ct, final List<Delete> deletes)
throws IOException {
HTable t = MetaReader.getMetaHTable(ct);
try {
t.delete(d);
t.delete(deletes);
} finally {
t.close();
}
}
/**
* Execute the passed <code>mutations</code> against <code>.META.</code> table.
* @param ct CatalogTracker on whose back we will ride the edit.
* @param mutations Puts and Deletes to execute on .META.
* @throws IOException
*/
static void mutateMetaTable(final CatalogTracker ct, final List<Mutation> mutations)
throws IOException {
HTable t = MetaReader.getMetaHTable(ct);
try {
t.batch(mutations);
} catch (InterruptedException e) {
InterruptedIOException ie = new InterruptedIOException(e.getMessage());
ie.initCause(e);
throw ie;
} finally {
t.close();
}
@ -328,10 +364,56 @@ public class MetaEditor {
HRegionInfo regionInfo)
throws IOException {
Delete delete = new Delete(regionInfo.getRegionName());
deleteMetaTable(catalogTracker, delete);
deleteFromMetaTable(catalogTracker, delete);
LOG.info("Deleted region " + regionInfo.getRegionNameAsString() + " from META");
}
/**
* Deletes the specified regions from META.
* @param catalogTracker
* @param regionsInfo list of regions to be deleted from META
* @throws IOException
*/
public static void deleteRegions(CatalogTracker catalogTracker,
List<HRegionInfo> regionsInfo) throws IOException {
List<Delete> deletes = new ArrayList<Delete>(regionsInfo.size());
for (HRegionInfo hri: regionsInfo) {
deletes.add(new Delete(hri.getRegionName()));
}
deleteFromMetaTable(catalogTracker, deletes);
LOG.info("Deleted from META, regions: " + regionsInfo);
}
/**
* Adds and Removes the specified regions from .META.
* @param catalogTracker
* @param regionsToRemove list of regions to be deleted from META
* @param regionsToAdd list of regions to be added to META
* @throws IOException
*/
public static void mutateRegions(CatalogTracker catalogTracker,
final List<HRegionInfo> regionsToRemove, final List<HRegionInfo> regionsToAdd)
throws IOException {
List<Mutation> mutation = new ArrayList<Mutation>();
if (regionsToRemove != null) {
for (HRegionInfo hri: regionsToRemove) {
mutation.add(new Delete(hri.getRegionName()));
}
}
if (regionsToAdd != null) {
for (HRegionInfo hri: regionsToAdd) {
mutation.add(makePutFromRegionInfo(hri));
}
}
mutateMetaTable(catalogTracker, mutation);
if (regionsToRemove != null && regionsToRemove.size() > 0) {
LOG.debug("Deleted from META, regions: " + regionsToRemove);
}
if (regionsToAdd != null && regionsToAdd.size() > 0) {
LOG.debug("Add to META, regions: " + regionsToAdd);
}
}
/**
* Deletes daughters references in offlined split parent.
* @param catalogTracker
@ -345,7 +427,7 @@ public class MetaEditor {
Delete delete = new Delete(parent.getRegionName());
delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER);
delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER);
deleteMetaTable(catalogTracker, delete);
deleteFromMetaTable(catalogTracker, delete);
LOG.info("Deleted daughters references, qualifier=" + Bytes.toStringBinary(HConstants.SPLITA_QUALIFIER) +
" and qualifier=" + Bytes.toStringBinary(HConstants.SPLITB_QUALIFIER) +
", from parent " + parent.getRegionNameAsString());

View File

@ -74,20 +74,31 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRespo
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AddColumnRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteColumnRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteSnapshotRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ListSnapshotRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyColumnRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.RestoreSnapshotRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.RestoreSnapshotResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetSchemaAlterStatusRequest;
@ -95,8 +106,14 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetSchemaA
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsResponse;
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StringUtils;
@ -671,9 +688,23 @@ public class HBaseAdmin implements Abortable, Closeable {
enableTableAsync(tableName);
// Wait until all regions are enabled
waitUntilTableIsEnabled(tableName);
LOG.info("Enabled table " + Bytes.toString(tableName));
}
/**
* Wait for the table to be enabled and available
* If enabling the table exceeds the retry period, an exception is thrown.
* @param tableName name of the table
* @throws IOException if a remote or network exception occurs or
* table is not enabled after the retries period.
*/
private void waitUntilTableIsEnabled(final byte[] tableName) throws IOException {
boolean enabled = false;
long start = EnvironmentEdgeManager.currentTimeMillis();
for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
enabled = isTableEnabled(tableName);
enabled = isTableEnabled(tableName) && isTableAvailable(tableName);
if (enabled) {
break;
}
@ -692,10 +723,10 @@ public class HBaseAdmin implements Abortable, Closeable {
}
}
if (!enabled) {
throw new IOException("Unable to enable table " +
Bytes.toString(tableName));
long msec = EnvironmentEdgeManager.currentTimeMillis() - start;
throw new IOException("Table '" + Bytes.toString(tableName) +
"' not yet enabled, after " + msec + "ms.");
}
LOG.info("Enabled table " + Bytes.toString(tableName));
}
public void enableTableAsync(final String tableName)
@ -2083,6 +2114,410 @@ public class HBaseAdmin implements Abortable, Closeable {
return state;
}
/**
* Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
* taken. If the table is disabled, an offline snapshot is taken.
* <p>
* Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
* snapshot with the same name (even a different type or with different parameters) will fail with
* a {@link SnapshotCreationException} indicating the duplicate naming.
* <p>
* Snapshot names follow the same naming constraints as tables in HBase. See
* {@link HTableDescriptor#isLegalTableName(byte[])}.
* @param snapshotName name of the snapshot to be created
* @param tableName name of the table for which snapshot is created
* @throws IOException if a remote or network exception occurs
* @throws SnapshotCreationException if snapshot creation failed
* @throws IllegalArgumentException if the snapshot request is formatted incorrectly
*/
public void snapshot(final String snapshotName, final String tableName) throws IOException,
SnapshotCreationException, IllegalArgumentException {
snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH);
}
/**
* Create a timestamp consistent snapshot for the given table.
* <p>
* Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
* snapshot with the same name (even a different type or with different parameters) will fail with
* a {@link SnapshotCreationException} indicating the duplicate naming.
* <p>
* Snapshot names follow the same naming constraints as tables in HBase. See
* {@link HTableDescriptor#isLegalTableName(byte[])}.
* @param snapshotName name of the snapshot to be created
* @param tableName name of the table for which snapshot is created
* @throws IOException if a remote or network exception occurs
* @throws SnapshotCreationException if snapshot creation failed
* @throws IllegalArgumentException if the snapshot request is formatted incorrectly
*/
public void snapshot(final byte[] snapshotName, final byte[] tableName) throws IOException,
SnapshotCreationException, IllegalArgumentException {
snapshot(Bytes.toString(snapshotName), Bytes.toString(tableName));
}
/**
* Create typed snapshot of the table.
* <p>
* Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
* snapshot with the same name (even a different type or with different parameters) will fail with
* a {@link SnapshotCreationException} indicating the duplicate naming.
* <p>
* Snapshot names follow the same naming constraints as tables in HBase. See
* {@link HTableDescriptor#isLegalTableName(byte[])}.
* <p>
* @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
* snapshots stored on the cluster
* @param tableName name of the table to snapshot
* @param type type of snapshot to take
* @throws IOException we fail to reach the master
* @throws SnapshotCreationException if snapshot creation failed
* @throws IllegalArgumentException if the snapshot request is formatted incorrectly
*/
public void snapshot(final String snapshotName, final String tableName,
SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
IllegalArgumentException {
SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
builder.setTable(tableName);
builder.setName(snapshotName);
builder.setType(type);
snapshot(builder.build());
}
/**
* Take a snapshot and wait for the server to complete that snapshot (blocking).
* <p>
* Only a single snapshot should be taken at a time for an instance of HBase, or results may be
* undefined (you can tell multiple HBase clusters to snapshot at the same time, but only one at a
* time for a single cluster).
* <p>
* Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
* snapshot with the same name (even a different type or with different parameters) will fail with
* a {@link SnapshotCreationException} indicating the duplicate naming.
* <p>
* Snapshot names follow the same naming constraints as tables in HBase. See
* {@link HTableDescriptor#isLegalTableName(byte[])}.
* <p>
* You should probably use {@link #snapshot(String, String)} or {@link #snapshot(byte[], byte[])}
* unless you are sure about the type of snapshot that you want to take.
* @param snapshot snapshot to take
* @throws IOException or we lose contact with the master.
* @throws SnapshotCreationException if snapshot failed to be taken
* @throws IllegalArgumentException if the snapshot request is formatted incorrectly
*/
public void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException,
IllegalArgumentException {
// actually take the snapshot
TakeSnapshotResponse response = takeSnapshotAsync(snapshot);
final IsSnapshotDoneRequest request = IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot)
.build();
IsSnapshotDoneResponse done = null;
long start = EnvironmentEdgeManager.currentTimeMillis();
long max = response.getExpectedTimeout();
long maxPauseTime = max / this.numRetries;
int tries = 0;
LOG.debug("Waiting a max of " + max + " ms for snapshot '" +
SnapshotDescriptionUtils.toString(snapshot) + "'' to complete. (max " +
maxPauseTime + " ms per retry)");
while (tries == 0
|| ((EnvironmentEdgeManager.currentTimeMillis() - start) < max && !done.getDone())) {
try {
// sleep a backoff <= pauseTime amount
long sleep = getPauseTime(tries++);
sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
LOG.debug("(#" + tries + ") Sleeping: " + sleep +
"ms while waiting for snapshot completion.");
Thread.sleep(sleep);
} catch (InterruptedException e) {
LOG.debug("Interrupted while waiting for snapshot " + snapshot + " to complete");
Thread.currentThread().interrupt();
}
LOG.debug("Getting current status of snapshot from master...");
done = execute(new MasterAdminCallable<IsSnapshotDoneResponse>() {
@Override
public IsSnapshotDoneResponse call() throws ServiceException {
return masterAdmin.isSnapshotDone(null, request);
}
});
};
if (!done.getDone()) {
throw new SnapshotCreationException("Snapshot '" + snapshot.getName()
+ "' wasn't completed in expectedTime:" + max + " ms", snapshot);
}
}
/**
* Take a snapshot without waiting for the server to complete that snapshot (asynchronous)
* <p>
* Only a single snapshot should be taken at a time, or results may be undefined.
* @param snapshot snapshot to take
* @return response from the server indicating the max time to wait for the snapshot
* @throws IOException if the snapshot did not succeed or we lose contact with the master.
* @throws SnapshotCreationException if snapshot creation failed
* @throws IllegalArgumentException if the snapshot request is formatted incorrectly
*/
public TakeSnapshotResponse takeSnapshotAsync(SnapshotDescription snapshot) throws IOException,
SnapshotCreationException {
SnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
final TakeSnapshotRequest request = TakeSnapshotRequest.newBuilder().setSnapshot(snapshot)
.build();
// run the snapshot on the master
return execute(new MasterAdminCallable<TakeSnapshotResponse>() {
@Override
public TakeSnapshotResponse call() throws ServiceException {
return masterAdmin.snapshot(null, request);
}
});
}
/**
* Check the current state of the passed snapshot.
* <p>
* There are three possible states:
* <ol>
* <li>running - returns <tt>false</tt></li>
* <li>finished - returns <tt>true</tt></li>
* <li>finished with error - throws the exception that caused the snapshot to fail</li>
* </ol>
* <p>
* The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been
* run/started since the snapshot your are checking, you will recieve an
* {@link UnknownSnapshotException}.
* @param snapshot description of the snapshot to check
* @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the snapshot is still
* running
* @throws IOException if we have a network issue
* @throws HBaseSnapshotException if the snapshot failed
* @throws UnknownSnapshotException if the requested snapshot is unknown
*/
public boolean isSnapshotFinished(final SnapshotDescription snapshot)
throws IOException, HBaseSnapshotException, UnknownSnapshotException {
return execute(new MasterAdminCallable<IsSnapshotDoneResponse>() {
@Override
public IsSnapshotDoneResponse call() throws ServiceException {
return masterAdmin.isSnapshotDone(null,
IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build());
}
}).getDone();
}
/**
* Restore the specified snapshot on the original table. (The table must be disabled)
* Before restoring the table, a new snapshot with the current table state is created.
* In case of failure, the table will be rolled back to its original state.
*
* @param snapshotName name of the snapshot to restore
* @throws IOException if a remote or network exception occurs
* @throws RestoreSnapshotException if snapshot failed to be restored
* @throws IllegalArgumentException if the restore request is formatted incorrectly
*/
public void restoreSnapshot(final byte[] snapshotName)
throws IOException, RestoreSnapshotException {
restoreSnapshot(Bytes.toString(snapshotName));
}
/**
* Restore the specified snapshot on the original table. (The table must be disabled)
* Before restoring the table, a new snapshot with the current table state is created.
* In case of failure, the table will be rolled back to the its original state.
*
* @param snapshotName name of the snapshot to restore
* @throws IOException if a remote or network exception occurs
* @throws RestoreSnapshotException if snapshot failed to be restored
* @throws IllegalArgumentException if the restore request is formatted incorrectly
*/
public void restoreSnapshot(final String snapshotName)
throws IOException, RestoreSnapshotException {
String rollbackSnapshot = snapshotName + "-" + EnvironmentEdgeManager.currentTimeMillis();
String tableName = null;
for (SnapshotDescription snapshotInfo: getCompletedSnapshots()) {
if (snapshotInfo.getName().equals(snapshotName)) {
tableName = snapshotInfo.getTable();
break;
}
}
if (tableName == null) {
throw new RestoreSnapshotException(
"Unable to find the table name for snapshot=" + snapshotName);
}
// Take a snapshot of the current state
snapshot(rollbackSnapshot, tableName);
// Restore snapshot
try {
internalRestoreSnapshot(snapshotName, tableName);
} catch (IOException e) {
// Try to rollback
try {
String msg = "Restore snapshot=" + snapshotName +
" failed. Rollback to snapshot=" + rollbackSnapshot + " succeeded.";
LOG.error(msg, e);
internalRestoreSnapshot(rollbackSnapshot, tableName);
throw new RestoreSnapshotException(msg, e);
} catch (IOException ex) {
String msg = "Failed to restore and rollback to snapshot=" + rollbackSnapshot;
LOG.error(msg, ex);
throw new RestoreSnapshotException(msg, ex);
}
}
}
/**
* Create a new table by cloning the snapshot content.
*
* @param snapshotName name of the snapshot to be cloned
* @param tableName name of the table where the snapshot will be restored
* @throws IOException if a remote or network exception occurs
* @throws TableExistsException if table to be created already exists
* @throws RestoreSnapshotException if snapshot failed to be cloned
* @throws IllegalArgumentException if the specified table has not a valid name
*/
public void cloneSnapshot(final byte[] snapshotName, final byte[] tableName)
throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
cloneSnapshot(Bytes.toString(snapshotName), Bytes.toString(tableName));
}
/**
* Create a new table by cloning the snapshot content.
*
* @param snapshotName name of the snapshot to be cloned
* @param tableName name of the table where the snapshot will be restored
* @throws IOException if a remote or network exception occurs
* @throws TableExistsException if table to be created already exists
* @throws RestoreSnapshotException if snapshot failed to be cloned
* @throws IllegalArgumentException if the specified table has not a valid name
*/
public void cloneSnapshot(final String snapshotName, final String tableName)
throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
if (tableExists(tableName)) {
throw new TableExistsException("Table '" + tableName + " already exists");
}
internalRestoreSnapshot(snapshotName, tableName);
waitUntilTableIsEnabled(Bytes.toBytes(tableName));
}
/**
* Execute Restore/Clone snapshot and wait for the server to complete (blocking).
* To check if the cloned table exists, use {@link #isTableAvailable} -- it is not safe to
* create an HTable instance to this table before it is available.
* @param snapshot snapshot to restore
* @param tableName table name to restore the snapshot on
* @throws IOException if a remote or network exception occurs
* @throws RestoreSnapshotException if snapshot failed to be restored
* @throws IllegalArgumentException if the restore request is formatted incorrectly
*/
private void internalRestoreSnapshot(final String snapshotName, final String tableName)
throws IOException, RestoreSnapshotException {
SnapshotDescription snapshot = SnapshotDescription.newBuilder()
.setName(snapshotName).setTable(tableName).build();
// actually restore the snapshot
internalRestoreSnapshotAsync(snapshot);
final IsRestoreSnapshotDoneRequest request = IsRestoreSnapshotDoneRequest.newBuilder()
.setSnapshot(snapshot).build();
IsRestoreSnapshotDoneResponse done = IsRestoreSnapshotDoneResponse.newBuilder().buildPartial();
final long maxPauseTime = 5000;
int tries = 0;
while (!done.getDone()) {
try {
// sleep a backoff <= pauseTime amount
long sleep = getPauseTime(tries++);
sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
LOG.debug(tries + ") Sleeping: " + sleep + " ms while we wait for snapshot restore to complete.");
Thread.sleep(sleep);
} catch (InterruptedException e) {
LOG.debug("Interrupted while waiting for snapshot " + snapshot + " restore to complete");
Thread.currentThread().interrupt();
}
LOG.debug("Getting current status of snapshot restore from master...");
done = execute(new MasterAdminCallable<IsRestoreSnapshotDoneResponse>() {
@Override
public IsRestoreSnapshotDoneResponse call() throws ServiceException {
return masterAdmin.isRestoreSnapshotDone(null, request);
}
});
}
if (!done.getDone()) {
throw new RestoreSnapshotException("Snapshot '" + snapshot.getName() + "' wasn't restored.");
}
}
/**
* Execute Restore/Clone snapshot and wait for the server to complete (asynchronous)
* <p>
* Only a single snapshot should be restored at a time, or results may be undefined.
* @param snapshot snapshot to restore
* @return response from the server indicating the max time to wait for the snapshot
* @throws IOException if a remote or network exception occurs
* @throws RestoreSnapshotException if snapshot failed to be restored
* @throws IllegalArgumentException if the restore request is formatted incorrectly
*/
private RestoreSnapshotResponse internalRestoreSnapshotAsync(final SnapshotDescription snapshot)
throws IOException, RestoreSnapshotException {
SnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
final RestoreSnapshotRequest request = RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot)
.build();
// run the snapshot restore on the master
return execute(new MasterAdminCallable<RestoreSnapshotResponse>() {
@Override
public RestoreSnapshotResponse call() throws ServiceException {
return masterAdmin.restoreSnapshot(null, request);
}
});
}
/**
* List completed snapshots.
* @return a list of snapshot descriptors for completed snapshots
* @throws IOException if a network error occurs
*/
public List<SnapshotDescription> getCompletedSnapshots() throws IOException {
return execute(new MasterAdminCallable<List<SnapshotDescription>>() {
@Override
public List<SnapshotDescription> call() throws ServiceException {
return masterAdmin.getCompletedSnapshots(null, ListSnapshotRequest.newBuilder().build())
.getSnapshotsList();
}
});
}
/**
* Delete an existing snapshot.
* @param snapshotName name of the snapshot
* @throws IOException if a remote or network exception occurs
*/
public void deleteSnapshot(final byte[] snapshotName) throws IOException {
deleteSnapshot(Bytes.toString(snapshotName));
}
/**
* Delete an existing snapshot.
* @param snapshotName name of the snapshot
* @throws IOException if a remote or network exception occurs
*/
public void deleteSnapshot(final String snapshotName) throws IOException {
// make sure the snapshot is possibly valid
HTableDescriptor.isLegalTableName(Bytes.toBytes(snapshotName));
// do the delete
execute(new MasterAdminCallable<Void>() {
@Override
public Void call() throws ServiceException {
masterAdmin.deleteSnapshot(
null,
DeleteSnapshotRequest.newBuilder()
.setSnapshot(SnapshotDescription.newBuilder().setName(snapshotName).build()).build());
return null;
}
});
}
/**
* @see {@link #execute(MasterAdminCallable<V>)}
*/

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import java.io.IOException;
import java.util.List;
@ -287,4 +288,50 @@ public class BaseMasterObserver implements MasterObserver {
HRegionInfo region, ServerName srcServer, ServerName destServer)
throws IOException {
}
@Override
public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void postSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void postCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void postRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void preDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot) throws IOException {
}
@Override
public void postDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot) throws IOException {
}
}

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
/**
* Defines coprocessor hooks for interacting with operations on the
@ -496,4 +497,100 @@ public interface MasterObserver extends Coprocessor {
*/
void postStartMaster(final ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException;
/**
* Called before a new snapshot is taken.
* Called as part of snapshot RPC call.
* It can't bypass the default action, e.g., ctx.bypass() won't have effect.
* @param ctx the environment to interact with the framework and master
* @param snapshot the SnapshotDescriptor for the snapshot
* @param hTableDescriptor the hTableDescriptor of the table to snapshot
* @throws IOException
*/
void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException;
/**
* Called after the snapshot operation has been requested.
* Called as part of snapshot RPC call.
* @param ctx the environment to interact with the framework and master
* @param snapshot the SnapshotDescriptor for the snapshot
* @param hTableDescriptor the hTableDescriptor of the table to snapshot
* @throws IOException
*/
void postSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException;
/**
* Called before a snapshot is cloned.
* Called as part of restoreSnapshot RPC call.
* It can't bypass the default action, e.g., ctx.bypass() won't have effect.
* @param ctx the environment to interact with the framework and master
* @param snapshot the SnapshotDescriptor for the snapshot
* @param hTableDescriptor the hTableDescriptor of the table to create
* @throws IOException
*/
void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException;
/**
* Called after a snapshot clone operation has been requested.
* Called as part of restoreSnapshot RPC call.
* @param ctx the environment to interact with the framework and master
* @param snapshot the SnapshotDescriptor for the snapshot
* @param hTableDescriptor the hTableDescriptor of the table to create
* @throws IOException
*/
void postCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException;
/**
* Called before a snapshot is restored.
* Called as part of restoreSnapshot RPC call.
* It can't bypass the default action, e.g., ctx.bypass() won't have effect.
* @param ctx the environment to interact with the framework and master
* @param snapshot the SnapshotDescriptor for the snapshot
* @param hTableDescriptor the hTableDescriptor of the table to restore
* @throws IOException
*/
void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException;
/**
* Called after a snapshot restore operation has been requested.
* Called as part of restoreSnapshot RPC call.
* @param ctx the environment to interact with the framework and master
* @param snapshot the SnapshotDescriptor for the snapshot
* @param hTableDescriptor the hTableDescriptor of the table to restore
* @throws IOException
*/
void postRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException;
/**
* Called before a snapshot is deleted.
* Called as part of deleteSnapshot RPC call.
* It can't bypass the default action, e.g., ctx.bypass() won't have effect.
* @param ctx the environment to interact with the framework and master
* @param snapshot the SnapshotDescriptor of the snapshot to delete
* @throws IOException
*/
void preDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot) throws IOException;
/**
* Called after the delete snapshot operation has been requested.
* Called as part of deleteSnapshot RPC call.
* @param ctx the environment to interact with the framework and master
* @param snapshot the SnapshotDescriptor of the snapshot to delete
* @throws IOException
*/
void postDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot) throws IOException;
}

View File

@ -0,0 +1,194 @@
/**
* 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.errorhandling;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage;
import org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage;
import org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* A ForeignException is an exception from another thread or process.
* <p>
* ForeignExceptions are sent to 'remote' peers to signal an abort in the face of failures.
* When serialized for transmission we encode using Protobufs to ensure version compatibility.
* <p>
* Foreign exceptions contain a Throwable as its cause. This can be a "regular" exception
* generated locally or a ProxyThrowable that is a representation of the original exception
* created on original 'remote' source. These ProxyThrowables have their their stacks traces and
* messages overridden to reflect the original 'remote' exception. The only way these
* ProxyThrowables are generated are by this class's {@link #deserialize(byte[])} method.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@SuppressWarnings("serial")
public class ForeignException extends IOException {
/**
* Name of the throwable's source such as a host or thread name. Must be non-null.
*/
private final String source;
/**
* Create a new ForeignException that can be serialized. It is assumed that this came form a
* local source.
* @param source
* @param cause
*/
public ForeignException(String source, Throwable cause) {
super(cause);
assert source != null;
assert cause != null;
this.source = source;
}
/**
* Create a new ForeignException that can be serialized. It is assumed that this is locally
* generated.
* @param source
* @param msg
*/
public ForeignException(String source, String msg) {
super(new IllegalArgumentException(msg));
this.source = source;
}
public String getSource() {
return source;
}
/**
* The cause of a ForeignException can be an exception that was generated on a local in process
* thread, or a thread from a 'remote' separate process.
*
* If the cause is a ProxyThrowable, we know it came from deserialization which usually means
* it came from not only another thread, but also from a remote thread.
*
* @return true if went through deserialization, false if locally generated
*/
public boolean isRemote() {
return getCause() instanceof ProxyThrowable;
}
@Override
public String toString() {
String className = getCause().getClass().getName() ;
return className + " via " + getSource() + ":" + getLocalizedMessage();
}
/**
* Convert a stack trace to list of {@link StackTraceElement}.
* @param trace the stack trace to convert to protobuf message
* @return <tt>null</tt> if the passed stack is <tt>null</tt>.
*/
private static List<StackTraceElementMessage> toStackTraceElementMessages(
StackTraceElement[] trace) {
// if there is no stack trace, ignore it and just return the message
if (trace == null) return null;
// build the stack trace for the message
List<StackTraceElementMessage> pbTrace =
new ArrayList<StackTraceElementMessage>(trace.length);
for (StackTraceElement elem : trace) {
StackTraceElementMessage.Builder stackBuilder = StackTraceElementMessage.newBuilder();
stackBuilder.setDeclaringClass(elem.getClassName());
stackBuilder.setFileName(elem.getFileName());
stackBuilder.setLineNumber(elem.getLineNumber());
stackBuilder.setMethodName(elem.getMethodName());
pbTrace.add(stackBuilder.build());
}
return pbTrace;
}
/**
* This is a Proxy Throwable that contains the information of the original remote exception
*/
private static class ProxyThrowable extends Throwable {
ProxyThrowable(String msg, StackTraceElement[] trace) {
super(msg);
this.setStackTrace(trace);
}
}
/**
* Converts a ForeignException to an array of bytes.
* @param source the name of the external exception source
* @param t the "local" external exception (local)
* @return protobuf serialized version of ForeignException
*/
public static byte[] serialize(String source, Throwable t) {
GenericExceptionMessage.Builder gemBuilder = GenericExceptionMessage.newBuilder();
gemBuilder.setClassName(t.getClass().getName());
if (t.getMessage() != null) {
gemBuilder.setMessage(t.getMessage());
}
// set the stack trace, if there is one
List<StackTraceElementMessage> stack =
ForeignException.toStackTraceElementMessages(t.getStackTrace());
if (stack != null) {
gemBuilder.addAllTrace(stack);
}
GenericExceptionMessage payload = gemBuilder.build();
ForeignExceptionMessage.Builder exception = ForeignExceptionMessage.newBuilder();
exception.setGenericException(payload).setSource(source);
ForeignExceptionMessage eem = exception.build();
return eem.toByteArray();
}
/**
* Takes a series of bytes and tries to generate an ForeignException instance for it.
* @param bytes
* @return the ForeignExcpetion instance
* @throws InvalidProtocolBufferException if there was deserialization problem this is thrown.
*/
public static ForeignException deserialize(byte[] bytes) throws InvalidProtocolBufferException {
// figure out the data we need to pass
ForeignExceptionMessage eem = ForeignExceptionMessage.parseFrom(bytes);
GenericExceptionMessage gem = eem.getGenericException();
StackTraceElement [] trace = ForeignException.toStackTrace(gem.getTraceList());
ProxyThrowable dfe = new ProxyThrowable(gem.getMessage(), trace);
ForeignException e = new ForeignException(eem.getSource(), dfe);
return e;
}
/**
* Unwind a serialized array of {@link StackTraceElementMessage}s to a
* {@link StackTraceElement}s.
* @param traceList list that was serialized
* @return the deserialized list or <tt>null</tt> if it couldn't be unwound (e.g. wasn't set on
* the sender).
*/
private static StackTraceElement[] toStackTrace(List<StackTraceElementMessage> traceList) {
if (traceList == null || traceList.size() == 0) {
return new StackTraceElement[0]; // empty array
}
StackTraceElement[] trace = new StackTraceElement[traceList.size()];
for (int i = 0; i < traceList.size(); i++) {
StackTraceElementMessage elem = traceList.get(i);
trace[i] = new StackTraceElement(
elem.getDeclaringClass(), elem.getMethodName(), elem.getFileName(), elem.getLineNumber());
}
return trace;
}
}

View File

@ -0,0 +1,119 @@
/**
* 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.errorhandling;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* The dispatcher acts as the state holding entity for foreign error handling. The first
* exception received by the dispatcher get passed directly to the listeners. Subsequent
* exceptions are dropped.
* <p>
* If there are multiple dispatchers that are all in the same foreign exception monitoring group,
* ideally all these monitors are "peers" -- any error on one dispatcher should get propagated to
* all others (via rpc, or some other mechanism). Due to racing error conditions the exact reason
* for failure may be different on different peers, but the fact that they are in error state
* should eventually hold on all.
* <p>
* This is thread-safe and must be because this is expected to be used to propagate exceptions
* from foreign threads.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ForeignExceptionDispatcher implements ForeignExceptionListener, ForeignExceptionSnare {
public static final Log LOG = LogFactory.getLog(ForeignExceptionDispatcher.class);
protected final String name;
protected final List<ForeignExceptionListener> listeners =
new ArrayList<ForeignExceptionListener>();
private ForeignException exception;
public ForeignExceptionDispatcher(String name) {
this.name = name;
}
public ForeignExceptionDispatcher() {
this("");
}
public String getName() {
return name;
}
@Override
public synchronized void receive(ForeignException e) {
// if we already have an exception, then ignore it
if (exception != null) return;
LOG.debug(name + " accepting received exception" , e);
// mark that we got the error
if (e != null) {
exception = e;
} else {
exception = new ForeignException(name, "");
}
// notify all the listeners
dispatch(e);
}
@Override
public synchronized void rethrowException() throws ForeignException {
if (exception != null) {
// This gets the stack where this is caused, (instead of where it was deserialized).
// This is much more useful for debugging
throw new ForeignException(exception.getSource(), exception.getCause());
}
}
@Override
public synchronized boolean hasException() {
return exception != null;
}
@Override
synchronized public ForeignException getException() {
return exception;
}
/**
* Sends an exception to all listeners.
* @param message human readable message passed to the listener
* @param e {@link ForeignException} containing the cause. Can be null.
*/
private void dispatch(ForeignException e) {
// update all the listeners with the passed error
for (ForeignExceptionListener l: listeners) {
l.receive(e);
}
}
/**
* Listen for failures to a given process. This method should only be used during
* initialization and not added to after exceptions are accepted.
* @param errorable listener for the errors. may be null.
*/
public synchronized void addListener(ForeignExceptionListener errorable) {
this.listeners.add(errorable);
}
}

View File

@ -0,0 +1,40 @@
/**
* 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.errorhandling;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* The ForeignExceptionListener is an interface for objects that can receive a ForeignException.
* <p>
* Implementations must be thread-safe, because this is expected to be used to propagate exceptions
* from foreign threads.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface ForeignExceptionListener {
/**
* Receive a ForeignException.
* <p>
* Implementers must ensure that this method is thread-safe.
* @param e exception causing the error. Implementations must accept and handle null here.
*/
public void receive(ForeignException e);
}

View File

@ -0,0 +1,67 @@
/**
* 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.errorhandling;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* This is an interface for a cooperative exception throwing mechanism. Implementations are
* containers that holds an exception from a separate thread. This can be used to receive
* exceptions from 'foreign' threads or from separate 'foreign' processes.
* <p>
* To use, one would pass an implementation of this object to a long running method and
* periodically check by calling {@link #rethrowException()}. If any foreign exceptions have
* been received, the calling thread is then responsible for handling the rethrown exception.
* <p>
* One could use the boolean {@link #hasException()} to determine if there is an exceptoin as well.
* <p>
* NOTE: This is very similar to the InterruptedException/interrupt/interrupted pattern. There,
* the notification state is bound to a Thread. Using this, applications receive Exceptions in
* the snare. The snare is referenced and checked by multiple threads which enables exception
* notification in all the involved threads/processes.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface ForeignExceptionSnare {
/**
* Rethrow an exception currently held by the {@link ForeignExceptionSnare}. If there is
* no exception this is a no-op
*
* @throws ForeignException
* all exceptions from remote sources are procedure exceptions
*/
public void rethrowException() throws ForeignException;
/**
* Non-exceptional form of {@link #rethrowException()}. Checks to see if any
* process to which the exception checkers is bound has created an error that
* would cause a failure.
*
* @return <tt>true</tt> if there has been an error,<tt>false</tt> otherwise
*/
public boolean hasException();
/**
* Get the value of the captured exception.
*
* @return the captured foreign exception or null if no exception captured.
*/
public ForeignException getException();
}

View File

@ -0,0 +1,67 @@
/**
* 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.errorhandling;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Exception for timeout of a task.
* @see TimeoutExceptionInjector
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@SuppressWarnings("serial")
public class TimeoutException extends Exception {
private final String sourceName;
private final long start;
private final long end;
private final long expected;
/**
* Exception indicating that an operation attempt has timed out
* @param start time the operation started (ms since epoch)
* @param end time the timeout was triggered (ms since epoch)
* @param expected expected amount of time for the operation to complete (ms) (ideally, expected <= end-start)
*/
public TimeoutException(String sourceName, long start, long end, long expected) {
super("Timeout elapsed! Source:" + sourceName + " Start:" + start + ", End:" + end
+ ", diff:" + (end - start) + ", max:" + expected + " ms");
this.sourceName = sourceName;
this.start = start;
this.end = end;
this.expected = expected;
}
public long getStart() {
return start;
}
public long getEnd() {
return end;
}
public long getMaxAllowedOperationTime() {
return expected;
}
public String getSourceName() {
return sourceName;
}
}

View File

@ -0,0 +1,130 @@
/**
* 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.errorhandling;
import java.util.Timer;
import java.util.TimerTask;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* Time a given process/operation and report a failure if the elapsed time exceeds the max allowed
* time.
* <p>
* The timer won't start tracking time until calling {@link #start()}. If {@link #complete()} or
* {@link #trigger()} is called before {@link #start()}, calls to {@link #start()} will fail.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class TimeoutExceptionInjector {
private static final Log LOG = LogFactory.getLog(TimeoutExceptionInjector.class);
private final long maxTime;
private volatile boolean complete;
private final Timer timer;
private final TimerTask timerTask;
private long start = -1;
/**
* Create a generic timer for a task/process.
* @param listener listener to notify if the process times out
* @param maxTime max allowed running time for the process. Timer starts on calls to
* {@link #start()}
*/
public TimeoutExceptionInjector(final ForeignExceptionListener listener, final long maxTime) {
this.maxTime = maxTime;
timer = new Timer();
timerTask = new TimerTask() {
@Override
public void run() {
// ensure we don't run this task multiple times
synchronized (this) {
// quick exit if we already marked the task complete
if (TimeoutExceptionInjector.this.complete) return;
// mark the task is run, to avoid repeats
TimeoutExceptionInjector.this.complete = true;
}
long end = EnvironmentEdgeManager.currentTimeMillis();
TimeoutException tee = new TimeoutException(
"Timeout caused Foreign Exception", start, end, maxTime);
String source = "timer-" + timer;
listener.receive(new ForeignException(source, tee));
}
};
}
public long getMaxTime() {
return maxTime;
}
/**
* For all time forward, do not throw an error because the process has completed.
*/
public void complete() {
// warn if the timer is already marked complete. This isn't going to be thread-safe, but should
// be good enough and its not worth locking just for a warning.
if (this.complete) {
LOG.warn("Timer already marked completed, ignoring!");
return;
}
LOG.debug("Marking timer as complete - no error notifications will be received for this timer.");
synchronized (this.timerTask) {
this.complete = true;
}
this.timer.cancel();
}
/**
* Start a timer to fail a process if it takes longer than the expected time to complete.
* <p>
* Non-blocking.
* @throws IllegalStateException if the timer has already been marked done via {@link #complete()}
* or {@link #trigger()}
*/
public synchronized void start() throws IllegalStateException {
if (this.start >= 0) {
LOG.warn("Timer already started, can't be started again. Ignoring second request.");
return;
}
LOG.debug("Scheduling process timer to run in: " + maxTime + " ms");
timer.schedule(timerTask, maxTime);
this.start = EnvironmentEdgeManager.currentTimeMillis();
}
/**
* Trigger the timer immediately.
* <p>
* Exposed for testing.
*/
public void trigger() {
synchronized (timerTask) {
if (this.complete) {
LOG.warn("Timer already completed, not triggering.");
return;
}
LOG.debug("Triggering timer immediately!");
this.timer.cancel();
this.timerTask.run();
}
}
}

View File

@ -135,6 +135,8 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
C_M_DELETE_FAMILY (45, null), // Client asking Master to delete family of table
C_M_MODIFY_FAMILY (46, null), // Client asking Master to modify family of table
C_M_CREATE_TABLE (47, ExecutorType.MASTER_TABLE_OPERATIONS), // Client asking Master to create a table
C_M_SNAPSHOT_TABLE (48, ExecutorType.MASTER_TABLE_OPERATIONS), // Client asking Master to snapshot an offline table
C_M_RESTORE_SNAPSHOT (49, ExecutorType.MASTER_TABLE_OPERATIONS), // Client asking Master to restore a snapshot
// Updates from master to ZK. This is done by the master and there is
// nothing to process by either Master or RS

View File

@ -28,8 +28,11 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
@ -52,12 +55,36 @@ import org.apache.hadoop.hbase.util.HFileArchiveUtil;
public class HFileLink extends FileLink {
private static final Log LOG = LogFactory.getLog(HFileLink.class);
/** Define the HFile Link name pattern in the form of: hfile-region-table */
public static final Pattern LINK_NAME_PARSER =
Pattern.compile("^([0-9a-f\\.]+)-([0-9a-f]+)-([a-zA-Z_0-9]+[a-zA-Z0-9_\\-\\.]*)$");
/**
* A non-capture group, for HFileLink, so that this can be embedded.
* The HFileLink describe a link to an hfile in a different table/region
* and the name is in the form: table=region-hfile.
* <p>
* Table name is ([a-zA-Z_0-9][a-zA-Z_0-9.-]*), so '=' is an invalid character for the table name.
* Region name is ([a-f0-9]+), so '-' is an invalid character for the region name.
* HFile is ([0-9a-f]+(?:_SeqId_[0-9]+_)?) covering the plain hfiles (uuid)
* and the bulk loaded (_SeqId_[0-9]+_) hfiles.
*/
public static final String LINK_NAME_REGEX =
String.format("%s=%s-%s", HTableDescriptor.VALID_USER_TABLE_REGEX,
HRegionInfo.ENCODED_REGION_NAME_REGEX, StoreFile.HFILE_NAME_REGEX);
/** Define the HFile Link name parser in the form of: table=region-hfile */
private static final Pattern LINK_NAME_PATTERN =
Pattern.compile(String.format("^(%s)=(%s)-(%s)$", HTableDescriptor.VALID_USER_TABLE_REGEX,
HRegionInfo.ENCODED_REGION_NAME_REGEX, StoreFile.HFILE_NAME_REGEX));
/**
* The pattern should be used for hfile and reference links
* that can be found in /hbase/table/region/family/
*/
private static final Pattern REF_OR_HFILE_LINK_PATTERN =
Pattern.compile(String.format("^(%s)=(%s)-(.+)$", HTableDescriptor.VALID_USER_TABLE_REGEX,
HRegionInfo.ENCODED_REGION_NAME_REGEX));
private final Path archivePath;
private final Path originPath;
private final Path tempPath;
/**
* @param conf {@link Configuration} from which to extract specific archive locations
@ -75,19 +102,10 @@ public class HFileLink extends FileLink {
*/
public HFileLink(final Path rootDir, final Path archiveDir, final Path path) {
Path hfilePath = getRelativeTablePath(path);
this.tempPath = new Path(new Path(rootDir, HConstants.HBASE_TEMP_DIRECTORY), hfilePath);
this.originPath = new Path(rootDir, hfilePath);
this.archivePath = new Path(archiveDir, hfilePath);
setLocations(originPath, archivePath);
}
/**
* @param originPath Path to the hfile in the table directory
* @param archivePath Path to the hfile in the archive directory
*/
public HFileLink(final Path originPath, final Path archivePath) {
this.originPath = originPath;
this.archivePath = archivePath;
setLocations(originPath, archivePath);
setLocations(originPath, tempPath, archivePath);
}
/**
@ -118,67 +136,15 @@ public class HFileLink extends FileLink {
* @return True if the path is a HFileLink.
*/
public static boolean isHFileLink(String fileName) {
Matcher m = LINK_NAME_PARSER.matcher(fileName);
Matcher m = LINK_NAME_PATTERN.matcher(fileName);
if (!m.matches()) return false;
return m.groupCount() > 2 && m.group(2) != null && m.group(3) != null;
}
/**
* The returned path can be the "original" file path like: /hbase/table/region/cf/hfile
* or a path to the archived file like: /hbase/archive/table/region/cf/hfile
*
* @param fs {@link FileSystem} on which to check the HFileLink
* @param path HFileLink path
* @return Referenced path (original path or archived path)
* @throws IOException on unexpected error.
*/
public static Path getReferencedPath(FileSystem fs, final Path path) throws IOException {
return getReferencedPath(fs.getConf(), fs, path);
}
/**
* The returned path can be the "original" file path like: /hbase/table/region/cf/hfile
* or a path to the archived file like: /hbase/.archive/table/region/cf/hfile
*
* @param fs {@link FileSystem} on which to check the HFileLink
* @param conf {@link Configuration} from which to extract specific archive locations
* @param path HFileLink path
* @return Referenced path (original path or archived path)
* @throws IOException on unexpected error.
*/
public static Path getReferencedPath(final Configuration conf, final FileSystem fs,
final Path path) throws IOException {
return getReferencedPath(fs, FSUtils.getRootDir(conf),
HFileArchiveUtil.getArchivePath(conf), path);
}
/**
* The returned path can be the "original" file path like: /hbase/table/region/cf/hfile
* or a path to the archived file like: /hbase/.archive/table/region/cf/hfile
*
* @param fs {@link FileSystem} on which to check the HFileLink
* @param rootDir root hbase directory
* @param archiveDir Path to the hbase archive directory
* @param path HFileLink path
* @return Referenced path (original path or archived path)
* @throws IOException on unexpected error.
*/
public static Path getReferencedPath(final FileSystem fs, final Path rootDir,
final Path archiveDir, final Path path) throws IOException {
Path hfilePath = getRelativeTablePath(path);
Path originPath = new Path(rootDir, hfilePath);
if (fs.exists(originPath)) {
return originPath;
}
return new Path(archiveDir, hfilePath);
return m.groupCount() > 2 && m.group(3) != null && m.group(2) != null && m.group(1) != null;
}
/**
* Convert a HFileLink path to a table relative path.
* e.g. the link: /hbase/test/0123/cf/abcd-4567-testtb
* e.g. the link: /hbase/test/0123/cf/testtb=4567-abcd
* becomes: /hbase/testtb/4567/cf/abcd
*
* @param path HFileLink path
@ -186,16 +152,16 @@ public class HFileLink extends FileLink {
* @throws IOException on unexpected error.
*/
private static Path getRelativeTablePath(final Path path) {
// hfile-region-table
Matcher m = LINK_NAME_PARSER.matcher(path.getName());
// table=region-hfile
Matcher m = REF_OR_HFILE_LINK_PATTERN.matcher(path.getName());
if (!m.matches()) {
throw new IllegalArgumentException(path.getName() + " is not a valid HFileLink name!");
}
// Convert the HFileLink name into a real table/region/cf/hfile path.
String hfileName = m.group(1);
String tableName = m.group(1);
String regionName = m.group(2);
String tableName = m.group(3);
String hfileName = m.group(3);
String familyName = path.getParent().getName();
return new Path(new Path(tableName, regionName), new Path(familyName, hfileName));
}
@ -207,11 +173,11 @@ public class HFileLink extends FileLink {
* @return the name of the referenced HFile
*/
public static String getReferencedHFileName(final String fileName) {
Matcher m = LINK_NAME_PARSER.matcher(fileName);
Matcher m = REF_OR_HFILE_LINK_PATTERN.matcher(fileName);
if (!m.matches()) {
throw new IllegalArgumentException(fileName + " is not a valid HFileLink name!");
}
return(m.group(1));
return(m.group(3));
}
/**
@ -221,7 +187,7 @@ public class HFileLink extends FileLink {
* @return the name of the referenced Region
*/
public static String getReferencedRegionName(final String fileName) {
Matcher m = LINK_NAME_PARSER.matcher(fileName);
Matcher m = REF_OR_HFILE_LINK_PATTERN.matcher(fileName);
if (!m.matches()) {
throw new IllegalArgumentException(fileName + " is not a valid HFileLink name!");
}
@ -235,11 +201,11 @@ public class HFileLink extends FileLink {
* @return the name of the referenced Table
*/
public static String getReferencedTableName(final String fileName) {
Matcher m = LINK_NAME_PARSER.matcher(fileName);
Matcher m = REF_OR_HFILE_LINK_PATTERN.matcher(fileName);
if (!m.matches()) {
throw new IllegalArgumentException(fileName + " is not a valid HFileLink name!");
}
return(m.group(3));
return(m.group(1));
}
/**
@ -265,13 +231,13 @@ public class HFileLink extends FileLink {
*/
public static String createHFileLinkName(final String tableName,
final String regionName, final String hfileName) {
return String.format("%s-%s-%s", hfileName, regionName, tableName);
return String.format("%s=%s-%s", tableName, regionName, hfileName);
}
/**
* Create a new HFileLink
*
* <p>It also add a back-reference to the hfile back-reference directory
* <p>It also adds a back-reference to the hfile back-reference directory
* to simplify the reference-count and the cleaning process.
*
* @param conf {@link Configuration} to read for the archive directory name
@ -285,11 +251,34 @@ public class HFileLink extends FileLink {
public static boolean create(final Configuration conf, final FileSystem fs,
final Path dstFamilyPath, final HRegionInfo hfileRegionInfo,
final String hfileName) throws IOException {
String linkedTable = hfileRegionInfo.getTableNameAsString();
String linkedRegion = hfileRegionInfo.getEncodedName();
return create(conf, fs, dstFamilyPath, linkedTable, linkedRegion, hfileName);
}
/**
* Create a new HFileLink
*
* <p>It also adds a back-reference to the hfile back-reference directory
* to simplify the reference-count and the cleaning process.
*
* @param conf {@link Configuration} to read for the archive directory name
* @param fs {@link FileSystem} on which to write the HFileLink
* @param dstFamilyPath - Destination path (table/region/cf/)
* @param linkedTable - Linked Table Name
* @param linkedRegion - Linked Region Name
* @param hfileName - Linked HFile name
* @return true if the file is created, otherwise the file exists.
* @throws IOException on file or parent directory creation failure
*/
public static boolean create(final Configuration conf, final FileSystem fs,
final Path dstFamilyPath, final String linkedTable, final String linkedRegion,
final String hfileName) throws IOException {
String familyName = dstFamilyPath.getName();
String regionName = dstFamilyPath.getParent().getName();
String tableName = dstFamilyPath.getParent().getParent().getName();
String name = createHFileLinkName(hfileRegionInfo, hfileName);
String name = createHFileLinkName(linkedTable, linkedRegion, hfileName);
String refName = createBackReferenceName(tableName, regionName);
// Make sure the destination directory exists
@ -297,7 +286,7 @@ public class HFileLink extends FileLink {
// Make sure the FileLink reference directory exists
Path archiveStoreDir = HFileArchiveUtil.getStoreArchivePath(conf,
hfileRegionInfo.getTableNameAsString(), hfileRegionInfo.getEncodedName(), familyName);
linkedTable, linkedRegion, familyName);
Path backRefssDir = getBackReferencesDir(archiveStoreDir, hfileName);
fs.mkdirs(backRefssDir);
@ -315,6 +304,28 @@ public class HFileLink extends FileLink {
}
}
/**
* Create a new HFileLink starting from a hfileLink name
*
* <p>It also adds a back-reference to the hfile back-reference directory
* to simplify the reference-count and the cleaning process.
*
* @param conf {@link Configuration} to read for the archive directory name
* @param fs {@link FileSystem} on which to write the HFileLink
* @param dstFamilyPath - Destination path (table/region/cf/)
* @param hfileLinkName - HFileLink name (it contains hfile-region-table)
* @return true if the file is created, otherwise the file exists.
* @throws IOException on file or parent directory creation failure
*/
public static boolean createFromHFileLink(final Configuration conf, final FileSystem fs,
final Path dstFamilyPath, final String hfileLinkName) throws IOException {
Matcher m = LINK_NAME_PATTERN.matcher(hfileLinkName);
if (!m.matches()) {
throw new IllegalArgumentException(hfileLinkName + " is not a valid HFileLink name!");
}
return create(conf, fs, dstFamilyPath, m.group(1), m.group(2), m.group(3));
}
/**
* Create the back reference name
*/

View File

@ -0,0 +1,69 @@
/**
* 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.io;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* HLogLink describes a link to a WAL.
*
* An hlog can be in /hbase/.logs/<server>/<hlog>
* or it can be in /hbase/.oldlogs/<hlog>
*
* The link checks first in the original path,
* if it is not present it fallbacks to the archived path.
*/
@InterfaceAudience.Private
public class HLogLink extends FileLink {
/**
* @param conf {@link Configuration} from which to extract specific archive locations
* @param serverName Region Server owner of the log
* @param logName WAL file name
* @throws IOException on unexpected error.
*/
public HLogLink(final Configuration conf,
final String serverName, final String logName) throws IOException {
this(FSUtils.getRootDir(conf), serverName, logName);
}
/**
* @param rootDir Path to the root directory where hbase files are stored
* @param serverName Region Server owner of the log
* @param logName WAL file name
*/
public HLogLink(final Path rootDir, final String serverName, final String logName) {
final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
final Path logDir = new Path(new Path(rootDir, HConstants.HREGION_LOGDIR_NAME), serverName);
setLocations(new Path(logDir, logName), new Path(oldLogDir, logName));
}
/**
* @param originPath Path to the wal in the log directory
* @param archivePath Path to the wal in the archived log directory
*/
public HLogLink(final Path originPath, final Path archivePath) {
setLocations(originPath, archivePath);
}
}

View File

@ -59,10 +59,12 @@ public class HalfStoreFileReader extends StoreFile.Reader {
private boolean firstKeySeeked = false;
/**
* @param fs
* @param p
* Creates a half file reader for a normal hfile.
* @param fs fileystem to read from
* @param p path to hfile
* @param cacheConf
* @param r
* @param r original reference file (contains top or bottom)
* @param preferredEncodingInCache
* @throws IOException
*/
public HalfStoreFileReader(final FileSystem fs, final Path p,
@ -79,6 +81,30 @@ public class HalfStoreFileReader extends StoreFile.Reader {
this.top = Reference.isTopFileRegion(r.getFileRegion());
}
/**
* Creates a half file reader for a hfile referred to by an hfilelink.
* @param fs fileystem to read from
* @param p path to hfile
* @param link
* @param cacheConf
* @param r original reference file (contains top or bottom)
* @param preferredEncodingInCache
* @throws IOException
*/
public HalfStoreFileReader(final FileSystem fs, final Path p, final HFileLink link,
final CacheConfig cacheConf, final Reference r,
DataBlockEncoding preferredEncodingInCache) throws IOException {
super(fs, p, link, link.getFileStatus(fs).getLen(), cacheConf, preferredEncodingInCache, true);
// This is not actual midkey for this half-file; its just border
// around which we split top and bottom. Have to look in files to find
// actual last and first keys for bottom and top halves. Half-files don't
// have an actual midkey themselves. No midkey is how we indicate file is
// not splittable.
this.splitkey = r.getSplitKey();
// Is it top or bottom half?
this.top = Reference.isTopFileRegion(r.getFileRegion());
}
protected boolean isTop() {
return this.top;
}

View File

@ -97,8 +97,8 @@ import org.apache.hadoop.hbase.master.handler.ModifyTableHandler;
import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
import org.apache.hadoop.hbase.master.handler.TableEventHandler;
import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
@ -109,6 +109,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AddColumnRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AddColumnResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest;
@ -121,6 +122,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableR
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteColumnRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteColumnResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteSnapshotRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteSnapshotResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableRequest;
@ -131,6 +134,12 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableR
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ListSnapshotRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ListSnapshotResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyColumnRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyColumnResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest;
@ -139,12 +148,16 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionRe
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.RestoreSnapshotRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.RestoreSnapshotResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest;
@ -165,6 +178,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Repor
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
import org.apache.hadoop.hbase.replication.regionserver.Replication;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CompressionTest;
@ -320,6 +334,9 @@ Server {
private Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
// monitor for snapshot of hbase tables
private SnapshotManager snapshotManager;
/** The health check chore. */
private HealthCheckChore healthCheckChore;
@ -505,6 +522,7 @@ Server {
if (this.serverManager != null) this.serverManager.stop();
if (this.assignmentManager != null) this.assignmentManager.stop();
if (this.fileSystemManager != null) this.fileSystemManager.stop();
if (this.snapshotManager != null) this.snapshotManager.stop("server shutting down.");
this.zooKeeper.close();
}
LOG.info("HMaster main thread exiting");
@ -568,6 +586,9 @@ Server {
", sessionid=0x" +
Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()) +
", cluster-up flag was=" + wasUp);
// create the snapshot manager
this.snapshotManager = new SnapshotManager(this);
}
/**
@ -2015,6 +2036,7 @@ Server {
return zooKeeper;
}
@Override
public MasterCoprocessorHost getCoprocessorHost() {
return cpHost;
}
@ -2419,6 +2441,165 @@ Server {
return this.hfileCleaner;
}
/**
* Exposed for TESTING!
* @return the underlying snapshot manager
*/
public SnapshotManager getSnapshotManagerForTesting() {
return this.snapshotManager;
}
/**
* Triggers an asynchronous attempt to take a snapshot.
* {@inheritDoc}
*/
@Override
public TakeSnapshotResponse snapshot(RpcController controller, TakeSnapshotRequest request)
throws ServiceException {
try {
this.snapshotManager.checkSnapshotSupport();
} catch (UnsupportedOperationException e) {
throw new ServiceException(e);
}
LOG.debug("Submitting snapshot request for:" +
SnapshotDescriptionUtils.toString(request.getSnapshot()));
// get the snapshot information
SnapshotDescription snapshot = SnapshotDescriptionUtils.validate(request.getSnapshot(),
this.conf);
try {
snapshotManager.takeSnapshot(snapshot);
} catch (IOException e) {
throw new ServiceException(e);
}
// send back the max amount of time the client should wait for the snapshot to complete
long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout(conf, snapshot.getType(),
SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
return TakeSnapshotResponse.newBuilder().setExpectedTimeout(waitTime).build();
}
/**
* List the currently available/stored snapshots. Any in-progress snapshots are ignored
*/
@Override
public ListSnapshotResponse getCompletedSnapshots(RpcController controller,
ListSnapshotRequest request) throws ServiceException {
try {
ListSnapshotResponse.Builder builder = ListSnapshotResponse.newBuilder();
List<SnapshotDescription> snapshots = snapshotManager.getCompletedSnapshots();
// convert to protobuf
for (SnapshotDescription snapshot : snapshots) {
builder.addSnapshots(snapshot);
}
return builder.build();
} catch (IOException e) {
throw new ServiceException(e);
}
}
/**
* Execute Delete Snapshot operation.
* @return DeleteSnapshotResponse (a protobuf wrapped void) if the snapshot existed and was
* deleted properly.
* @throws ServiceException wrapping SnapshotDoesNotExistException if specified snapshot did not
* exist.
*/
@Override
public DeleteSnapshotResponse deleteSnapshot(RpcController controller,
DeleteSnapshotRequest request) throws ServiceException {
try {
this.snapshotManager.checkSnapshotSupport();
} catch (UnsupportedOperationException e) {
throw new ServiceException(e);
}
try {
snapshotManager.deleteSnapshot(request.getSnapshot());
return DeleteSnapshotResponse.newBuilder().build();
} catch (IOException e) {
throw new ServiceException(e);
}
}
/**
* Checks if the specified snapshot is done.
* @return true if the snapshot is in file system ready to use,
* false if the snapshot is in the process of completing
* @throws ServiceException wrapping UnknownSnapshotException if invalid snapshot, or
* a wrapped HBaseSnapshotException with progress failure reason.
*/
@Override
public IsSnapshotDoneResponse isSnapshotDone(RpcController controller,
IsSnapshotDoneRequest request) throws ServiceException {
LOG.debug("Checking to see if snapshot from request:" +
SnapshotDescriptionUtils.toString(request.getSnapshot()) + " is done");
try {
IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder();
boolean done = snapshotManager.isSnapshotDone(request.getSnapshot());
builder.setDone(done);
return builder.build();
} catch (IOException e) {
throw new ServiceException(e);
}
}
/**
* Execute Restore/Clone snapshot operation.
*
* <p>If the specified table exists a "Restore" is executed, replacing the table
* schema and directory data with the content of the snapshot.
* The table must be disabled, or a UnsupportedOperationException will be thrown.
*
* <p>If the table doesn't exist a "Clone" is executed, a new table is created
* using the schema at the time of the snapshot, and the content of the snapshot.
*
* <p>The restore/clone operation does not require copying HFiles. Since HFiles
* are immutable the table can point to and use the same files as the original one.
*/
@Override
public RestoreSnapshotResponse restoreSnapshot(RpcController controller,
RestoreSnapshotRequest request) throws ServiceException {
try {
this.snapshotManager.checkSnapshotSupport();
} catch (UnsupportedOperationException e) {
throw new ServiceException(e);
}
try {
SnapshotDescription reqSnapshot = request.getSnapshot();
snapshotManager.restoreSnapshot(reqSnapshot);
return RestoreSnapshotResponse.newBuilder().build();
} catch (IOException e) {
throw new ServiceException(e);
}
}
/**
* Returns the status of the requested snapshot restore/clone operation.
* This method is not exposed to the user, it is just used internally by HBaseAdmin
* to verify if the restore is completed.
*
* No exceptions are thrown if the restore is not running, the result will be "done".
*
* @return done <tt>true</tt> if the restore/clone operation is completed.
* @throws RestoreSnapshotExcepton if the operation failed.
*/
@Override
public IsRestoreSnapshotDoneResponse isRestoreSnapshotDone(RpcController controller,
IsRestoreSnapshotDoneRequest request) throws ServiceException {
try {
SnapshotDescription snapshot = request.getSnapshot();
IsRestoreSnapshotDoneResponse.Builder builder = IsRestoreSnapshotDoneResponse.newBuilder();
boolean isRestoring = snapshotManager.isRestoringTable(snapshot);
builder.setDone(!isRestoring);
return builder.build();
} catch (IOException e) {
throw new ServiceException(e);
}
}
private boolean isHealthCheckerConfigured() {
String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation);

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.coprocessor.*;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import java.io.IOException;
import java.util.List;
@ -160,6 +161,7 @@ public class MasterCoprocessorHost
}
}
}
public void preDeleteTable(byte[] tableName) throws IOException {
ObserverContext<MasterCoprocessorEnvironment> ctx = null;
for (MasterEnvironment env: coprocessors) {
@ -935,4 +937,146 @@ public class MasterCoprocessorHost
}
}
}
public void preSnapshot(final SnapshotDescription snapshot,
final HTableDescriptor hTableDescriptor) throws IOException {
ObserverContext<MasterCoprocessorEnvironment> ctx = null;
for (MasterEnvironment env: coprocessors) {
if (env.getInstance() instanceof MasterObserver) {
ctx = ObserverContext.createAndPrepare(env, ctx);
try {
((MasterObserver)env.getInstance()).preSnapshot(ctx, snapshot, hTableDescriptor);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
}
if (ctx.shouldComplete()) {
break;
}
}
}
}
public void postSnapshot(final SnapshotDescription snapshot,
final HTableDescriptor hTableDescriptor) throws IOException {
ObserverContext<MasterCoprocessorEnvironment> ctx = null;
for (MasterEnvironment env: coprocessors) {
if (env.getInstance() instanceof MasterObserver) {
ctx = ObserverContext.createAndPrepare(env, ctx);
try {
((MasterObserver)env.getInstance()).postSnapshot(ctx, snapshot, hTableDescriptor);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
}
if (ctx.shouldComplete()) {
break;
}
}
}
}
public void preCloneSnapshot(final SnapshotDescription snapshot,
final HTableDescriptor hTableDescriptor) throws IOException {
ObserverContext<MasterCoprocessorEnvironment> ctx = null;
for (MasterEnvironment env: coprocessors) {
if (env.getInstance() instanceof MasterObserver) {
ctx = ObserverContext.createAndPrepare(env, ctx);
try {
((MasterObserver)env.getInstance()).preCloneSnapshot(ctx, snapshot, hTableDescriptor);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
}
if (ctx.shouldComplete()) {
break;
}
}
}
}
public void postCloneSnapshot(final SnapshotDescription snapshot,
final HTableDescriptor hTableDescriptor) throws IOException {
ObserverContext<MasterCoprocessorEnvironment> ctx = null;
for (MasterEnvironment env: coprocessors) {
if (env.getInstance() instanceof MasterObserver) {
ctx = ObserverContext.createAndPrepare(env, ctx);
try {
((MasterObserver)env.getInstance()).postCloneSnapshot(ctx, snapshot, hTableDescriptor);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
}
if (ctx.shouldComplete()) {
break;
}
}
}
}
public void preRestoreSnapshot(final SnapshotDescription snapshot,
final HTableDescriptor hTableDescriptor) throws IOException {
ObserverContext<MasterCoprocessorEnvironment> ctx = null;
for (MasterEnvironment env: coprocessors) {
if (env.getInstance() instanceof MasterObserver) {
ctx = ObserverContext.createAndPrepare(env, ctx);
try {
((MasterObserver)env.getInstance()).preRestoreSnapshot(ctx, snapshot, hTableDescriptor);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
}
if (ctx.shouldComplete()) {
break;
}
}
}
}
public void postRestoreSnapshot(final SnapshotDescription snapshot,
final HTableDescriptor hTableDescriptor) throws IOException {
ObserverContext<MasterCoprocessorEnvironment> ctx = null;
for (MasterEnvironment env: coprocessors) {
if (env.getInstance() instanceof MasterObserver) {
ctx = ObserverContext.createAndPrepare(env, ctx);
try {
((MasterObserver)env.getInstance()).postRestoreSnapshot(ctx, snapshot, hTableDescriptor);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
}
if (ctx.shouldComplete()) {
break;
}
}
}
}
public void preDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
ObserverContext<MasterCoprocessorEnvironment> ctx = null;
for (MasterEnvironment env: coprocessors) {
if (env.getInstance() instanceof MasterObserver) {
ctx = ObserverContext.createAndPrepare(env, ctx);
try {
((MasterObserver)env.getInstance()).preDeleteSnapshot(ctx, snapshot);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
}
if (ctx.shouldComplete()) {
break;
}
}
}
}
public void postDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
ObserverContext<MasterCoprocessorEnvironment> ctx = null;
for (MasterEnvironment env: coprocessors) {
if (env.getInstance() instanceof MasterObserver) {
ctx = ObserverContext.createAndPrepare(env, ctx);
try {
((MasterObserver)env.getInstance()).postDeleteSnapshot(ctx, snapshot);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
}
if (ctx.shouldComplete()) {
break;
}
}
}
}
}

View File

@ -79,6 +79,8 @@ public class MasterFileSystem {
private final Path oldLogDir;
// root hbase directory on the FS
private final Path rootdir;
// hbase temp directory used for table construction and deletion
private final Path tempdir;
// create the split log lock
final Lock splitLogLock = new ReentrantLock();
final boolean distributedLogSplitting;
@ -109,6 +111,7 @@ public class MasterFileSystem {
// default localfs. Presumption is that rootdir is fully-qualified before
// we get to here with appropriate fs scheme.
this.rootdir = FSUtils.getRootDir(conf);
this.tempdir = new Path(this.rootdir, HConstants.HBASE_TEMP_DIRECTORY);
// Cover both bases, the old way of setting default fs and the new.
// We're supposed to run on 0.20 and 0.21 anyways.
this.fs = this.rootdir.getFileSystem(conf);
@ -146,6 +149,9 @@ public class MasterFileSystem {
// check if the root directory exists
checkRootDir(this.rootdir, conf, this.fs);
// check if temp directory exists and clean it
checkTempDir(this.tempdir, conf, this.fs);
Path oldLogDir = new Path(this.rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
// Make sure the region servers can archive their old logs
@ -193,6 +199,13 @@ public class MasterFileSystem {
return this.rootdir;
}
/**
* @return HBase temp dir.
*/
public Path getTempDir() {
return this.tempdir;
}
/**
* @return The unique identifier generated for this cluster
*/
@ -439,6 +452,32 @@ public class MasterFileSystem {
return rd;
}
/**
* Make sure the hbase temp directory exists and is empty.
* NOTE that this method is only executed once just after the master becomes the active one.
*/
private void checkTempDir(final Path tmpdir, final Configuration c, final FileSystem fs)
throws IOException {
// If the temp directory exists, clear the content (left over, from the previous run)
if (fs.exists(tmpdir)) {
// Archive table in temp, maybe left over from failed deletion,
// if not the cleaner will take care of them.
for (Path tabledir: FSUtils.getTableDirs(fs, tmpdir)) {
for (Path regiondir: FSUtils.getRegionDirs(fs, tabledir)) {
HFileArchiver.archiveRegion(fs, this.rootdir, tabledir, regiondir);
}
}
if (!fs.delete(tmpdir, true)) {
throw new IOException("Unable to clean the temp directory: " + tmpdir);
}
}
// Create the temp directory
if (!fs.mkdirs(tmpdir)) {
throw new IOException("HBase temp directory '" + tmpdir + "' creation failure.");
}
}
private static void bootstrap(final Path rd, final Configuration c)
throws IOException {
LOG.info("BOOTSTRAP: creating ROOT and first META regions");
@ -503,6 +542,37 @@ public class MasterFileSystem {
fs.delete(new Path(rootdir, Bytes.toString(tableName)), true);
}
/**
* Move the specified file/directory to the hbase temp directory.
* @param path The path of the file/directory to move
* @return The temp location of the file/directory moved
* @throws IOException in case of file-system failure
*/
public Path moveToTemp(final Path path) throws IOException {
Path tempPath = new Path(this.tempdir, path.getName());
// Ensure temp exists
if (!fs.exists(tempdir) && !fs.mkdirs(tempdir)) {
throw new IOException("HBase temp directory '" + tempdir + "' creation failure.");
}
if (!fs.rename(path, tempPath)) {
throw new IOException("Unable to move '" + path + "' to temp '" + tempPath + "'");
}
return tempPath;
}
/**
* Move the specified table to the hbase temp directory
* @param tableName Table name to move
* @return The temp location of the table moved
* @throws IOException in case of file-system failure
*/
public Path moveTableToTemp(byte[] tableName) throws IOException {
return moveToTemp(HTableDescriptor.getTableDir(this.rootdir, tableName));
}
public void updateRegionInfo(HRegionInfo region) {
// TODO implement this. i think this is currently broken in trunk i don't
// see this getting updated.

View File

@ -55,6 +55,11 @@ public interface MasterServices extends Server {
*/
public ExecutorService getExecutorService();
/**
* @return Master's instance of {@link MasterCoprocessorHost}
*/
public MasterCoprocessorHost getCoprocessorHost();
/**
* Check table is modifiable; i.e. exists and is offline.
* @param tableName Name of table to check.

View File

@ -0,0 +1,57 @@
/**
* 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;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
/**
* Watch the current snapshot under process
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public interface SnapshotSentinel {
/**
* Check to see if the snapshot is finished, where finished may be success or failure.
* @return <tt>false</tt> if the snapshot is still in progress, <tt>true</tt> if the snapshot has
* finished
*/
public boolean isFinished();
/**
* Actively cancel a running snapshot.
* @param why Reason for cancellation.
*/
public void cancel(String why);
/**
* @return the description of the snapshot being run
*/
public SnapshotDescription getSnapshot();
/**
* Get the exception that caused the snapshot to fail, if the snapshot has failed.
* @return {@link ForeignException} that caused the snapshot to fail, or <tt>null</tt> if the
* snapshot is still in progress or has succeeded
*/
public ForeignException getExceptionIfFailed();
}

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.master.handler;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.CompletionService;
@ -36,6 +35,8 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
@ -60,9 +61,9 @@ import org.apache.zookeeper.KeeperException;
@InterfaceAudience.Private
public class CreateTableHandler extends EventHandler {
private static final Log LOG = LogFactory.getLog(CreateTableHandler.class);
private MasterFileSystem fileSystemManager;
private final HTableDescriptor hTableDescriptor;
private Configuration conf;
protected final MasterFileSystem fileSystemManager;
protected final HTableDescriptor hTableDescriptor;
protected final Configuration conf;
private final AssignmentManager assignmentManager;
private final CatalogTracker catalogTracker;
private final HRegionInfo [] newRegions;
@ -88,7 +89,9 @@ public class CreateTableHandler extends EventHandler {
}
} catch (InterruptedException e) {
LOG.warn("Interrupted waiting for meta availability", e);
throw new IOException(e);
InterruptedIOException ie = new InterruptedIOException(e.getMessage());
ie.initCause(e);
throw ie;
}
String tableName = this.hTableDescriptor.getNameAsString();
@ -133,28 +136,95 @@ public class CreateTableHandler extends EventHandler {
cpHost.preCreateTableHandler(this.hTableDescriptor, this.newRegions);
}
handleCreateTable(tableName);
completed(null);
if (cpHost != null) {
cpHost.postCreateTableHandler(this.hTableDescriptor, this.newRegions);
}
} catch (IOException e) {
LOG.error("Error trying to create the table " + tableName, e);
} catch (KeeperException e) {
} catch (Throwable e) {
LOG.error("Error trying to create the table " + tableName, e);
completed(e);
}
}
private void handleCreateTable(String tableName) throws IOException,
KeeperException {
/**
* Called after that process() is completed.
* @param exception null if process() is successful or not null if something has failed.
*/
protected void completed(final Throwable exception) {
}
/**
* Responsible of table creation (on-disk and META) and assignment.
* - Create the table directory and descriptor (temp folder)
* - Create the on-disk regions (temp folder)
* [If something fails here: we've just some trash in temp]
* - Move the table from temp to the root directory
* [If something fails here: we've the table in place but some of the rows required
* present in META. (hbck needed)]
* - Add regions to META
* [If something fails here: we don't have regions assigned: table disabled]
* - Assign regions to Region Servers
* [If something fails here: we still have the table in disabled state]
* - Update ZooKeeper with the enabled state
*/
private void handleCreateTable(String tableName) throws IOException, KeeperException {
Path tempdir = fileSystemManager.getTempDir();
FileSystem fs = fileSystemManager.getFileSystem();
// 1. Create Table Descriptor
FSTableDescriptors.createTableDescriptor(fs, tempdir, this.hTableDescriptor);
Path tempTableDir = new Path(tempdir, tableName);
Path tableDir = new Path(fileSystemManager.getRootDir(), tableName);
// 2. Create Regions
List<HRegionInfo> regionInfos = handleCreateHdfsRegions(tempdir, tableName);
// 3. Move Table temp directory to the hbase root location
if (!fs.rename(tempTableDir, tableDir)) {
throw new IOException("Unable to move table from temp=" + tempTableDir +
" to hbase root=" + tableDir);
}
if (regionInfos != null && regionInfos.size() > 0) {
// 4. Add regions to META
MetaEditor.addRegionsToMeta(this.catalogTracker, regionInfos);
// 5. Trigger immediate assignment of the regions in round-robin fashion
try {
assignmentManager.getRegionStates().createRegionStates(regionInfos);
assignmentManager.assign(regionInfos);
} catch (InterruptedException e) {
LOG.error("Caught " + e + " during round-robin assignment");
InterruptedIOException ie = new InterruptedIOException(e.getMessage());
ie.initCause(e);
throw ie;
}
}
// 6. Set table enabled flag up in zk.
try {
assignmentManager.getZKTable().setEnabledTable(tableName);
} catch (KeeperException e) {
throw new IOException("Unable to ensure that " + tableName + " will be" +
" enabled because of a ZooKeeper issue", e);
}
}
/**
* Create the on-disk structure for the table, and returns the regions info.
* @param tableRootDir directory where the table is being created
* @param tableName name of the table under construction
* @return the list of regions created
*/
protected List<HRegionInfo> handleCreateHdfsRegions(final Path tableRootDir,
final String tableName)
throws IOException {
int regionNumber = newRegions.length;
ThreadPoolExecutor regionOpenAndInitThreadPool = getRegionOpenAndInitThreadPool(
"RegionOpenAndInitThread-" + tableName, regionNumber);
CompletionService<HRegion> completionService = new ExecutorCompletionService<HRegion>(
regionOpenAndInitThreadPool);
// TODO: Currently we make the table descriptor and as side-effect the
// tableDir is created. Should we change below method to be createTable
// where we create table in tmp dir with its table descriptor file and then
// do rename to move it into place?
FSTableDescriptors.createTableDescriptor(this.hTableDescriptor, this.conf);
List<HRegionInfo> regionInfos = new ArrayList<HRegionInfo>();
for (final HRegionInfo newRegion : newRegions) {
completionService.submit(new Callable<HRegion>() {
@ -162,9 +232,8 @@ public class CreateTableHandler extends EventHandler {
// 1. Create HRegion
HRegion region = HRegion.createHRegion(newRegion,
fileSystemManager.getRootDir(), conf, hTableDescriptor, null,
tableRootDir, conf, hTableDescriptor, null,
false, true);
// 2. Close the new region to flush to disk. Close log file too.
region.close();
return region;
@ -185,28 +254,8 @@ public class CreateTableHandler extends EventHandler {
} finally {
regionOpenAndInitThreadPool.shutdownNow();
}
if (regionInfos.size() > 0) {
MetaEditor.addRegionsToMeta(this.catalogTracker, regionInfos);
}
// 4. Trigger immediate assignment of the regions in round-robin fashion
try {
List<HRegionInfo> regions = Arrays.asList(newRegions);
assignmentManager.getRegionStates().createRegionStates(regions);
assignmentManager.assign(regions);
} catch (InterruptedException ie) {
LOG.error("Caught " + ie + " during round-robin assignment");
throw new IOException(ie);
}
// 5. Set table enabled flag up in zk.
try {
assignmentManager.getZKTable().
setEnabledTable(this.hTableDescriptor.getNameAsString());
} catch (KeeperException e) {
throw new IOException("Unable to ensure that the table will be" +
" enabled because of a ZooKeeper issue", e);
}
return regionInfos;
}
protected ThreadPoolExecutor getRegionOpenAndInitThreadPool(

View File

@ -24,12 +24,16 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
@ -55,6 +59,8 @@ public class DeleteTableHandler extends TableEventHandler {
if (cpHost != null) {
cpHost.preDeleteTableHandler(this.tableName);
}
// 1. Wait because of region in transition
AssignmentManager am = this.masterServices.getAssignmentManager();
long waitTime = server.getConfiguration().
getLong("hbase.master.wait.on.region", 5 * 60 * 1000);
@ -71,21 +77,37 @@ public class DeleteTableHandler extends TableEventHandler {
waitTime + "ms) for region to leave region " +
region.getRegionNameAsString() + " in transitions");
}
LOG.debug("Deleting region " + region.getRegionNameAsString() +
" from META and FS");
// Remove region from META
MetaEditor.deleteRegion(this.server.getCatalogTracker(), region);
// Delete region from FS
this.masterServices.getMasterFileSystem().deleteRegion(region);
}
// Delete table from FS
this.masterServices.getMasterFileSystem().deleteTable(tableName);
// Update table descriptor cache
this.masterServices.getTableDescriptors().remove(Bytes.toString(tableName));
// If entry for this table in zk, and up in AssignmentManager, remove it.
// 2. Remove regions from META
LOG.debug("Deleting regions from META");
MetaEditor.deleteRegions(this.server.getCatalogTracker(), regions);
// 3. Move the table in /hbase/.tmp
MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
Path tempTableDir = mfs.moveTableToTemp(tableName);
try {
// 4. Delete regions from FS (temp directory)
FileSystem fs = mfs.getFileSystem();
for (HRegionInfo hri: regions) {
LOG.debug("Archiving region " + hri.getRegionNameAsString() + " from FS");
HFileArchiver.archiveRegion(fs, mfs.getRootDir(),
tempTableDir, new Path(tempTableDir, hri.getEncodedName()));
}
// 5. Delete table from FS (temp directory)
if (!fs.delete(tempTableDir, true)) {
LOG.error("Couldn't delete " + tempTableDir);
}
} finally {
// 6. Update table descriptor cache
this.masterServices.getTableDescriptors().remove(Bytes.toString(tableName));
// 7. If entry for this table in zk, and up in AssignmentManager, remove it.
am.getZKTable().setDeletedTable(Bytes.toString(tableName));
}
am.getZKTable().setDeletedTable(Bytes.toString(tableName));
if (cpHost != null) {
cpHost.postDeleteTableHandler(this.tableName);
}

View File

@ -184,6 +184,7 @@ public class DisableTableHandler extends EventHandler {
while (!server.isStopped() && remaining > 0) {
Thread.sleep(waitingTimeForEvents);
regions = assignmentManager.getRegionStates().getRegionsOfTable(tableName);
LOG.debug("Disable waiting until done; " + remaining + " ms remaining; " + regions);
if (regions.isEmpty()) break;
remaining = timeout - (System.currentTimeMillis() - startTime);
}

View File

@ -163,12 +163,14 @@ public abstract class TableEventHandler extends EventHandler {
/**
* Gets a TableDescriptor from the masterServices. Can Throw exceptions.
*
* @return Table descriptor for this table
* @throws TableExistsException
* @throws FileNotFoundException
* @throws IOException
*/
HTableDescriptor getTableDescriptor()
public HTableDescriptor getTableDescriptor()
throws FileNotFoundException, IOException {
final String name = Bytes.toString(tableName);
HTableDescriptor htd =

View File

@ -0,0 +1,149 @@
/**
*
* 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.snapshot;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CancellationException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.SnapshotSentinel;
import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions;
/**
* Handler to Clone a snapshot.
*
* <p>Uses {@link RestoreSnapshotHelper} to create a new table with the same
* content of the specified snapshot.
*/
@InterfaceAudience.Private
public class CloneSnapshotHandler extends CreateTableHandler implements SnapshotSentinel {
private static final Log LOG = LogFactory.getLog(CloneSnapshotHandler.class);
private final static String NAME = "Master CloneSnapshotHandler";
private final SnapshotDescription snapshot;
private final ForeignExceptionDispatcher monitor;
private volatile boolean stopped = false;
public CloneSnapshotHandler(final MasterServices masterServices,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws NotAllMetaRegionsOnlineException, TableExistsException, IOException {
super(masterServices, masterServices.getMasterFileSystem(), hTableDescriptor,
masterServices.getConfiguration(), null, masterServices.getCatalogTracker(),
masterServices.getAssignmentManager());
// Snapshot information
this.snapshot = snapshot;
// Monitor
this.monitor = new ForeignExceptionDispatcher();
}
/**
* Create the on-disk regions, using the tableRootDir provided by the CreateTableHandler.
* The cloned table will be created in a temp directory, and then the CreateTableHandler
* will be responsible to add the regions returned by this method to META and do the assignment.
*/
@Override
protected List<HRegionInfo> handleCreateHdfsRegions(final Path tableRootDir, final String tableName)
throws IOException {
FileSystem fs = fileSystemManager.getFileSystem();
Path rootDir = fileSystemManager.getRootDir();
Path tableDir = new Path(tableRootDir, tableName);
try {
// 1. Execute the on-disk Clone
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(conf, fs,
snapshot, snapshotDir, hTableDescriptor, tableDir, monitor);
RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions();
// Clone operation should not have stuff to restore or remove
Preconditions.checkArgument(!metaChanges.hasRegionsToRestore(),
"A clone should not have regions to restore");
Preconditions.checkArgument(!metaChanges.hasRegionsToRemove(),
"A clone should not have regions to remove");
// At this point the clone is complete. Next step is enabling the table.
LOG.info("Clone snapshot=" + snapshot.getName() + " on table=" + tableName + " completed!");
// 2. let the CreateTableHandler add the regions to meta
return metaChanges.getRegionsToAdd();
} catch (Exception e) {
String msg = "clone snapshot=" + SnapshotDescriptionUtils.toString(snapshot) + " failed";
LOG.error(msg, e);
IOException rse = new RestoreSnapshotException(msg, e, snapshot);
// these handlers aren't futures so we need to register the error here.
this.monitor.receive(new ForeignException(NAME, rse));
throw rse;
}
}
@Override
protected void completed(final Throwable exception) {
this.stopped = true;
}
@Override
public boolean isFinished() {
return this.stopped;
}
@Override
public SnapshotDescription getSnapshot() {
return snapshot;
}
@Override
public void cancel(String why) {
if (this.stopped) return;
this.stopped = true;
LOG.info("Stopping clone snapshot=" + snapshot + " because: " + why);
this.monitor.receive(new ForeignException(NAME, new CancellationException(why)));
}
@Override
public ForeignException getExceptionIfFailed() {
return this.monitor.getException();
}
}

View File

@ -0,0 +1,131 @@
/**
* 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.snapshot;
import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.snapshot.CopyRecoveredEditsTask;
import org.apache.hadoop.hbase.snapshot.ReferenceRegionHFilesTask;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.TableInfoCopyTask;
import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.zookeeper.KeeperException;
/**
* Take a snapshot of a disabled table.
* <p>
* Table must exist when taking the snapshot, or results are undefined.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
private static final Log LOG = LogFactory.getLog(DisabledTableSnapshotHandler.class);
private final TimeoutExceptionInjector timeoutInjector;
/**
* @param snapshot descriptor of the snapshot to take
* @param masterServices master services provider
* @throws IOException on unexpected error
*/
public DisabledTableSnapshotHandler(SnapshotDescription snapshot,
final MasterServices masterServices) throws IOException {
super(snapshot, masterServices);
// setup the timer
timeoutInjector = TakeSnapshotUtils.getMasterTimerAndBindToMonitor(snapshot, conf, monitor);
}
// TODO consider parallelizing these operations since they are independent. Right now its just
// easier to keep them serial though
@Override
public void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regionsAndLocations) throws IOException,
KeeperException {
try {
timeoutInjector.start();
// 1. get all the regions hosting this table.
// extract each pair to separate lists
Set<String> serverNames = new HashSet<String>();
Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
regions.add(p.getFirst());
serverNames.add(p.getSecond().toString());
}
// 2. for each region, write all the info to disk
LOG.info("Starting to write region info and WALs for regions for offline snapshot:"
+ SnapshotDescriptionUtils.toString(snapshot));
for (HRegionInfo regionInfo : regions) {
// 2.1 copy the regionInfo files to the snapshot
Path snapshotRegionDir = TakeSnapshotUtils.getRegionSnapshotDirectory(snapshot, rootDir,
regionInfo.getEncodedName());
HRegion.writeRegioninfoOnFilesystem(regionInfo, snapshotRegionDir, fs, conf);
// check for error for each region
monitor.rethrowException();
// 2.2 for each region, copy over its recovered.edits directory
Path regionDir = HRegion.getRegionDir(rootDir, regionInfo);
new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir, snapshotRegionDir).call();
monitor.rethrowException();
// 2.3 reference all the files in the region
new ReferenceRegionHFilesTask(snapshot, monitor, regionDir, fs, snapshotRegionDir).call();
monitor.rethrowException();
}
// 3. write the table info to disk
LOG.info("Starting to copy tableinfo for offline snapshot: " +
SnapshotDescriptionUtils.toString(snapshot));
TableInfoCopyTask tableInfoCopyTask = new TableInfoCopyTask(this.monitor, snapshot, fs,
FSUtils.getRootDir(conf));
tableInfoCopyTask.call();
monitor.rethrowException();
} catch (Exception e) {
// make sure we capture the exception to propagate back to the client later
String reason = "Failed snapshot " + SnapshotDescriptionUtils.toString(snapshot)
+ " due to exception:" + e.getMessage();
ForeignException ee = new ForeignException(reason, e);
monitor.receive(ee);
} finally {
LOG.debug("Marking snapshot" + SnapshotDescriptionUtils.toString(snapshot)
+ " as finished.");
// 6. mark the timer as finished - even if we got an exception, we don't need to time the
// operation any further
timeoutInjector.complete();
}
}
}

View File

@ -0,0 +1,97 @@
/**
* 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.snapshot;
import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.procedure.Procedure;
import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
import org.apache.hadoop.hbase.util.Pair;
import com.google.common.collect.Lists;
/**
* Handle the master side of taking a snapshot of an online table, regardless of snapshot type.
* Uses a {@link Procedure} to run the snapshot across all the involved region servers.
* @see ProcedureCoordinator
*/
@InterfaceAudience.Private
public class EnabledTableSnapshotHandler extends TakeSnapshotHandler {
private static final Log LOG = LogFactory.getLog(EnabledTableSnapshotHandler.class);
private final ProcedureCoordinator coordinator;
public EnabledTableSnapshotHandler(SnapshotDescription snapshot, MasterServices master,
SnapshotManager manager) throws IOException {
super(snapshot, master);
this.coordinator = manager.getCoordinator();
}
// TODO consider switching over to using regionnames, rather than server names. This would allow
// regions to migrate during a snapshot, and then be involved when they are ready. Still want to
// enforce a snapshot time constraints, but lets us be potentially a bit more robust.
/**
* This method kicks off a snapshot procedure. Other than that it hangs around for various
* phases to complete.
*/
@Override
protected void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regions)
throws HBaseSnapshotException {
Set<String> regionServers = new HashSet<String>(regions.size());
for (Pair<HRegionInfo, ServerName> region : regions) {
regionServers.add(region.getSecond().toString());
}
// start the snapshot on the RS
Procedure proc = coordinator.startProcedure(this.monitor, this.snapshot.getName(),
this.snapshot.toByteArray(), Lists.newArrayList(regionServers));
if (proc == null) {
String msg = "Failed to submit distributed procedure for snapshot '"
+ snapshot.getName() + "'";
LOG.error(msg);
throw new HBaseSnapshotException(msg);
}
try {
// wait for the snapshot to complete. A timer thread is kicked off that should cancel this
// if it takes too long.
proc.waitForCompleted();
LOG.info("Done waiting - snapshot for " + this.snapshot.getName() + " finished!");
} catch (InterruptedException e) {
ForeignException ee =
new ForeignException("Interrupted while waiting for snapshot to finish", e);
monitor.receive(ee);
Thread.currentThread().interrupt();
} catch (ForeignException e) {
monitor.receive(e);
}
}
}

View File

@ -0,0 +1,233 @@
/**
* 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.snapshot;
import java.io.IOException;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
/**
* General snapshot verification on the master.
* <p>
* This is a light-weight verification mechanism for all the files in a snapshot. It doesn't
* attempt to verify that the files are exact copies (that would be paramount to taking the
* snapshot again!), but instead just attempts to ensure that the files match the expected
* files and are the same length.
* <p>
* Taking an online snapshots can race against other operations and this is an last line of
* defense. For example, if meta changes between when snapshots are taken not all regions of a
* table may be present. This can be caused by a region split (daughters present on this scan,
* but snapshot took parent), or move (snapshots only checks lists of region servers, a move could
* have caused a region to be skipped or done twice).
* <p>
* Current snapshot files checked:
* <ol>
* <li>SnapshotDescription is readable</li>
* <li>Table info is readable</li>
* <li>Regions</li>
* <ul>
* <li>Matching regions in the snapshot as currently in the table</li>
* <li>{@link HRegionInfo} matches the current and stored regions</li>
* <li>All referenced hfiles have valid names</li>
* <li>All the hfiles are present (either in .archive directory in the region)</li>
* <li>All recovered.edits files are present (by name) and have the correct file size</li>
* </ul>
* </ol>
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public final class MasterSnapshotVerifier {
private SnapshotDescription snapshot;
private FileSystem fs;
private Path rootDir;
private String tableName;
private MasterServices services;
/**
* @param services services for the master
* @param snapshot snapshot to check
* @param rootDir root directory of the hbase installation.
*/
public MasterSnapshotVerifier(MasterServices services, SnapshotDescription snapshot, Path rootDir) {
this.fs = services.getMasterFileSystem().getFileSystem();
this.services = services;
this.snapshot = snapshot;
this.rootDir = rootDir;
this.tableName = snapshot.getTable();
}
/**
* Verify that the snapshot in the directory is a valid snapshot
* @param snapshotDir snapshot directory to check
* @param snapshotServers {@link ServerName} of the servers that are involved in the snapshot
* @throws CorruptedSnapshotException if the snapshot is invalid
* @throws IOException if there is an unexpected connection issue to the filesystem
*/
public void verifySnapshot(Path snapshotDir, Set<String> snapshotServers)
throws CorruptedSnapshotException, IOException {
// verify snapshot info matches
verifySnapshotDescription(snapshotDir);
// check that tableinfo is a valid table description
verifyTableInfo(snapshotDir);
// check that each region is valid
verifyRegions(snapshotDir);
}
/**
* Check that the snapshot description written in the filesystem matches the current snapshot
* @param snapshotDir snapshot directory to check
*/
private void verifySnapshotDescription(Path snapshotDir) throws CorruptedSnapshotException {
SnapshotDescription found = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
if (!this.snapshot.equals(found)) {
throw new CorruptedSnapshotException("Snapshot read (" + found
+ ") doesn't equal snapshot we ran (" + snapshot + ").", snapshot);
}
}
/**
* Check that the table descriptor for the snapshot is a valid table descriptor
* @param snapshotDir snapshot directory to check
*/
private void verifyTableInfo(Path snapshotDir) throws IOException {
FSTableDescriptors.getTableDescriptor(fs, snapshotDir);
}
/**
* Check that all the regions in the snapshot are valid, and accounted for.
* @param snapshotDir snapshot directory to check
* @throws IOException if we can't reach .META. or read the files from the FS
*/
private void verifyRegions(Path snapshotDir) throws IOException {
List<HRegionInfo> regions = MetaReader.getTableRegions(this.services.getCatalogTracker(),
Bytes.toBytes(tableName));
for (HRegionInfo region : regions) {
// if offline split parent, skip it
if (region.isOffline() && (region.isSplit() || region.isSplitParent())) {
continue;
}
verifyRegion(fs, snapshotDir, region);
}
}
/**
* Verify that the region (regioninfo, hfiles) are valid
* @param fs the FileSystem instance
* @param snapshotDir snapshot directory to check
* @param region the region to check
*/
private void verifyRegion(FileSystem fs, Path snapshotDir, HRegionInfo region) throws IOException {
// make sure we have region in the snapshot
Path regionDir = new Path(snapshotDir, region.getEncodedName());
if (!fs.exists(regionDir)) {
// could happen due to a move or split race.
throw new CorruptedSnapshotException("No region directory found for region:" + region,
snapshot);
}
// make sure we have the region info in the snapshot
Path regionInfo = new Path(regionDir, HRegion.REGIONINFO_FILE);
// make sure the file exists
if (!fs.exists(regionInfo)) {
throw new CorruptedSnapshotException("No region info found for region:" + region, snapshot);
}
FSDataInputStream in = fs.open(regionInfo);
HRegionInfo found;
try {
found = HRegionInfo.parseFrom(in);
if (!region.equals(found)) {
throw new CorruptedSnapshotException("Found region info (" + found
+ ") doesn't match expected region:" + region, snapshot);
}
} finally {
in.close();
}
// make sure we have the expected recovered edits files
TakeSnapshotUtils.verifyRecoveredEdits(fs, snapshotDir, found, snapshot);
// check for the existance of each hfile
PathFilter familiesDirs = new FSUtils.FamilyDirFilter(fs);
FileStatus[] columnFamilies = FSUtils.listStatus(fs, regionDir, familiesDirs);
// should we do some checking here to make sure the cfs are correct?
if (columnFamilies == null) return;
// setup the suffixes for the snapshot directories
Path tableNameSuffix = new Path(tableName);
Path regionNameSuffix = new Path(tableNameSuffix, region.getEncodedName());
// get the potential real paths
Path archivedRegion = new Path(HFileArchiveUtil.getArchivePath(services.getConfiguration()),
regionNameSuffix);
Path realRegion = new Path(rootDir, regionNameSuffix);
// loop through each cf and check we can find each of the hfiles
for (FileStatus cf : columnFamilies) {
FileStatus[] hfiles = FSUtils.listStatus(fs, cf.getPath(), null);
// should we check if there should be hfiles?
if (hfiles == null || hfiles.length == 0) continue;
Path realCfDir = new Path(realRegion, cf.getPath().getName());
Path archivedCfDir = new Path(archivedRegion, cf.getPath().getName());
for (FileStatus hfile : hfiles) {
// make sure the name is correct
if (!StoreFile.validateStoreFileName(hfile.getPath().getName())) {
throw new CorruptedSnapshotException("HFile: " + hfile.getPath()
+ " is not a valid hfile name.", snapshot);
}
// check to see if hfile is present in the real table
String fileName = hfile.getPath().getName();
Path file = new Path(realCfDir, fileName);
Path archived = new Path(archivedCfDir, fileName);
if (!fs.exists(file) && !file.equals(archived)) {
throw new CorruptedSnapshotException("Can't find hfile: " + hfile.getPath()
+ " in the real (" + realCfDir + ") or archive (" + archivedCfDir
+ ") directory for the primary table.", snapshot);
}
}
}
}
}

View File

@ -0,0 +1,155 @@
/**
*
* 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.snapshot;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CancellationException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.SnapshotSentinel;
import org.apache.hadoop.hbase.master.handler.TableEventHandler;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Handler to Restore a snapshot.
*
* <p>Uses {@link RestoreSnapshotHelper} to replace the table content with the
* data available in the snapshot.
*/
@InterfaceAudience.Private
public class RestoreSnapshotHandler extends TableEventHandler implements SnapshotSentinel {
private static final Log LOG = LogFactory.getLog(RestoreSnapshotHandler.class);
private final HTableDescriptor hTableDescriptor;
private final SnapshotDescription snapshot;
private final ForeignExceptionDispatcher monitor;
private volatile boolean stopped = false;
public RestoreSnapshotHandler(final MasterServices masterServices,
final SnapshotDescription snapshot, final HTableDescriptor htd)
throws IOException {
super(EventType.C_M_RESTORE_SNAPSHOT, htd.getName(), masterServices, masterServices);
// Snapshot information
this.snapshot = snapshot;
// Monitor
this.monitor = new ForeignExceptionDispatcher();
// Check table exists.
getTableDescriptor();
// This is the new schema we are going to write out as this modification.
this.hTableDescriptor = htd;
}
/**
* The restore table is executed in place.
* - The on-disk data will be restored - reference files are put in place without moving data
* - [if something fail here: you need to delete the table and re-run the restore]
* - META will be updated
* - [if something fail here: you need to run hbck to fix META entries]
* The passed in list gets changed in this method
*/
@Override
protected void handleTableOperation(List<HRegionInfo> hris) throws IOException {
MasterFileSystem fileSystemManager = masterServices.getMasterFileSystem();
CatalogTracker catalogTracker = masterServices.getCatalogTracker();
FileSystem fs = fileSystemManager.getFileSystem();
Path rootDir = fileSystemManager.getRootDir();
byte[] tableName = hTableDescriptor.getName();
Path tableDir = HTableDescriptor.getTableDir(rootDir, tableName);
try {
// 1. Update descriptor
this.masterServices.getTableDescriptors().add(hTableDescriptor);
// 2. Execute the on-disk Restore
LOG.debug("Starting restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot));
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(
masterServices.getConfiguration(), fs,
snapshot, snapshotDir, hTableDescriptor, tableDir, monitor);
RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions();
// 3. Applies changes to .META.
hris.clear();
if (metaChanges.hasRegionsToAdd()) hris.addAll(metaChanges.getRegionsToAdd());
if (metaChanges.hasRegionsToRestore()) hris.addAll(metaChanges.getRegionsToRestore());
List<HRegionInfo> hrisToRemove = metaChanges.getRegionsToRemove();
MetaEditor.mutateRegions(catalogTracker, hrisToRemove, hris);
// At this point the restore is complete. Next step is enabling the table.
LOG.info("Restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot) + " on table=" +
Bytes.toString(tableName) + " completed!");
} catch (IOException e) {
String msg = "restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot)
+ " failed. Try re-running the restore command.";
LOG.error(msg, e);
monitor.receive(new ForeignException(masterServices.getServerName().toString(), e));
throw new RestoreSnapshotException(msg, e);
} finally {
this.stopped = true;
}
}
@Override
public boolean isFinished() {
return this.stopped;
}
@Override
public SnapshotDescription getSnapshot() {
return snapshot;
}
@Override
public void cancel(String why) {
if (this.stopped) return;
this.stopped = true;
String msg = "Stopping restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot)
+ " because: " + why;
LOG.info(msg);
CancellationException ce = new CancellationException(why);
this.monitor.receive(new ForeignException(masterServices.getServerName().toString(), ce));
}
public ForeignException getExceptionIfFailed() {
return this.monitor.getException();
}
}

View File

@ -0,0 +1,308 @@
/**
* 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.snapshot;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.Timer;
import java.util.TimerTask;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* Intelligently keep track of all the files for all the snapshots.
* <p>
* A cache of files is kept to avoid querying the {@link FileSystem} frequently. If there is a cache
* miss the directory modification time is used to ensure that we don't rescan directories that we
* already have in cache. We only check the modification times of the snapshot directories
* (/hbase/.snapshot/[snapshot_name]) to determine if the files need to be loaded into the cache.
* <p>
* New snapshots will be added to the cache and deleted snapshots will be removed when we refresh
* the cache. If the files underneath a snapshot directory are changed, but not the snapshot itself,
* we will ignore updates to that snapshot's files.
* <p>
* This is sufficient because each snapshot has its own directory and is added via an atomic rename
* <i>once</i>, when the snapshot is created. We don't need to worry about the data in the snapshot
* being run.
* <p>
* Further, the cache is periodically refreshed ensure that files in snapshots that were deleted are
* also removed from the cache.
* <p>
* A {@link SnapshotFileCache.SnapshotFileInspector} must be passed when creating <tt>this</tt> to
* allow extraction of files under /hbase/.snapshot/[snapshot name] directory, for each snapshot.
* This allows you to only cache files under, for instance, all the logs in the .logs directory or
* all the files under all the regions.
* <p>
* <tt>this</tt> also considers all running snapshots (those under /hbase/.snapshot/.tmp) as valid
* snapshots and will attempt to cache files from those snapshots as well.
* <p>
* Queries about a given file are thread-safe with respect to multiple queries and cache refreshes.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class SnapshotFileCache implements Stoppable {
interface SnapshotFileInspector {
/**
* Returns a collection of file names needed by the snapshot.
* @param snapshotDir {@link Path} to the snapshot directory to scan.
* @return the collection of file names needed by the snapshot.
*/
Collection<String> filesUnderSnapshot(final Path snapshotDir) throws IOException;
}
private static final Log LOG = LogFactory.getLog(SnapshotFileCache.class);
private volatile boolean stop = false;
private final FileSystem fs;
private final SnapshotFileInspector fileInspector;
private final Path snapshotDir;
private final Set<String> cache = new HashSet<String>();
/**
* This is a helper map of information about the snapshot directories so we don't need to rescan
* them if they haven't changed since the last time we looked.
*/
private final Map<String, SnapshotDirectoryInfo> snapshots =
new HashMap<String, SnapshotDirectoryInfo>();
private final Timer refreshTimer;
private long lastModifiedTime = Long.MIN_VALUE;
/**
* Create a snapshot file cache for all snapshots under the specified [root]/.snapshot on the
* filesystem.
* <p>
* Immediately loads the file cache.
* @param conf to extract the configured {@link FileSystem} where the snapshots are stored and
* hbase root directory
* @param cacheRefreshPeriod frequency (ms) with which the cache should be refreshed
* @param refreshThreadName name of the cache refresh thread
* @param inspectSnapshotFiles Filter to apply to each snapshot to extract the files.
* @throws IOException if the {@link FileSystem} or root directory cannot be loaded
*/
public SnapshotFileCache(Configuration conf, long cacheRefreshPeriod, String refreshThreadName,
SnapshotFileInspector inspectSnapshotFiles) throws IOException {
this(FSUtils.getCurrentFileSystem(conf), FSUtils.getRootDir(conf), 0, cacheRefreshPeriod,
refreshThreadName, inspectSnapshotFiles);
}
/**
* Create a snapshot file cache for all snapshots under the specified [root]/.snapshot on the
* filesystem
* @param fs {@link FileSystem} where the snapshots are stored
* @param rootDir hbase root directory
* @param cacheRefreshPeriod period (ms) with which the cache should be refreshed
* @param cacheRefreshDelay amount of time to wait for the cache to be refreshed
* @param refreshThreadName name of the cache refresh thread
* @param inspectSnapshotFiles Filter to apply to each snapshot to extract the files.
*/
public SnapshotFileCache(FileSystem fs, Path rootDir, long cacheRefreshPeriod,
long cacheRefreshDelay, String refreshThreadName, SnapshotFileInspector inspectSnapshotFiles) {
this.fs = fs;
this.fileInspector = inspectSnapshotFiles;
this.snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
// periodically refresh the file cache to make sure we aren't superfluously saving files.
this.refreshTimer = new Timer(refreshThreadName, true);
this.refreshTimer.scheduleAtFixedRate(new RefreshCacheTask(), cacheRefreshDelay,
cacheRefreshPeriod);
}
/**
* Trigger a cache refresh, even if its before the next cache refresh. Does not affect pending
* cache refreshes.
* <p>
* Blocks until the cache is refreshed.
* <p>
* Exposed for TESTING.
*/
public void triggerCacheRefreshForTesting() {
try {
SnapshotFileCache.this.refreshCache();
} catch (IOException e) {
LOG.warn("Failed to refresh snapshot hfile cache!", e);
}
LOG.debug("Current cache:" + cache);
}
/**
* Check to see if the passed file name is contained in any of the snapshots. First checks an
* in-memory cache of the files to keep. If its not in the cache, then the cache is refreshed and
* the cache checked again for that file. This ensures that we always return <tt>true</tt> for a
* files that exists.
* <p>
* Note this may lead to periodic false positives for the file being referenced. Periodically, the
* cache is refreshed even if there are no requests to ensure that the false negatives get removed
* eventually. For instance, suppose you have a file in the snapshot and it gets loaded into the
* cache. Then at some point later that snapshot is deleted. If the cache has not been refreshed
* at that point, cache will still think the file system contains that file and return
* <tt>true</tt>, even if it is no longer present (false positive). However, if the file never was
* on the filesystem, we will never find it and always return <tt>false</tt>.
* @param fileName file to check
* @return <tt>false</tt> if the file is not referenced in any current or running snapshot,
* <tt>true</tt> if the file is in the cache.
* @throws IOException if there is an unexpected error reaching the filesystem.
*/
// XXX this is inefficient to synchronize on the method, when what we really need to guard against
// is an illegal access to the cache. Really we could do a mutex-guarded pointer swap on the
// cache, but that seems overkill at the moment and isn't necessarily a bottleneck.
public synchronized boolean contains(String fileName) throws IOException {
if (this.cache.contains(fileName)) return true;
refreshCache();
// then check again
return this.cache.contains(fileName);
}
private synchronized void refreshCache() throws IOException {
// get the status of the snapshots directory
FileStatus status;
try {
status = fs.getFileStatus(snapshotDir);
} catch (FileNotFoundException e) {
LOG.error("Snapshot directory: " + snapshotDir + " doesn't exist");
return;
}
// if the snapshot directory wasn't modified since we last check, we are done
if (status.getModificationTime() <= lastModifiedTime) return;
// directory was modified, so we need to reload our cache
// there could be a slight race here where we miss the cache, check the directory modification
// time, then someone updates the directory, causing us to not scan the directory again.
// However, snapshot directories are only created once, so this isn't an issue.
// 1. update the modified time
this.lastModifiedTime = status.getModificationTime();
// 2.clear the cache
this.cache.clear();
Map<String, SnapshotDirectoryInfo> known = new HashMap<String, SnapshotDirectoryInfo>();
// 3. check each of the snapshot directories
FileStatus[] snapshots = FSUtils.listStatus(fs, snapshotDir);
if (snapshots == null) {
// remove all the remembered snapshots because we don't have any left
LOG.debug("No snapshots on-disk, cache empty");
this.snapshots.clear();
return;
}
// 3.1 iterate through the on-disk snapshots
for (FileStatus snapshot : snapshots) {
String name = snapshot.getPath().getName();
// its the tmp dir
if (name.equals(SnapshotDescriptionUtils.SNAPSHOT_TMP_DIR_NAME)) {
// only add those files to the cache, but not to the known snapshots
FileStatus[] running = FSUtils.listStatus(fs, snapshot.getPath());
if (running == null) continue;
for (FileStatus run : running) {
this.cache.addAll(fileInspector.filesUnderSnapshot(run.getPath()));
}
} else {
SnapshotDirectoryInfo files = this.snapshots.remove(name);
// 3.1.1 if we don't know about the snapshot or its been modified, we need to update the files
// the latter could occur where I create a snapshot, then delete it, and then make a new
// snapshot with the same name. We will need to update the cache the information from that new
// snapshot, even though it has the same name as the files referenced have probably changed.
if (files == null || files.hasBeenModified(snapshot.getModificationTime())) {
// get all files for the snapshot and create a new info
Collection<String> storedFiles = fileInspector.filesUnderSnapshot(snapshot.getPath());
files = new SnapshotDirectoryInfo(snapshot.getModificationTime(), storedFiles);
}
// 3.2 add all the files to cache
this.cache.addAll(files.getFiles());
known.put(name, files);
}
}
// 4. set the snapshots we are tracking
this.snapshots.clear();
this.snapshots.putAll(known);
}
/**
* Simple helper task that just periodically attempts to refresh the cache
*/
public class RefreshCacheTask extends TimerTask {
@Override
public void run() {
try {
SnapshotFileCache.this.refreshCache();
} catch (IOException e) {
LOG.warn("Failed to refresh snapshot hfile cache!", e);
}
}
}
@Override
public void stop(String why) {
if (!this.stop) {
this.stop = true;
this.refreshTimer.cancel();
}
}
@Override
public boolean isStopped() {
return this.stop;
}
/**
* Information about a snapshot directory
*/
private static class SnapshotDirectoryInfo {
long lastModified;
Collection<String> files;
public SnapshotDirectoryInfo(long mtime, Collection<String> files) {
this.lastModified = mtime;
this.files = files;
}
/**
* @return the hfiles in the snapshot when <tt>this</tt> was made.
*/
public Collection<String> getFiles() {
return this.files;
}
/**
* Check if the snapshot directory has been modified
* @param mtime current modification time of the directory
* @return <tt>true</tt> if it the modification time of the directory is newer time when we
* created <tt>this</tt>
*/
public boolean hasBeenModified(long mtime) {
return this.lastModified < mtime;
}
}
}

View File

@ -0,0 +1,104 @@
/**
* 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.snapshot;
import java.io.IOException;
import java.util.Collection;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* Implementation of a file cleaner that checks if a hfile is still used by snapshots of HBase
* tables.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class SnapshotHFileCleaner extends BaseHFileCleanerDelegate {
private static final Log LOG = LogFactory.getLog(SnapshotHFileCleaner.class);
/**
* Conf key for the frequency to attempt to refresh the cache of hfiles currently used in
* snapshots (ms)
*/
public static final String HFILE_CACHE_REFRESH_PERIOD_CONF_KEY =
"hbase.master.hfilecleaner.plugins.snapshot.period";
/** Refresh cache, by default, every 5 minutes */
private static final long DEFAULT_HFILE_CACHE_REFRESH_PERIOD = 300000;
/** File cache for HFiles in the completed and currently running snapshots */
private SnapshotFileCache cache;
@Override
public synchronized boolean isFileDeletable(Path filePath) {
try {
return !cache.contains(filePath.getName());
} catch (IOException e) {
LOG.error("Exception while checking if:" + filePath + " was valid, keeping it just in case.",
e);
return false;
}
}
@Override
public void setConf(Configuration conf) {
super.setConf(conf);
try {
long cacheRefreshPeriod = conf.getLong(HFILE_CACHE_REFRESH_PERIOD_CONF_KEY,
DEFAULT_HFILE_CACHE_REFRESH_PERIOD);
final FileSystem fs = FSUtils.getCurrentFileSystem(conf);
Path rootDir = FSUtils.getRootDir(conf);
cache = new SnapshotFileCache(fs, rootDir, cacheRefreshPeriod, cacheRefreshPeriod,
"snapshot-hfile-cleaner-cache-refresher", new SnapshotFileCache.SnapshotFileInspector() {
public Collection<String> filesUnderSnapshot(final Path snapshotDir)
throws IOException {
return SnapshotReferenceUtil.getHFileNames(fs, snapshotDir);
}
});
} catch (IOException e) {
LOG.error("Failed to create cleaner util", e);
}
}
@Override
public void stop(String why) {
this.cache.stop(why);
}
@Override
public boolean isStopped() {
return this.cache.isStopped();
}
/**
* Exposed for Testing!
* @return the cache of all hfiles
*/
public SnapshotFileCache getFileCacheForTesting() {
return this.cache;
}
}

View File

@ -0,0 +1,102 @@
/**
* 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.snapshot;
import java.io.IOException;
import java.util.Collection;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* Implementation of a log cleaner that checks if a log is still used by
* snapshots of HBase tables.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class SnapshotLogCleaner extends BaseLogCleanerDelegate {
private static final Log LOG = LogFactory.getLog(SnapshotLogCleaner.class);
/**
* Conf key for the frequency to attempt to refresh the cache of hfiles currently used in
* snapshots (ms)
*/
static final String HLOG_CACHE_REFRESH_PERIOD_CONF_KEY =
"hbase.master.hlogcleaner.plugins.snapshot.period";
/** Refresh cache, by default, every 5 minutes */
private static final long DEFAULT_HLOG_CACHE_REFRESH_PERIOD = 300000;
private SnapshotFileCache cache;
@Override
public synchronized boolean isFileDeletable(Path filePath) {
try {
if (null == cache) return false;
return !cache.contains(filePath.getName());
} catch (IOException e) {
LOG.error("Exception while checking if:" + filePath + " was valid, keeping it just in case.",
e);
return false;
}
}
/**
* This method should only be called <b>once</b>, as it starts a thread to keep the cache
* up-to-date.
* <p>
* {@inheritDoc}
*/
@Override
public void setConf(Configuration conf) {
super.setConf(conf);
try {
long cacheRefreshPeriod = conf.getLong(
HLOG_CACHE_REFRESH_PERIOD_CONF_KEY, DEFAULT_HLOG_CACHE_REFRESH_PERIOD);
final FileSystem fs = FSUtils.getCurrentFileSystem(conf);
Path rootDir = FSUtils.getRootDir(conf);
cache = new SnapshotFileCache(fs, rootDir, cacheRefreshPeriod, cacheRefreshPeriod,
"snapshot-log-cleaner-cache-refresher", new SnapshotFileCache.SnapshotFileInspector() {
public Collection<String> filesUnderSnapshot(final Path snapshotDir)
throws IOException {
return SnapshotReferenceUtil.getHLogNames(fs, snapshotDir);
}
});
} catch (IOException e) {
LOG.error("Failed to create snapshot log cleaner", e);
}
}
@Override
public void stop(String why) {
this.cache.stop(why);
}
@Override
public boolean isStopped() {
return this.cache.isStopped();
}
}

View File

@ -0,0 +1,916 @@
/**
* 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.snapshot;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ThreadPoolExecutor;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.SnapshotSentinel;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner;
import org.apache.hadoop.hbase.procedure.Procedure;
import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinatorRpcs;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type;
import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException;
import org.apache.hadoop.hbase.snapshot.SnapshotExistsException;
import org.apache.hadoop.hbase.snapshot.TablePartiallyOpenException;
import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.zookeeper.KeeperException;
/**
* This class manages the procedure of taking and restoring snapshots. There is only one
* SnapshotManager for the master.
* <p>
* The class provides methods for monitoring in-progress snapshot actions.
* <p>
* Note: Currently there can only be one snapshot being taken at a time over the cluster. This is a
* simplification in the current implementation.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class SnapshotManager implements Stoppable {
private static final Log LOG = LogFactory.getLog(SnapshotManager.class);
/** By default, check to see if the snapshot is complete every WAKE MILLIS (ms) */
private static final int SNAPSHOT_WAKE_MILLIS_DEFAULT = 500;
/** Enable or disable snapshot support */
public static final String HBASE_SNAPSHOT_ENABLED = "hbase.snapshot.enabled";
/**
* Conf key for # of ms elapsed between checks for snapshot errors while waiting for
* completion.
*/
private static final String SNAPSHOT_WAKE_MILLIS_KEY = "hbase.snapshot.master.wakeMillis";
/** By default, check to see if the snapshot is complete (ms) */
private static final int SNAPSHOT_TIMEOUT_MILLIS_DEFAULT = 5000;
/**
* Conf key for # of ms elapsed before injecting a snapshot timeout error when waiting for
* completion.
*/
private static final String SNAPSHOT_TIMEOUT_MILLIS_KEY = "hbase.snapshot.master.timeoutMillis";
/** Name of the operation to use in the controller */
public static final String ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION = "online-snapshot";
// TODO - enable having multiple snapshots with multiple monitors/threads
// this needs to be configuration based when running multiple snapshots is implemented
/** number of current operations running on the master */
private static final int opThreads = 1;
private boolean stopped;
private final long wakeFrequency;
private final MasterServices master; // Needed by TableEventHandlers
private final ProcedureCoordinator coordinator;
// Is snapshot feature enabled?
private boolean isSnapshotSupported = false;
// A reference to a handler. If the handler is non-null, then it is assumed that a snapshot is
// in progress currently
// TODO: this is a bad smell; likely replace with a collection in the future. Also this gets
// reset by every operation.
private TakeSnapshotHandler handler;
private final Path rootDir;
private final ExecutorService executorService;
// Restore Sentinels map, with table name as key
private Map<String, SnapshotSentinel> restoreHandlers = new HashMap<String, SnapshotSentinel>();
/**
* Construct a snapshot manager.
* @param master
*/
public SnapshotManager(final MasterServices master) throws KeeperException, IOException,
UnsupportedOperationException {
this.master = master;
checkSnapshotSupport(master.getConfiguration(), master.getMasterFileSystem());
// get the configuration for the coordinator
Configuration conf = master.getConfiguration();
this.wakeFrequency = conf.getInt(SNAPSHOT_WAKE_MILLIS_KEY, SNAPSHOT_WAKE_MILLIS_DEFAULT);
long keepAliveTime = conf.getLong(SNAPSHOT_TIMEOUT_MILLIS_KEY, SNAPSHOT_TIMEOUT_MILLIS_DEFAULT);
// setup the default procedure coordinator
String name = master.getServerName().toString();
ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, keepAliveTime, opThreads, wakeFrequency);
ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinatorRpcs(
master.getZooKeeper(), SnapshotManager.ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION, name);
this.coordinator = new ProcedureCoordinator(comms, tpool);
this.rootDir = master.getMasterFileSystem().getRootDir();
this.executorService = master.getExecutorService();
resetTempDir();
}
/**
* Fully specify all necessary components of a snapshot manager. Exposed for testing.
* @param master services for the master where the manager is running
* @param coordinator procedure coordinator instance. exposed for testing.
* @param pool HBase ExecutorServcie instance, exposed for testing.
*/
public SnapshotManager(final MasterServices master, ProcedureCoordinator coordinator, ExecutorService pool)
throws IOException, UnsupportedOperationException {
this.master = master;
checkSnapshotSupport(master.getConfiguration(), master.getMasterFileSystem());
this.wakeFrequency = master.getConfiguration().getInt(SNAPSHOT_WAKE_MILLIS_KEY,
SNAPSHOT_WAKE_MILLIS_DEFAULT);
this.coordinator = coordinator;
this.rootDir = master.getMasterFileSystem().getRootDir();
this.executorService = pool;
resetTempDir();
}
/**
* Gets the list of all completed snapshots.
* @return list of SnapshotDescriptions
* @throws IOException File system exception
*/
public List<SnapshotDescription> getCompletedSnapshots() throws IOException {
List<SnapshotDescription> snapshotDescs = new ArrayList<SnapshotDescription>();
// first create the snapshot root path and check to see if it exists
Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
FileSystem fs = master.getMasterFileSystem().getFileSystem();
// if there are no snapshots, return an empty list
if (!fs.exists(snapshotDir)) {
return snapshotDescs;
}
// ignore all the snapshots in progress
FileStatus[] snapshots = fs.listStatus(snapshotDir,
new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs));
// loop through all the completed snapshots
for (FileStatus snapshot : snapshots) {
Path info = new Path(snapshot.getPath(), SnapshotDescriptionUtils.SNAPSHOTINFO_FILE);
// if the snapshot is bad
if (!fs.exists(info)) {
LOG.error("Snapshot information for " + snapshot.getPath() + " doesn't exist");
continue;
}
FSDataInputStream in = null;
try {
in = fs.open(info);
SnapshotDescription desc = SnapshotDescription.parseFrom(in);
snapshotDescs.add(desc);
} catch (IOException e) {
LOG.warn("Found a corrupted snapshot " + snapshot.getPath(), e);
} finally {
if (in != null) {
in.close();
}
}
}
return snapshotDescs;
}
/**
* Cleans up any snapshots in the snapshot/.tmp directory that were left from failed
* snapshot attempts.
*
* @throws IOException if we can't reach the filesystem
*/
void resetTempDir() throws IOException {
// cleanup any existing snapshots.
Path tmpdir = SnapshotDescriptionUtils.getWorkingSnapshotDir(rootDir);
if (!master.getMasterFileSystem().getFileSystem().delete(tmpdir, true)) {
LOG.warn("Couldn't delete working snapshot directory: " + tmpdir);
}
}
/**
* Delete the specified snapshot
* @param snapshot
* @throws SnapshotDoesNotExistException If the specified snapshot does not exist.
* @throws IOException For filesystem IOExceptions
*/
public void deleteSnapshot(SnapshotDescription snapshot) throws SnapshotDoesNotExistException, IOException {
// call coproc pre hook
MasterCoprocessorHost cpHost = master.getCoprocessorHost();
if (cpHost != null) {
cpHost.preDeleteSnapshot(snapshot);
}
// check to see if it is completed
if (!isSnapshotCompleted(snapshot)) {
throw new SnapshotDoesNotExistException(snapshot);
}
String snapshotName = snapshot.getName();
LOG.debug("Deleting snapshot: " + snapshotName);
// first create the snapshot description and check to see if it exists
MasterFileSystem fs = master.getMasterFileSystem();
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
// delete the existing snapshot
if (!fs.getFileSystem().delete(snapshotDir, true)) {
throw new HBaseSnapshotException("Failed to delete snapshot directory: " + snapshotDir);
}
// call coproc post hook
if (cpHost != null) {
cpHost.postDeleteSnapshot(snapshot);
}
}
/**
* Return the handler if it is currently running and has the same snapshot target name.
* @param snapshot
* @return null if doesn't match, else a live handler.
*/
private synchronized TakeSnapshotHandler getTakeSnapshotHandler(SnapshotDescription snapshot) {
TakeSnapshotHandler h = this.handler;
if (h == null) {
return null;
}
if (!h.getSnapshot().getName().equals(snapshot.getName())) {
// specified snapshot is to the one currently running
return null;
}
return h;
}
/**
* Check if the specified snapshot is done
* @param expected
* @return true if snapshot is ready to be restored, false if it is still being taken.
* @throws IOException IOException if error from HDFS or RPC
* @throws UnknownSnapshotException if snapshot is invalid or does not exist.
*/
public boolean isSnapshotDone(SnapshotDescription expected) throws IOException {
// check the request to make sure it has a snapshot
if (expected == null) {
throw new UnknownSnapshotException(
"No snapshot name passed in request, can't figure out which snapshot you want to check.");
}
String ssString = SnapshotDescriptionUtils.toString(expected);
// check to see if the sentinel exists
TakeSnapshotHandler handler = getTakeSnapshotHandler(expected);
if (handler == null) {
// doesn't exist, check if it is already completely done.
if (!isSnapshotCompleted(expected)) {
throw new UnknownSnapshotException("Snapshot " + ssString
+ " is not currently running or one of the known completed snapshots.");
}
// was done, return true;
return true;
}
// pass on any failure we find in the sentinel
try {
handler.rethrowException();
} catch (ForeignException e) {
// Give some procedure info on an exception.
String status;
Procedure p = coordinator.getProcedure(expected.getName());
if (p != null) {
status = p.getStatus();
} else {
status = expected.getName() + " not found in proclist " + coordinator.getProcedureNames();
}
throw new HBaseSnapshotException("Snapshot " + ssString + " had an error. " + status, e,
expected);
}
// check to see if we are done
if (handler.isFinished()) {
LOG.debug("Snapshot '" + ssString + "' has completed, notifying client.");
return true;
} else if (LOG.isDebugEnabled()) {
LOG.debug("Snapshoting '" + ssString + "' is still in progress!");
}
return false;
}
/**
* Check to see if there are any snapshots in progress currently. Currently we have a
* limitation only allowing a single snapshot attempt at a time.
* @return <tt>true</tt> if there any snapshots in progress, <tt>false</tt> otherwise
* @throws SnapshotCreationException if the snapshot failed
*/
synchronized boolean isTakingSnapshot() throws SnapshotCreationException {
// TODO later when we handle multiple there would be a map with ssname to handler.
return handler != null && !handler.isFinished();
}
/**
* Check to see if the specified table has a snapshot in progress. Currently we have a
* limitation only allowing a single snapshot attempt at a time.
* @param tableName name of the table being snapshotted.
* @return <tt>true</tt> if there is a snapshot in progress on the specified table.
*/
private boolean isTakingSnapshot(final String tableName) {
if (handler != null && handler.getSnapshot().getTable().equals(tableName)) {
return !handler.isFinished();
}
return false;
}
/**
* Check to make sure that we are OK to run the passed snapshot. Checks to make sure that we
* aren't already running a snapshot.
* @param snapshot description of the snapshot we want to start
* @throws HBaseSnapshotException if the filesystem could not be prepared to start the snapshot
*/
private synchronized void prepareToTakeSnapshot(SnapshotDescription snapshot)
throws HBaseSnapshotException {
FileSystem fs = master.getMasterFileSystem().getFileSystem();
Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
// make sure we aren't already running a snapshot
if (isTakingSnapshot()) {
throw new SnapshotCreationException("Rejected taking "
+ SnapshotDescriptionUtils.toString(snapshot)
+ " because we are already running another snapshot "
+ SnapshotDescriptionUtils.toString(this.handler.getSnapshot()), snapshot);
}
// make sure we aren't running a restore on the same table
if (isRestoringTable(snapshot.getTable())) {
throw new SnapshotCreationException("Rejected taking "
+ SnapshotDescriptionUtils.toString(snapshot)
+ " because we are already have a restore in progress on the same snapshot "
+ SnapshotDescriptionUtils.toString(this.handler.getSnapshot()), snapshot);
}
try {
// delete the working directory, since we aren't running the snapshot. Likely leftovers
// from a failed attempt.
fs.delete(workingDir, true);
// recreate the working directory for the snapshot
if (!fs.mkdirs(workingDir)) {
throw new SnapshotCreationException("Couldn't create working directory (" + workingDir
+ ") for snapshot" , snapshot);
}
} catch (HBaseSnapshotException e) {
throw e;
} catch (IOException e) {
throw new SnapshotCreationException(
"Exception while checking to see if snapshot could be started.", e, snapshot);
}
}
/**
* Take a snapshot of an enabled table.
* <p>
* The thread limitation on the executorService's thread pool for snapshots ensures the
* snapshot won't be started if there is another snapshot already running. Does
* <b>not</b> check to see if another snapshot of the same name already exists.
* @param snapshot description of the snapshot to take.
* @throws HBaseSnapshotException if the snapshot could not be started
*/
private synchronized void snapshotEnabledTable(SnapshotDescription snapshot)
throws HBaseSnapshotException {
TakeSnapshotHandler handler;
try {
handler = new EnabledTableSnapshotHandler(snapshot, master, this);
this.executorService.submit(handler);
this.handler = handler;
} catch (IOException e) {
// cleanup the working directory by trying to delete it from the fs.
Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
try {
if (!this.master.getMasterFileSystem().getFileSystem().delete(workingDir, true)) {
LOG.warn("Couldn't delete working directory (" + workingDir + " for snapshot:"
+ SnapshotDescriptionUtils.toString(snapshot));
}
} catch (IOException e1) {
LOG.warn("Couldn't delete working directory (" + workingDir + " for snapshot:" +
SnapshotDescriptionUtils.toString(snapshot));
}
// fail the snapshot
throw new SnapshotCreationException("Could not build snapshot handler", e, snapshot);
}
}
/**
* Take a snapshot based on the enabled/disabled state of the table.
*
* @param snapshot
* @throws HBaseSnapshotException when a snapshot specific exception occurs.
* @throws IOException when some sort of generic IO exception occurs.
*/
public void takeSnapshot(SnapshotDescription snapshot) throws IOException {
// check to see if we already completed the snapshot
if (isSnapshotCompleted(snapshot)) {
throw new SnapshotExistsException("Snapshot '" + snapshot.getName()
+ "' already stored on the filesystem.", snapshot);
}
LOG.debug("No existing snapshot, attempting snapshot...");
// check to see if the table exists
HTableDescriptor desc = null;
try {
desc = master.getTableDescriptors().get(snapshot.getTable());
} catch (FileNotFoundException e) {
String msg = "Table:" + snapshot.getTable() + " info doesn't exist!";
LOG.error(msg);
throw new SnapshotCreationException(msg, e, snapshot);
} catch (IOException e) {
throw new SnapshotCreationException("Error while geting table description for table "
+ snapshot.getTable(), e, snapshot);
}
if (desc == null) {
throw new SnapshotCreationException("Table '" + snapshot.getTable()
+ "' doesn't exist, can't take snapshot.", snapshot);
}
// set the snapshot version, now that we are ready to take it
snapshot = snapshot.toBuilder().setVersion(SnapshotDescriptionUtils.SNAPSHOT_LAYOUT_VERSION)
.build();
// call pre coproc hook
MasterCoprocessorHost cpHost = master.getCoprocessorHost();
if (cpHost != null) {
cpHost.preSnapshot(snapshot, desc);
}
// setup the snapshot
prepareToTakeSnapshot(snapshot);
// if the table is enabled, then have the RS run actually the snapshot work
AssignmentManager assignmentMgr = master.getAssignmentManager();
if (assignmentMgr.getZKTable().isEnabledTable(snapshot.getTable())) {
LOG.debug("Table enabled, starting distributed snapshot.");
snapshotEnabledTable(snapshot);
LOG.debug("Started snapshot: " + SnapshotDescriptionUtils.toString(snapshot));
}
// For disabled table, snapshot is created by the master
else if (assignmentMgr.getZKTable().isDisabledTable(snapshot.getTable())) {
LOG.debug("Table is disabled, running snapshot entirely on master.");
snapshotDisabledTable(snapshot);
LOG.debug("Started snapshot: " + SnapshotDescriptionUtils.toString(snapshot));
} else {
LOG.error("Can't snapshot table '" + snapshot.getTable()
+ "', isn't open or closed, we don't know what to do!");
TablePartiallyOpenException tpoe = new TablePartiallyOpenException(snapshot.getTable()
+ " isn't fully open.");
throw new SnapshotCreationException("Table is not entirely open or closed", tpoe, snapshot);
}
// call post coproc hook
if (cpHost != null) {
cpHost.postSnapshot(snapshot, desc);
}
}
/**
* Take a snapshot of a disabled table.
* <p>
* The thread limitation on the executorService's thread pool for snapshots ensures the
* snapshot won't be started if there is another snapshot already running. Does
* <b>not</b> check to see if another snapshot of the same name already exists.
* @param snapshot description of the snapshot to take. Modified to be {@link Type#DISABLED}.
* @throws HBaseSnapshotException if the snapshot could not be started
*/
private synchronized void snapshotDisabledTable(SnapshotDescription snapshot)
throws HBaseSnapshotException {
// set the snapshot to be a disabled snapshot, since the client doesn't know about that
snapshot = snapshot.toBuilder().setType(Type.DISABLED).build();
DisabledTableSnapshotHandler handler;
try {
handler = new DisabledTableSnapshotHandler(snapshot, this.master);
this.executorService.submit(handler);
this.handler = handler;
} catch (IOException e) {
// cleanup the working directory by trying to delete it from the fs.
Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
try {
if (!this.master.getMasterFileSystem().getFileSystem().delete(workingDir, true)) {
LOG.error("Couldn't delete working directory (" + workingDir + " for snapshot:" +
SnapshotDescriptionUtils.toString(snapshot));
}
} catch (IOException e1) {
LOG.error("Couldn't delete working directory (" + workingDir + " for snapshot:" +
SnapshotDescriptionUtils.toString(snapshot));
}
// fail the snapshot
throw new SnapshotCreationException("Could not build snapshot handler", e, snapshot);
}
}
/**
* Set the handler for the current snapshot
* <p>
* Exposed for TESTING
* @param handler handler the master should use
*
* TODO get rid of this if possible, repackaging, modify tests.
*/
public synchronized void setSnapshotHandlerForTesting(TakeSnapshotHandler handler) {
this.handler = handler;
}
/**
* @return distributed commit coordinator for all running snapshots
*/
ProcedureCoordinator getCoordinator() {
return coordinator;
}
/**
* Check to see if the snapshot is one of the currently completed snapshots
* @param expected snapshot to check
* @return <tt>true</tt> if the snapshot is stored on the {@link FileSystem}, <tt>false</tt> if is
* not stored
* @throws IOException if the filesystem throws an unexpected exception,
* @throws IllegalArgumentException if snapshot name is invalid.
*/
private boolean isSnapshotCompleted(SnapshotDescription snapshot) throws IOException {
try {
final Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
FileSystem fs = master.getMasterFileSystem().getFileSystem();
// check to see if the snapshot already exists
return fs.exists(snapshotDir);
} catch (IllegalArgumentException iae) {
throw new UnknownSnapshotException("Unexpected exception thrown", iae);
}
}
/**
* Clone the specified snapshot into a new table.
* The operation will fail if the destination table has a snapshot or restore in progress.
*
* @param snapshot Snapshot Descriptor
* @param hTableDescriptor Table Descriptor of the table to create
* @param waitTime timeout before considering the clone failed
*/
synchronized void cloneSnapshot(final SnapshotDescription snapshot,
final HTableDescriptor hTableDescriptor) throws HBaseSnapshotException {
String tableName = hTableDescriptor.getNameAsString();
// make sure we aren't running a snapshot on the same table
if (isTakingSnapshot(tableName)) {
throw new RestoreSnapshotException("Snapshot in progress on the restore table=" + tableName);
}
// make sure we aren't running a restore on the same table
if (isRestoringTable(tableName)) {
throw new RestoreSnapshotException("Restore already in progress on the table=" + tableName);
}
try {
CloneSnapshotHandler handler =
new CloneSnapshotHandler(master, snapshot, hTableDescriptor);
this.executorService.submit(handler);
restoreHandlers.put(tableName, handler);
} catch (Exception e) {
String msg = "Couldn't clone the snapshot=" + SnapshotDescriptionUtils.toString(snapshot) +
" on table=" + tableName;
LOG.error(msg, e);
throw new RestoreSnapshotException(msg, e);
}
}
/**
* Restore the specified snapshot
* @param reqSnapshot
* @throws IOException
*/
public void restoreSnapshot(SnapshotDescription reqSnapshot) throws IOException {
FileSystem fs = master.getMasterFileSystem().getFileSystem();
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(reqSnapshot, rootDir);
MasterCoprocessorHost cpHost = master.getCoprocessorHost();
// check if the snapshot exists
if (!fs.exists(snapshotDir)) {
LOG.error("A Snapshot named '" + reqSnapshot.getName() + "' does not exist.");
throw new SnapshotDoesNotExistException(reqSnapshot);
}
// read snapshot information
SnapshotDescription fsSnapshot = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
HTableDescriptor snapshotTableDesc = FSTableDescriptors.getTableDescriptor(fs, snapshotDir);
String tableName = reqSnapshot.getTable();
// stop tracking completed restores
cleanupRestoreSentinels();
// Execute the restore/clone operation
if (MetaReader.tableExists(master.getCatalogTracker(), tableName)) {
if (master.getAssignmentManager().getZKTable().isEnabledTable(fsSnapshot.getTable())) {
throw new UnsupportedOperationException("Table '" +
fsSnapshot.getTable() + "' must be disabled in order to perform a restore operation.");
}
// call coproc pre hook
if (cpHost != null) {
cpHost.preRestoreSnapshot(reqSnapshot, snapshotTableDesc);
}
restoreSnapshot(fsSnapshot, snapshotTableDesc);
LOG.info("Restore snapshot=" + fsSnapshot.getName() + " as table=" + tableName);
if (cpHost != null) {
cpHost.postRestoreSnapshot(reqSnapshot, snapshotTableDesc);
}
} else {
HTableDescriptor htd = RestoreSnapshotHelper.cloneTableSchema(snapshotTableDesc,
Bytes.toBytes(tableName));
if (cpHost != null) {
cpHost.preCloneSnapshot(reqSnapshot, htd);
}
cloneSnapshot(fsSnapshot, htd);
LOG.info("Clone snapshot=" + fsSnapshot.getName() + " as table=" + tableName);
if (cpHost != null) {
cpHost.postCloneSnapshot(reqSnapshot, htd);
}
}
}
/**
* Restore the specified snapshot.
* The restore will fail if the destination table has a snapshot or restore in progress.
*
* @param snapshot Snapshot Descriptor
* @param hTableDescriptor Table Descriptor
* @param waitTime timeout before considering the restore failed
*/
private synchronized void restoreSnapshot(final SnapshotDescription snapshot,
final HTableDescriptor hTableDescriptor) throws HBaseSnapshotException {
String tableName = hTableDescriptor.getNameAsString();
// make sure we aren't running a snapshot on the same table
if (isTakingSnapshot(tableName)) {
throw new RestoreSnapshotException("Snapshot in progress on the restore table=" + tableName);
}
// make sure we aren't running a restore on the same table
if (isRestoringTable(tableName)) {
throw new RestoreSnapshotException("Restore already in progress on the table=" + tableName);
}
try {
RestoreSnapshotHandler handler =
new RestoreSnapshotHandler(master, snapshot, hTableDescriptor);
this.executorService.submit(handler);
restoreHandlers.put(hTableDescriptor.getNameAsString(), handler);
} catch (Exception e) {
String msg = "Couldn't restore the snapshot=" + SnapshotDescriptionUtils.toString(snapshot) +
" on table=" + tableName;
LOG.error(msg, e);
throw new RestoreSnapshotException(msg, e);
}
}
/**
* Verify if the restore of the specified table is in progress.
*
* @param tableName table under restore
* @return <tt>true</tt> if there is a restore in progress of the specified table.
*/
private boolean isRestoringTable(final String tableName) {
SnapshotSentinel sentinel = restoreHandlers.get(tableName);
return(sentinel != null && !sentinel.isFinished());
}
/**
* Returns status of a restore request, specifically comparing source snapshot and target table
* names. Throws exception if not a known snapshot.
* @param snapshot
* @return true if in progress, false if snapshot is completed.
* @throws UnknownSnapshotException if specified source snapshot does not exit.
* @throws IOException if there was some sort of IO failure
*/
public boolean isRestoringTable(final SnapshotDescription snapshot) throws IOException {
// check to see if the snapshot is already on the fs
if (!isSnapshotCompleted(snapshot)) {
throw new UnknownSnapshotException("Snapshot:" + snapshot.getName()
+ " is not one of the known completed snapshots.");
}
SnapshotSentinel sentinel = getRestoreSnapshotSentinel(snapshot.getTable());
if (sentinel == null) {
// there is no sentinel so restore is not in progress.
return false;
}
if (!sentinel.getSnapshot().getName().equals(snapshot.getName())) {
// another handler is trying to restore to the table, but it isn't the same snapshot source.
return false;
}
LOG.debug("Verify snapshot=" + snapshot.getName() + " against="
+ sentinel.getSnapshot().getName() + " table=" + snapshot.getTable());
ForeignException e = sentinel.getExceptionIfFailed();
if (e != null) throw e;
// check to see if we are done
if (sentinel.isFinished()) {
LOG.debug("Restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot) +
" has completed. Notifying the client.");
return false;
}
if (LOG.isDebugEnabled()) {
LOG.debug("Sentinel is not yet finished with restoring snapshot=" +
SnapshotDescriptionUtils.toString(snapshot));
}
return true;
}
/**
* Get the restore snapshot sentinel for the specified table
* @param tableName table under restore
* @return the restore snapshot handler
*/
private synchronized SnapshotSentinel getRestoreSnapshotSentinel(final String tableName) {
try {
return restoreHandlers.get(tableName);
} finally {
cleanupRestoreSentinels();
}
}
/**
* Scan the restore handlers and remove the finished ones.
*/
private synchronized void cleanupRestoreSentinels() {
Iterator<Map.Entry<String, SnapshotSentinel>> it = restoreHandlers.entrySet().iterator();
while (it.hasNext()) {
Map.Entry<String, SnapshotSentinel> entry = it.next();
SnapshotSentinel sentinel = entry.getValue();
if (sentinel.isFinished()) {
it.remove();
}
}
}
//
// Implementing Stoppable interface
//
@Override
public void stop(String why) {
// short circuit
if (this.stopped) return;
// make sure we get stop
this.stopped = true;
// pass the stop onto take snapshot handlers
if (this.handler != null) this.handler.cancel(why);
// pass the stop onto all the restore handlers
for (SnapshotSentinel restoreHandler: this.restoreHandlers.values()) {
restoreHandler.cancel(why);
}
}
@Override
public boolean isStopped() {
return this.stopped;
}
/**
* Throws an exception if snapshot operations (take a snapshot, restore, clone) are not supported.
* Called at the beginning of snapshot() and restoreSnapshot() methods.
* @throws UnsupportedOperationException if snapshot are not supported
*/
public void checkSnapshotSupport() throws UnsupportedOperationException {
if (!this.isSnapshotSupported) {
throw new UnsupportedOperationException(
"To use snapshots, You must add to the hbase-site.xml of the HBase Master: '" +
HBASE_SNAPSHOT_ENABLED + "' property with value 'true'.");
}
}
/**
* Called at startup, to verify if snapshot operation is supported, and to avoid
* starting the master if there're snapshots present but the cleaners needed are missing.
* Otherwise we can end up with snapshot data loss.
* @param conf The {@link Configuration} object to use
* @param mfs The MasterFileSystem to use
* @throws IOException in case of file-system operation failure
* @throws UnsupportedOperationException in case cleaners are missing and
* there're snapshot in the system
*/
private void checkSnapshotSupport(final Configuration conf, final MasterFileSystem mfs)
throws IOException, UnsupportedOperationException {
// Verify if snapshot is disabled by the user
String enabled = conf.get(HBASE_SNAPSHOT_ENABLED);
boolean snapshotEnabled = conf.getBoolean(HBASE_SNAPSHOT_ENABLED, false);
boolean userDisabled = (enabled != null && enabled.trim().length() > 0 && !snapshotEnabled);
// Extract cleaners from conf
Set<String> hfileCleaners = new HashSet<String>();
String[] cleaners = conf.getStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS);
if (cleaners != null) Collections.addAll(hfileCleaners, cleaners);
Set<String> logCleaners = new HashSet<String>();
cleaners = conf.getStrings(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS);
if (cleaners != null) Collections.addAll(logCleaners, cleaners);
// If the user has enabled the snapshot, we force the cleaners to be present
// otherwise we still need to check if cleaners are enabled or not and verify
// that there're no snapshot in the .snapshot folder.
if (snapshotEnabled) {
// Inject snapshot cleaners, if snapshot.enable is true
hfileCleaners.add(SnapshotHFileCleaner.class.getName());
hfileCleaners.add(HFileLinkCleaner.class.getName());
logCleaners.add(SnapshotLogCleaner.class.getName());
// Set cleaners conf
conf.setStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS,
hfileCleaners.toArray(new String[hfileCleaners.size()]));
conf.setStrings(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS,
logCleaners.toArray(new String[logCleaners.size()]));
} else {
// Verify if cleaners are present
snapshotEnabled = logCleaners.contains(SnapshotLogCleaner.class.getName()) &&
hfileCleaners.contains(SnapshotHFileCleaner.class.getName()) &&
hfileCleaners.contains(HFileLinkCleaner.class.getName());
// Warn if the cleaners are enabled but the snapshot.enabled property is false/not set.
if (snapshotEnabled) {
LOG.warn("Snapshot log and hfile cleaners are present in the configuration, " +
"but the '" + HBASE_SNAPSHOT_ENABLED + "' property " +
(userDisabled ? "is set to 'false'." : "is not set."));
}
}
// Mark snapshot feature as enabled if cleaners are present and user has not disabled it.
this.isSnapshotSupported = snapshotEnabled && !userDisabled;
// If cleaners are not enabled, verify that there're no snapshot in the .snapshot folder
// otherwise we end up with snapshot data loss.
if (!snapshotEnabled) {
LOG.info("Snapshot feature is not enabled, missing log and hfile cleaners.");
Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(mfs.getRootDir());
FileSystem fs = mfs.getFileSystem();
if (fs.exists(snapshotDir)) {
FileStatus[] snapshots = FSUtils.listStatus(fs, snapshotDir,
new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs));
if (snapshots != null) {
LOG.error("Snapshots are present, but cleaners are not enabled.");
checkSnapshotSupport();
}
}
}
}
}

View File

@ -0,0 +1,237 @@
/**
* 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.snapshot;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CancellationException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.SnapshotSentinel;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.TableInfoCopyTask;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.zookeeper.KeeperException;
/**
* A handler for taking snapshots from the master.
*
* This is not a subclass of TableEventHandler because using that would incur an extra META scan.
*
* The {@link #snapshotRegions(List)} call should get implemented for each snapshot flavor.
*/
@InterfaceAudience.Private
public abstract class TakeSnapshotHandler extends EventHandler implements SnapshotSentinel,
ForeignExceptionSnare {
private static final Log LOG = LogFactory.getLog(TakeSnapshotHandler.class);
private volatile boolean finished;
// none of these should ever be null
protected final MasterServices master;
protected final SnapshotDescription snapshot;
protected final Configuration conf;
protected final FileSystem fs;
protected final Path rootDir;
private final Path snapshotDir;
protected final Path workingDir;
private final MasterSnapshotVerifier verifier;
protected final ForeignExceptionDispatcher monitor;
/**
* @param snapshot descriptor of the snapshot to take
* @param masterServices master services provider
* @throws IOException on unexpected error
*/
public TakeSnapshotHandler(SnapshotDescription snapshot,
final MasterServices masterServices) throws IOException {
super(masterServices, EventType.C_M_SNAPSHOT_TABLE);
assert snapshot != null : "SnapshotDescription must not be nul1";
assert masterServices != null : "MasterServices must not be nul1";
this.master = masterServices;
this.snapshot = snapshot;
this.conf = this.master.getConfiguration();
this.fs = this.master.getMasterFileSystem().getFileSystem();
this.rootDir = this.master.getMasterFileSystem().getRootDir();
this.snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
this.workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
this.monitor = new ForeignExceptionDispatcher();
loadTableDescriptor(); // check that .tableinfo is present
// prepare the verify
this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, rootDir);
}
private HTableDescriptor loadTableDescriptor()
throws FileNotFoundException, IOException {
final String name = snapshot.getTable();
HTableDescriptor htd =
this.master.getTableDescriptors().get(name);
if (htd == null) {
throw new IOException("HTableDescriptor missing for " + name);
}
return htd;
}
/**
* Execute the core common portions of taking a snapshot. The {@link #snapshotRegions(List)}
* call should get implemented for each snapshot flavor.
*/
@Override
public void process() {
LOG.info("Running table snapshot operation " + eventType + " on table " + snapshot.getTable());
try {
// If regions move after this meta scan, the region specific snapshot should fail, triggering
// an external exception that gets captured here.
// write down the snapshot info in the working directory
SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, workingDir, this.fs);
new TableInfoCopyTask(monitor, snapshot, fs, rootDir).call();
monitor.rethrowException();
List<Pair<HRegionInfo, ServerName>> regionsAndLocations =
MetaReader.getTableRegionsAndLocations(this.server.getCatalogTracker(),
Bytes.toBytes(snapshot.getTable()), true);
// run the snapshot
snapshotRegions(regionsAndLocations);
// extract each pair to separate lists
Set<String> serverNames = new HashSet<String>();
for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
serverNames.add(p.getSecond().toString());
}
// verify the snapshot is valid
verifier.verifySnapshot(this.workingDir, serverNames);
// complete the snapshot, atomically moving from tmp to .snapshot dir.
completeSnapshot(this.snapshotDir, this.workingDir, this.fs);
} catch (Exception e) {
String reason = "Failed taking snapshot " + SnapshotDescriptionUtils.toString(snapshot)
+ " due to exception:" + e.getMessage();
LOG.error(reason, e);
ForeignException ee = new ForeignException(reason, e);
monitor.receive(ee);
// need to mark this completed to close off and allow cleanup to happen.
cancel("Failed to take snapshot '" + SnapshotDescriptionUtils.toString(snapshot)
+ "' due to exception");
} finally {
LOG.debug("Launching cleanup of working dir:" + workingDir);
try {
// if the working dir is still present, the snapshot has failed. it is present we delete
// it.
if (fs.exists(workingDir) && !this.fs.delete(workingDir, true)) {
LOG.error("Couldn't delete snapshot working directory:" + workingDir);
}
} catch (IOException e) {
LOG.error("Couldn't delete snapshot working directory:" + workingDir);
}
}
}
/**
* Reset the manager to allow another snapshot to proceed
*
* @param snapshotDir final path of the snapshot
* @param workingDir directory where the in progress snapshot was built
* @param fs {@link FileSystem} where the snapshot was built
* @throws SnapshotCreationException if the snapshot could not be moved
* @throws IOException the filesystem could not be reached
*/
public void completeSnapshot(Path snapshotDir, Path workingDir, FileSystem fs)
throws SnapshotCreationException, IOException {
LOG.debug("Sentinel is done, just moving the snapshot from " + workingDir + " to "
+ snapshotDir);
if (!fs.rename(workingDir, snapshotDir)) {
throw new SnapshotCreationException("Failed to move working directory(" + workingDir
+ ") to completed directory(" + snapshotDir + ").");
}
finished = true;
}
/**
* Snapshot the specified regions
*/
protected abstract void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regions)
throws IOException, KeeperException;
@Override
public void cancel(String why) {
if (finished) return;
this.finished = true;
LOG.info("Stop taking snapshot=" + SnapshotDescriptionUtils.toString(snapshot) + " because: "
+ why);
CancellationException ce = new CancellationException(why);
monitor.receive(new ForeignException(master.getServerName().toString(), ce));
}
@Override
public boolean isFinished() {
return finished;
}
@Override
public SnapshotDescription getSnapshot() {
return snapshot;
}
@Override
public ForeignException getExceptionIfFailed() {
return monitor.getException();
}
@Override
public void rethrowException() throws ForeignException {
monitor.rethrowException();
}
@Override
public boolean hasException() {
return monitor.hasException();
}
@Override
public ForeignException getException() {
return monitor.getException();
}
}

View File

@ -0,0 +1,379 @@
/**
* 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.procedure;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionListener;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector;
import com.google.common.collect.Lists;
/**
* A globally-barriered distributed procedure. This class encapsulates state and methods for
* tracking and managing a distributed procedure, as well as aborting if any member encounters
* a problem or if a cancellation is requested.
* <p>
* All procedures first attempt to reach a barrier point with the {@link #sendGlobalBarrierStart()}
* method. The procedure contacts all members and waits for all subprocedures to execute
* {@link Subprocedure#acquireBarrier} to acquire its local piece of the global barrier and then
* send acquisition info back to the coordinator. If all acquisitions at subprocedures succeed,
* the coordinator then will call {@link #sendGlobalBarrierReached()}. This notifies members to
* execute the {@link Subprocedure#insideBarrier()} method. The procedure is blocked until all
* {@link Subprocedure#insideBarrier} executions complete at the members. When
* {@link Subprocedure#insideBarrier} completes at each member, the member sends notification to
* the coordinator. Once all members complete, the coordinator calls
* {@link #sendGlobalBarrierComplete()}.
* <p>
* If errors are encountered remotely, they are forwarded to the coordinator, and
* {@link Subprocedure#cleanup(Exception)} is called.
* <p>
* Each Procedure and each Subprocedure enforces a time limit on the execution time. If the time
* limit expires before the procedure completes the {@link TimeoutExceptionInjector} will trigger
* an {@link ForeignException} to abort the procedure. This is particularly useful for situations
* when running a distributed {@link Subprocedure} so participants can avoid blocking for extreme
* amounts of time if one of the participants fails or takes a really long time (e.g. GC pause).
* <p>
* Users should generally not directly create or subclass instances of this. They are created
* for them implicitly via {@link ProcedureCoordinator#startProcedure(ForeignExceptionDispatcher,
* String, byte[], List)}}
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class Procedure implements Callable<Void>, ForeignExceptionListener {
private static final Log LOG = LogFactory.getLog(Procedure.class);
//
// Arguments and naming
//
// Name of the procedure
final private String procName;
// Arguments for this procedure execution
final private byte[] args;
//
// Execution State
//
/** latch for waiting until all members have acquire in barrier state */
final CountDownLatch acquiredBarrierLatch;
/** latch for waiting until all members have executed and released their in barrier state */
final CountDownLatch releasedBarrierLatch;
/** latch for waiting until a procedure has completed */
final CountDownLatch completedLatch;
/** monitor to check for errors */
private final ForeignExceptionDispatcher monitor;
//
// Execution Timeout Handling.
//
/** frequency to check for errors (ms) */
protected final long wakeFrequency;
protected final TimeoutExceptionInjector timeoutInjector;
//
// Members' and Coordinator's state
//
/** lock to prevent nodes from acquiring and then releasing before we can track them */
private Object joinBarrierLock = new Object();
private final List<String> acquiringMembers;
private final List<String> inBarrierMembers;
private ProcedureCoordinator coord;
/**
* Creates a procedure. (FOR TESTING)
*
* {@link Procedure} state to be run by a {@link ProcedureCoordinator}.
* @param coord coordinator to call back to for general errors (e.g.
* {@link ProcedureCoordinator#rpcConnectionFailure(String, IOException)}).
* @param monitor error monitor to check for external errors
* @param wakeFreq frequency to check for errors while waiting
* @param timeout amount of time to allow the procedure to run before cancelling
* @param procName name of the procedure instance
* @param args argument data associated with the procedure instance
* @param expectedMembers names of the expected members
*/
public Procedure(ProcedureCoordinator coord, ForeignExceptionDispatcher monitor, long wakeFreq,
long timeout, String procName, byte[] args, List<String> expectedMembers) {
this.coord = coord;
this.acquiringMembers = new ArrayList<String>(expectedMembers);
this.inBarrierMembers = new ArrayList<String>(acquiringMembers.size());
this.procName = procName;
this.args = args;
this.monitor = monitor;
this.wakeFrequency = wakeFreq;
int count = expectedMembers.size();
this.acquiredBarrierLatch = new CountDownLatch(count);
this.releasedBarrierLatch = new CountDownLatch(count);
this.completedLatch = new CountDownLatch(1);
this.timeoutInjector = new TimeoutExceptionInjector(monitor, timeout);
}
/**
* Create a procedure.
*
* Users should generally not directly create instances of this. They are created them
* implicitly via {@link ProcedureCoordinator#createProcedure(ForeignExceptionDispatcher,
* String, byte[], List)}}
*
* @param coord coordinator to call back to for general errors (e.g.
* {@link ProcedureCoordinator#rpcConnectionFailure(String, IOException)}).
* @param wakeFreq frequency to check for errors while waiting
* @param timeout amount of time to allow the procedure to run before cancelling
* @param procName name of the procedure instance
* @param args argument data associated with the procedure instance
* @param expectedMembers names of the expected members
*/
public Procedure(ProcedureCoordinator coord, long wakeFreq, long timeout,
String procName, byte[] args, List<String> expectedMembers) {
this(coord, new ForeignExceptionDispatcher(), wakeFreq, timeout, procName, args,
expectedMembers);
}
public String getName() {
return procName;
}
/**
* @return String of the procedure members both trying to enter the barrier and already in barrier
*/
public String getStatus() {
String waiting, done;
synchronized (joinBarrierLock) {
waiting = acquiringMembers.toString();
done = inBarrierMembers.toString();
}
return "Procedure " + procName + " { waiting=" + waiting + " done="+ done + " }";
}
/**
* Get the ForeignExceptionDispatcher
* @return the Procedure's monitor.
*/
public ForeignExceptionDispatcher getErrorMonitor() {
return monitor;
}
/**
* This call is the main execution thread of the barriered procedure. It sends messages and
* essentially blocks until all procedure members acquire or later complete but periodically
* checks for foreign exceptions.
*/
@Override
@SuppressWarnings("finally")
final public Void call() {
LOG.info("Starting procedure '" + procName + "'");
// start the timer
timeoutInjector.start();
// run the procedure
try {
// start by checking for error first
monitor.rethrowException();
LOG.debug("Procedure '" + procName + "' starting 'acquire'");
sendGlobalBarrierStart();
// wait for all the members to report acquisition
LOG.debug("Waiting for all members to 'acquire'");
waitForLatch(acquiredBarrierLatch, monitor, wakeFrequency, "acquired");
monitor.rethrowException();
LOG.debug("Procedure '" + procName + "' starting 'in-barrier' execution.");
sendGlobalBarrierReached();
// wait for all members to report barrier release
waitForLatch(releasedBarrierLatch, monitor, wakeFrequency, "released");
// make sure we didn't get an error during in barrier execution and release
monitor.rethrowException();
LOG.info("Procedure '" + procName + "' execution completed");
} catch (Exception e) {
if (e instanceof InterruptedException) {
Thread.currentThread().interrupt();
}
String msg = "Procedure '" + procName +"' execution failed!";
LOG.error(msg, e);
receive(new ForeignException(getName(), e));
} finally {
LOG.debug("Running finish phase.");
sendGlobalBarrierComplete();
completedLatch.countDown();
// tell the timer we are done, if we get here successfully
timeoutInjector.complete();
return null;
}
}
/**
* Sends a message to Members to create a new {@link Subprocedure} for this Procedure and execute
* the {@link Subprocedure#acquireBarrier} step.
* @throws ForeignException
*/
public void sendGlobalBarrierStart() throws ForeignException {
// start the procedure
LOG.debug("Starting procedure '" + procName + "', kicking off acquire phase on members.");
try {
// send procedure barrier start to specified list of members. cloning the list to avoid
// concurrent modification from the controller setting the prepared nodes
coord.getRpcs().sendGlobalBarrierAcquire(this, args, Lists.newArrayList(this.acquiringMembers));
} catch (IOException e) {
coord.rpcConnectionFailure("Can't reach controller.", e);
} catch (IllegalArgumentException e) {
throw new ForeignException(getName(), e);
}
}
/**
* Sends a message to all members that the global barrier condition has been satisfied. This
* should only be executed after all members have completed its
* {@link Subprocedure#acquireBarrier()} call successfully. This triggers the member
* {@link Subprocedure#insideBarrier} method.
* @throws ForeignException
*/
public void sendGlobalBarrierReached() throws ForeignException {
try {
// trigger to have member run {@link Subprocedure#insideBarrier}
coord.getRpcs().sendGlobalBarrierReached(this, Lists.newArrayList(inBarrierMembers));
} catch (IOException e) {
coord.rpcConnectionFailure("Can't reach controller.", e);
}
}
/**
* Sends a message to members that all {@link Subprocedure#insideBarrier} calls have completed.
* After this executes, the coordinator can assume that any state resources about this barrier
* procedure state has been released.
*/
public void sendGlobalBarrierComplete() {
LOG.debug("Finished coordinator procedure - removing self from list of running procedures");
try {
coord.getRpcs().resetMembers(this);
} catch (IOException e) {
coord.rpcConnectionFailure("Failed to reset procedure:" + procName, e);
}
}
//
// Call backs from other external processes.
//
/**
* Call back triggered by an individual member upon successful local barrier acquisition
* @param member
*/
public void barrierAcquiredByMember(String member) {
LOG.debug("member: '" + member + "' joining prepared barrier for procedure '" + procName
+ "' on coordinator");
if (this.acquiringMembers.contains(member)) {
synchronized (joinBarrierLock) {
if (this.acquiringMembers.remove(member)) {
this.inBarrierMembers.add(member);
acquiredBarrierLatch.countDown();
}
}
LOG.debug("Waiting on: " + acquiredBarrierLatch + " remaining members to acquire global barrier");
} else {
LOG.warn("Member " + member + " joined barrier, but we weren't waiting on it to join." +
" Continuing on.");
}
}
/**
* Call back triggered by a individual member upon successful local in-barrier execution and
* release
* @param member
*/
public void barrierReleasedByMember(String member) {
boolean removed = false;
synchronized (joinBarrierLock) {
removed = this.inBarrierMembers.remove(member);
if (removed) {
releasedBarrierLatch.countDown();
}
}
if (removed) {
LOG.debug("Member: '" + member + "' released barrier for procedure'" + procName
+ "', counting down latch. Waiting for " + releasedBarrierLatch.getCount()
+ " more");
} else {
LOG.warn("Member: '" + member + "' released barrier for procedure'" + procName
+ "', but we weren't waiting on it to release!");
}
}
/**
* Waits until the entire procedure has globally completed, or has been aborted. If an
* exception is thrown the procedure may or not have run cleanup to trigger the completion latch
* yet.
* @throws ForeignException
* @throws InterruptedException
*/
public void waitForCompleted() throws ForeignException, InterruptedException {
waitForLatch(completedLatch, monitor, wakeFrequency, procName + " completed");
}
/**
* A callback that handles incoming ForeignExceptions.
*/
@Override
public void receive(ForeignException e) {
monitor.receive(e);
}
/**
* Wait for latch to count to zero, ignoring any spurious wake-ups, but waking periodically to
* check for errors
* @param latch latch to wait on
* @param monitor monitor to check for errors while waiting
* @param wakeFrequency frequency to wake up and check for errors (in
* {@link TimeUnit#MILLISECONDS})
* @param latchDescription description of the latch, for logging
* @throws ForeignException type of error the monitor can throw, if the task fails
* @throws InterruptedException if we are interrupted while waiting on latch
*/
public static void waitForLatch(CountDownLatch latch, ForeignExceptionSnare monitor,
long wakeFrequency, String latchDescription) throws ForeignException,
InterruptedException {
boolean released = false;
while (!released) {
if (monitor != null) {
monitor.rethrowException();
}
/*
ForeignExceptionDispatcher.LOG.debug("Waiting for '" + latchDescription + "' latch. (sleep:"
+ wakeFrequency + " ms)"); */
released = latch.await(wakeFrequency, TimeUnit.MILLISECONDS);
}
}
}

View File

@ -0,0 +1,268 @@
/**
* 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.procedure;
import java.io.IOException;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.DaemonThreadFactory;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import com.google.common.collect.MapMaker;
/**
* This is the master side of a distributed complex procedure execution.
* <p>
* The {@link Procedure} is generic and subclassing or customization shouldn't be
* necessary -- any customization should happen just in {@link Subprocedure}s.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ProcedureCoordinator {
private static final Log LOG = LogFactory.getLog(ProcedureCoordinator.class);
final static long TIMEOUT_MILLIS_DEFAULT = 60000;
final static long WAKE_MILLIS_DEFAULT = 500;
private final ProcedureCoordinatorRpcs rpcs;
private final ExecutorService pool;
// Running procedure table. Maps procedure name to running procedure reference
private final ConcurrentMap<String, Procedure> procedures =
new MapMaker().concurrencyLevel(4).weakValues().makeMap();
/**
* Create and start a ProcedureCoordinator.
*
* The rpc object registers the ProcedureCoordinator and starts any threads in this
* constructor.
*
* @param rpcs
* @param pool Used for executing procedures.
*/
public ProcedureCoordinator(ProcedureCoordinatorRpcs rpcs, ThreadPoolExecutor pool) {
this.rpcs = rpcs;
this.pool = pool;
this.rpcs.start(this);
}
/**
* Default thread pool for the procedure
*/
public static ThreadPoolExecutor defaultPool(String coordName, long keepAliveTime, int opThreads,
long wakeFrequency) {
return new ThreadPoolExecutor(1, opThreads, keepAliveTime, TimeUnit.SECONDS,
new SynchronousQueue<Runnable>(),
new DaemonThreadFactory("(" + coordName + ")-proc-coordinator-pool"));
}
/**
* Shutdown the thread pools and release rpc resources
* @throws IOException
*/
public void close() throws IOException {
// have to use shutdown now to break any latch waiting
pool.shutdownNow();
rpcs.close();
}
/**
* Submit an procedure to kick off its dependent subprocedures.
* @param proc Procedure to execute
* @return <tt>true</tt> if the procedure was started correctly, <tt>false</tt> if the
* procedure or any subprocedures could not be started. Failure could be due to
* submitting a procedure multiple times (or one with the same name), or some sort
* of IO problem. On errors, the procedure's monitor holds a reference to the exception
* that caused the failure.
*/
boolean submitProcedure(Procedure proc) {
// if the submitted procedure was null, then we don't want to run it
if (proc == null) {
return false;
}
String procName = proc.getName();
// make sure we aren't already running a procedure of that name
synchronized (procedures) {
Procedure oldProc = procedures.get(procName);
if (oldProc != null) {
// procedures are always eventually completed on both successful and failed execution
if (oldProc.completedLatch.getCount() != 0) {
LOG.warn("Procedure " + procName + " currently running. Rejecting new request");
return false;
}
LOG.debug("Procedure " + procName + " was in running list but was completed. Accepting new attempt.");
procedures.remove(procName);
}
}
// kick off the procedure's execution in a separate thread
Future<Void> f = null;
try {
synchronized (procedures) {
f = this.pool.submit(proc);
// if everything got started properly, we can add it known running procedures
this.procedures.put(procName, proc);
}
return true;
} catch (RejectedExecutionException e) {
LOG.warn("Procedure " + procName + " rejected by execution pool. Propagating error and " +
"cancelling operation.", e);
// the thread pool is full and we can't run the procedure
proc.receive(new ForeignException(procName, e));
// cancel procedure proactively
if (f != null) {
f.cancel(true);
}
}
return false;
}
/**
* The connection to the rest of the procedure group (members and coordinator) has been
* broken/lost/failed. This should fail any interested procedures, but not attempt to notify other
* members since we cannot reach them anymore.
* @param message description of the error
* @param cause the actual cause of the failure
*/
void rpcConnectionFailure(final String message, final IOException cause) {
Collection<Procedure> toNotify = procedures.values();
for (Procedure proc : toNotify) {
if (proc == null) {
continue;
}
// notify the elements, if they aren't null
proc.receive(new ForeignException(proc.getName(), cause));
}
}
/**
* Abort the procedure with the given name
* @param procName name of the procedure to abort
* @param reason serialized information about the abort
*/
public void abortProcedure(String procName, ForeignException reason) {
// if we know about the Procedure, notify it
synchronized(procedures) {
Procedure proc = procedures.get(procName);
if (proc == null) {
return;
}
proc.receive(reason);
}
}
/**
* Exposed for hooking with unit tests.
* @param procName
* @param procArgs
* @param expectedMembers
* @return
*/
Procedure createProcedure(ForeignExceptionDispatcher fed, String procName, byte[] procArgs,
List<String> expectedMembers) {
// build the procedure
return new Procedure(this, fed, WAKE_MILLIS_DEFAULT, TIMEOUT_MILLIS_DEFAULT,
procName, procArgs, expectedMembers);
}
/**
* Kick off the named procedure
* @param procName name of the procedure to start
* @param procArgs arguments for the procedure
* @param expectedMembers expected members to start
* @return handle to the running procedure, if it was started correctly, <tt>null</tt> otherwise
* @throws RejectedExecutionException if there are no more available threads to run the procedure
*/
public Procedure startProcedure(ForeignExceptionDispatcher fed, String procName, byte[] procArgs,
List<String> expectedMembers) throws RejectedExecutionException {
Procedure proc = createProcedure(fed, procName, procArgs, expectedMembers);
if (!this.submitProcedure(proc)) {
LOG.error("Failed to submit procedure '" + procName + "'");
return null;
}
return proc;
}
/**
* Notification that the procedure had the specified member acquired its part of the barrier
* via {@link Subprocedure#acquireBarrier()}.
* @param procName name of the procedure that acquired
* @param member name of the member that acquired
*/
void memberAcquiredBarrier(String procName, final String member) {
Procedure proc = procedures.get(procName);
if (proc != null) {
proc.barrierAcquiredByMember(member);
}
}
/**
* Notification that the procedure had another member finished executing its in-barrier subproc
* via {@link Subprocedure#insideBarrier()}.
* @param procName name of the subprocedure that finished
* @param member name of the member that executed and released its barrier
*/
void memberFinishedBarrier(String procName, final String member) {
Procedure proc = procedures.get(procName);
if (proc != null) {
proc.barrierReleasedByMember(member);
}
}
/**
* @return the rpcs implementation for all current procedures
*/
ProcedureCoordinatorRpcs getRpcs() {
return rpcs;
}
/**
* Returns the procedure. This Procedure is a live instance so should not be modified but can
* be inspected.
* @param name Name of the procedure
* @return Procedure or null if not present any more
*/
public Procedure getProcedure(String name) {
return procedures.get(name);
}
/**
* @return Return set of all procedure names.
*/
public Set<String> getProcedureNames() {
return new HashSet<String>(procedures.keySet());
}
}

View File

@ -0,0 +1,85 @@
/**
* 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.procedure;
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
/**
* RPCs for the coordinator to run a barriered procedure with subprocedures executed at
* distributed members.
* @see ProcedureCoordinator
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface ProcedureCoordinatorRpcs extends Closeable {
/**
* Initialize and start threads necessary to connect an implementation's rpc mechanisms.
* @param listener
* @return true if succeed, false if encountered initialization errors.
*/
public boolean start(final ProcedureCoordinator listener);
/**
* Notify the members that the coordinator has aborted the procedure and that it should release
* barrier resources.
*
* @param procName name of the procedure that was aborted
* @param cause the reason why the procedure needs to be aborted
* @throws IOException if the rpcs can't reach the other members of the procedure (and can't
* recover).
*/
public void sendAbortToMembers(Procedure procName, ForeignException cause) throws IOException;
/**
* Notify the members to acquire barrier for the procedure
*
* @param procName name of the procedure to start
* @param info information that should be passed to all members
* @param members names of the members requested to reach the acquired phase
* @throws IllegalArgumentException if the procedure was already marked as failed
* @throws IOException if we can't reach the remote notification mechanism
*/
public void sendGlobalBarrierAcquire(Procedure procName, byte[] info, List<String> members)
throws IOException, IllegalArgumentException;
/**
* Notify members that all members have acquired their parts of the barrier and that they can
* now execute under the global barrier.
*
* Must come after calling {@link #sendGlobalBarrierAcquire(Procedure, byte[], List)}
*
* @param procName name of the procedure to start
* @param members members to tell we have reached in-barrier phase
* @throws IOException if we can't reach the remote notification mechanism
*/
public void sendGlobalBarrierReached(Procedure procName, List<String> members) throws IOException;
/**
* Notify Members to reset the distributed state for procedure
* @param procName name of the procedure to reset
* @throws IOException if the remote notification mechanism cannot be reached
*/
public void resetMembers(Procedure procName) throws IOException;
}

View File

@ -0,0 +1,232 @@
/**
* 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.procedure;
import java.io.Closeable;
import java.io.IOException;
import java.util.Collection;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.DaemonThreadFactory;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import com.google.common.collect.MapMaker;
/**
* Process to kick off and manage a running {@link Subprocedure} on a member. This is the
* specialized part of a {@link Procedure} that actually does procedure type-specific work
* and reports back to the coordinator as it completes each phase.
* <p>
* If there is a connection error ({@link #controllerConnectionFailure(String, IOException)}), all
* currently running subprocedures are notify to failed since there is no longer a way to reach any
* other members or coordinators since the rpcs are down.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ProcedureMember implements Closeable {
private static final Log LOG = LogFactory.getLog(ProcedureMember.class);
private final SubprocedureFactory builder;
private final ProcedureMemberRpcs rpcs;
private final ConcurrentMap<String,Subprocedure> subprocs =
new MapMaker().concurrencyLevel(4).weakValues().makeMap();
private final ExecutorService pool;
/**
* Instantiate a new ProcedureMember. This is a slave that executes subprocedures.
*
* @param rpcs controller used to send notifications to the procedure coordinator
* @param pool thread pool to submit subprocedures
* @param factory class that creates instances of a subprocedure.
*/
public ProcedureMember(ProcedureMemberRpcs rpcs, ThreadPoolExecutor pool,
SubprocedureFactory factory) {
this.pool = pool;
this.rpcs = rpcs;
this.builder = factory;
}
public static ThreadPoolExecutor defaultPool(long wakeFrequency, long keepAlive,
int procThreads, String memberName) {
return new ThreadPoolExecutor(1, procThreads, keepAlive, TimeUnit.SECONDS,
new SynchronousQueue<Runnable>(),
new DaemonThreadFactory("member: '" + memberName + "' subprocedure-pool"));
}
/**
* Package exposed. Not for public use.
*
* @return reference to the Procedure member's rpcs object
*/
ProcedureMemberRpcs getRpcs() {
return rpcs;
}
/**
* This is separated from execution so that we can detect and handle the case where the
* subprocedure is invalid and inactionable due to bad info (like DISABLED snapshot type being
* sent here)
* @param opName
* @param data
* @return subprocedure
*/
public Subprocedure createSubprocedure(String opName, byte[] data) {
return builder.buildSubprocedure(opName, data);
}
/**
* Submit an subprocedure for execution. This starts the local acquire phase.
* @param subproc the subprocedure to execute.
* @return <tt>true</tt> if the subprocedure was started correctly, <tt>false</tt> if it
* could not be started. In the latter case, the subprocedure holds a reference to
* the exception that caused the failure.
*/
public boolean submitSubprocedure(Subprocedure subproc) {
// if the submitted subprocedure was null, bail.
if (subproc == null) {
LOG.warn("Submitted null subprocedure, nothing to run here.");
return false;
}
String procName = subproc.getName();
if (procName == null || procName.length() == 0) {
LOG.error("Subproc name cannot be null or the empty string");
return false;
}
// make sure we aren't already running an subprocedure of that name
Subprocedure rsub;
synchronized (subprocs) {
rsub = subprocs.get(procName);
}
if (rsub != null) {
if (!rsub.isComplete()) {
LOG.error("Subproc '" + procName + "' is already running. Bailing out");
return false;
}
LOG.warn("A completed old subproc " + procName + " is still present, removing");
subprocs.remove(procName);
}
LOG.debug("Submitting new Subprocedure:" + procName);
// kick off the subprocedure
Future<Void> future = null;
try {
future = this.pool.submit(subproc);
synchronized (subprocs) {
subprocs.put(procName, subproc);
}
return true;
} catch (RejectedExecutionException e) {
// the thread pool is full and we can't run the subprocedure
String msg = "Subprocedure pool is full!";
subproc.cancel(msg, e.getCause());
// cancel all subprocedures proactively
if (future != null) {
future.cancel(true);
}
}
LOG.error("Failed to start subprocedure '" + procName + "'");
return false;
}
/**
* Notification that procedure coordinator has reached the global barrier
* @param procName name of the subprocedure that should start running the the in-barrier phase
*/
public void receivedReachedGlobalBarrier(String procName) {
Subprocedure subproc = subprocs.get(procName);
if (subproc == null) {
LOG.warn("Unexpected reached glabal barrier message for Sub-Procedure '" + procName + "'");
return;
}
subproc.receiveReachedGlobalBarrier();
}
/**
* Best effort attempt to close the threadpool via Thread.interrupt.
*/
@Override
public void close() throws IOException {
// have to use shutdown now to break any latch waiting
pool.shutdownNow();
}
/**
* Shutdown the threadpool, and wait for upto timeoutMs millis before bailing
* @param timeoutMs timeout limit in millis
* @return true if successfully, false if bailed due to timeout.
* @throws InterruptedException
*/
boolean closeAndWait(long timeoutMs) throws InterruptedException {
pool.shutdown();
return pool.awaitTermination(timeoutMs, TimeUnit.MILLISECONDS);
}
/**
* The connection to the rest of the procedure group (member and coordinator) has been
* broken/lost/failed. This should fail any interested subprocedure, but not attempt to notify
* other members since we cannot reach them anymore.
* @param message description of the error
* @param cause the actual cause of the failure
*
* TODO i'm tempted to just remove this code completely and treat it like any other abort.
* Implementation wise, if this happens it is a ZK failure which means the RS will abort.
*/
public void controllerConnectionFailure(final String message, final IOException cause) {
Collection<Subprocedure> toNotify = subprocs.values();
LOG.error(message, cause);
for (Subprocedure sub : toNotify) {
// TODO notify the elements, if they aren't null
sub.cancel(message, cause);
}
}
/**
* Send abort to the specified procedure
* @param procName name of the procedure to about
* @param ee exception information about the abort
*/
public void receiveAbortProcedure(String procName, ForeignException ee) {
LOG.debug("Request received to abort procedure " + procName, ee);
// if we know about the procedure, notify it
Subprocedure sub = subprocs.get(procName);
if (sub == null) {
LOG.info("Received abort on procedure with no local subprocedure " + procName +
", ignoring it.", ee);
return; // Procedure has already completed
}
LOG.error("Propagating foreign exception to subprocedure " + sub.getName(), ee);
sub.monitor.receive(ee);
}
}

View File

@ -0,0 +1,73 @@
/**
* 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.procedure;
import java.io.Closeable;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
/**
* This is the notification interface for Procedures that encapsulates message passing from
* members to a coordinator. Each of these calls should send a message to the coordinator.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface ProcedureMemberRpcs extends Closeable {
/**
* Initialize and start any threads or connections the member needs.
*/
public void start(ProcedureMember member);
/**
* Each subprocedure is being executed on a member. This is the identifier for the member.
* @return the member name
*/
public String getMemberName();
/**
* Notify the coordinator that we aborted the specified {@link Subprocedure}
*
* @param sub the {@link Subprocedure} we are aborting
* @param cause the reason why the member's subprocedure aborted
* @throws IOException thrown when the rpcs can't reach the other members of the procedure (and
* thus can't recover).
*/
public void sendMemberAborted(Subprocedure sub, ForeignException cause) throws IOException;
/**
* Notify the coordinator that the specified {@link Subprocedure} has acquired the locally required
* barrier condition.
*
* @param sub the specified {@link Subprocedure}
* @throws IOException if we can't reach the coordinator
*/
public void sendMemberAcquired(Subprocedure sub) throws IOException;
/**
* Notify the coordinator that the specified {@link Subprocedure} has completed the work that
* needed to be done under the global barrier.
*
* @param sub the specified {@link Subprocedure}
* @throws IOException if we can't reach the coordinator
*/
public void sendMemberCompleted(Subprocedure sub) throws IOException;
}

View File

@ -0,0 +1,331 @@
/**
* 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.procedure;
import java.io.IOException;
import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionListener;
import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector;
/**
* Distributed procedure member's Subprocedure. A procedure is sarted on a ProcedureCoordinator
* which communicates with ProcedureMembers who create and start its part of the Procedure. This
* sub part is called a Subprocedure
*
* Users should subclass this and implement {@link #acquireBarrier()} (get local barrier for this
* member), {@link #insideBarrier()} (execute while globally barriered and release barrier) and
* {@link #cleanup(Exception)} (release state associated with subprocedure.)
*
* When submitted to a ProcedureMemeber, the call method is executed in a separate thread.
* Latches are use too block its progress and trigger continuations when barrier conditions are
* met.
*
* Exception that makes it out of calls to {@link #acquireBarrier()} or {@link #insideBarrier()}
* gets converted into {@link ForeignException}, which will get propagated to the
* {@link ProcedureCoordinator}.
*
* There is a category of procedure (ex: online-snapshots), and a user-specified instance-specific
* barrierName. (ex: snapshot121126).
*/
abstract public class Subprocedure implements Callable<Void> {
private static final Log LOG = LogFactory.getLog(Subprocedure.class);
// Name of the procedure
final private String barrierName;
//
// Execution state
//
/** wait on before allowing the in barrier phase to proceed */
private final CountDownLatch inGlobalBarrier;
/** counted down when the Subprocedure has completed */
private final CountDownLatch releasedLocalBarrier;
//
// Error handling
//
/** monitor to check for errors */
protected final ForeignExceptionDispatcher monitor;
/** frequency to check for errors (ms) */
protected final long wakeFrequency;
protected final TimeoutExceptionInjector executionTimeoutTimer;
protected final ProcedureMemberRpcs rpcs;
private volatile boolean complete = false;
/**
* @param member reference to the member managing this subprocedure
* @param procName name of the procedure this subprocedure is associated with
* @param monitor notified if there is an error in the subprocedure
* @param wakeFrequency time in millis to wake to check if there is an error via the monitor (in
* milliseconds).
* @param timeout time in millis that will trigger a subprocedure abort if it has not completed
*/
public Subprocedure(ProcedureMember member, String procName, ForeignExceptionDispatcher monitor,
long wakeFrequency, long timeout) {
// Asserts should be caught during unit testing
assert member != null : "procedure member should be non-null";
assert member.getRpcs() != null : "rpc handlers should be non-null";
assert procName != null : "procedure name should be non-null";
assert monitor != null : "monitor should be non-null";
// Default to a very large timeout
this.rpcs = member.getRpcs();
this.barrierName = procName;
this.monitor = monitor;
// forward any failures to coordinator. Since this is a dispatcher, resend loops should not be
// possible.
this.monitor.addListener(new ForeignExceptionListener() {
@Override
public void receive(ForeignException ee) {
// if this is a notification from a remote source, just log
if (ee.isRemote()) {
LOG.debug("Was remote foreign exception, not redispatching error", ee);
return;
}
// if it is local, then send it to the coordinator
try {
rpcs.sendMemberAborted(Subprocedure.this, ee);
} catch (IOException e) {
// this will fail all the running procedures, since the connection is down
LOG.error("Can't reach controller, not propagating error", e);
}
}
});
this.wakeFrequency = wakeFrequency;
this.inGlobalBarrier = new CountDownLatch(1);
this.releasedLocalBarrier = new CountDownLatch(1);
// accept error from timer thread, this needs to be started.
this.executionTimeoutTimer = new TimeoutExceptionInjector(monitor, timeout);
}
public String getName() {
return barrierName;
}
public String getMemberName() {
return rpcs.getMemberName();
}
private void rethrowException() throws ForeignException {
monitor.rethrowException();
}
/**
* Execute the Subprocedure {@link #acquireBarrier()} and {@link #insideBarrier()} methods
* while keeping some state for other threads to access.
*
* This would normally be executed by the ProcedureMemeber when a acquire message comes from the
* coordinator. Rpcs are used to spend message back to the coordinator after different phases
* are executed. Any exceptions caught during the execution (except for InterrupedException) get
* converted and propagated to coordinator via {@link ProcedureMemberRpcs#sendMemberAborted(
* Subprocedure, ForeignException)}.
*/
@SuppressWarnings("finally")
final public Void call() {
LOG.debug("Starting subprocedure '" + barrierName + "' with timeout " +
executionTimeoutTimer.getMaxTime() + "ms");
// start the execution timeout timer
executionTimeoutTimer.start();
try {
// start by checking for error first
rethrowException();
LOG.debug("Subprocedure '" + barrierName + "' starting 'acquire' stage");
acquireBarrier();
LOG.debug("Subprocedure '" + barrierName + "' locally acquired");
// vote yes to coordinator about being prepared
rpcs.sendMemberAcquired(this);
LOG.debug("Subprocedure '" + barrierName + "' coordinator notified of 'acquire', waiting on" +
" 'reached' or 'abort' from coordinator");
// wait for the procedure to reach global barrier before proceding
waitForReachedGlobalBarrier();
rethrowException(); // if Coordinator aborts, will bail from here with exception
// In traditional 2PC, if a member reaches this state the TX has been committed and the
// member is responsible for rolling forward and recovering and completing the subsequent
// operations in the case of failure. It cannot rollback.
//
// This implementation is not 2PC since it can still rollback here, and thus has different
// semantics.
LOG.debug("Subprocedure '" + barrierName + "' received 'reached' from coordinator.");
insideBarrier();
LOG.debug("Subprocedure '" + barrierName + "' locally completed");
// Ack that the member has executed and released local barrier
rpcs.sendMemberCompleted(this);
LOG.debug("Subprocedure '" + barrierName + "' has notified controller of completion");
// make sure we didn't get an external exception
rethrowException();
} catch (Exception e) {
String msg = null;
if (e instanceof InterruptedException) {
msg = "Procedure '" + barrierName + "' aborting due to interrupt!" +
" Likely due to pool shutdown.";
Thread.currentThread().interrupt();
} else if (e instanceof ForeignException) {
msg = "Subprocedure '" + barrierName + "' aborting due to a ForeignException!";
} else {
msg = "Subprocedure '" + barrierName + "' failed!";
}
LOG.error(msg , e);
cancel(msg, e);
LOG.debug("Subprocedure '" + barrierName + "' running cleanup.");
cleanup(e);
} finally {
releasedLocalBarrier.countDown();
// tell the timer we are done, if we get here successfully
executionTimeoutTimer.complete();
complete = true;
LOG.debug("Subprocedure '" + barrierName + "' completed.");
return null;
}
}
boolean isComplete() {
return complete;
}
/**
* exposed for testing.
*/
ForeignExceptionSnare getErrorCheckable() {
return this.monitor;
}
/**
* The implementation of this method should gather and hold required resources (locks, disk
* space, etc) to satisfy the Procedures barrier condition. For example, this would be where
* to make all the regions on a RS on the quiescent for an procedure that required all regions
* to be globally quiesed.
*
* Users should override this method. If a quiescent is not required, this is overkill but
* can still be used to execute a procedure on all members and to propagate any exceptions.
*
* @throws ForeignException
*/
abstract public void acquireBarrier() throws ForeignException;
/**
* The implementation of this method should act with the assumption that the barrier condition
* has been satisfied. Continuing the previous example, a condition could be that all RS's
* globally have been quiesced, and procedures that require this precondition could be
* implemented here.
*
* Users should override this method. If quiescense is not required, this can be a no-op
*
* @throws ForeignException
*/
abstract public void insideBarrier() throws ForeignException;
/**
* Users should override this method. This implementation of this method should rollback and
* cleanup any temporary or partially completed state that the {@link #acquireBarrier()} may have
* created.
* @param e
*/
abstract public void cleanup(Exception e);
/**
* Method to cancel the Subprocedure by injecting an exception from and external source.
* @param cause
*/
public void cancel(String msg, Throwable cause) {
LOG.error(msg, cause);
if (cause instanceof ForeignException) {
monitor.receive((ForeignException) cause);
} else {
monitor.receive(new ForeignException(getMemberName(), cause));
}
}
/**
* Callback for the member rpcs to call when the global barrier has been reached. This
* unblocks the main subprocedure exectuion thread so that the Subprocedure's
* {@link #insideBarrier()} method can be run.
*/
public void receiveReachedGlobalBarrier() {
inGlobalBarrier.countDown();
}
//
// Subprocedure Internal State interface
//
/**
* Wait for the reached global barrier notification.
*
* Package visibility for testing
*
* @throws ForeignException
* @throws InterruptedException
*/
void waitForReachedGlobalBarrier() throws ForeignException, InterruptedException {
Procedure.waitForLatch(inGlobalBarrier, monitor, wakeFrequency,
barrierName + ":remote acquired");
}
/**
* Waits until the entire procedure has globally completed, or has been aborted.
* @throws ForeignException
* @throws InterruptedException
*/
public void waitForLocallyCompleted() throws ForeignException, InterruptedException {
Procedure.waitForLatch(releasedLocalBarrier, monitor, wakeFrequency,
barrierName + ":completed");
}
/**
* Empty Subprocedure for testing.
*
* Must be public for stubbing used in testing to work.
*/
public static class SubprocedureImpl extends Subprocedure {
public SubprocedureImpl(ProcedureMember member, String opName,
ForeignExceptionDispatcher monitor, long wakeFrequency, long timeout) {
super(member, opName, monitor, wakeFrequency, timeout);
}
@Override
public void acquireBarrier() throws ForeignException {}
@Override
public void insideBarrier() throws ForeignException {}
@Override
public void cleanup(Exception e) {}
};
}

View File

@ -0,0 +1,40 @@
/**
* 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.procedure;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Task builder to build instances of a {@link ProcedureMember}'s {@link Subprocedure}s.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface SubprocedureFactory {
/**
* Build {@link Subprocedure} when requested.
* @param procName name of the procedure associated with this subprocedure
* @param procArgs arguments passed from the coordinator about the procedure
* @return {@link Subprocedure} to run or <tt>null</tt> if the no operation should be run
* @throws IllegalArgumentException if the operation could not be run because of errors in the
* request
* @throws IllegalStateException if the current runner cannot accept any more new requests
*/
public Subprocedure buildSubprocedure(String procName, byte[] procArgs);
}

View File

@ -0,0 +1,267 @@
/**
* 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.procedure;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* ZooKeeper based {@link ProcedureCoordinatorRpcs} for a {@link ProcedureCoordinator}
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs {
public static final Log LOG = LogFactory.getLog(ZKProcedureCoordinatorRpcs.class);
private ZKProcedureUtil zkProc = null;
protected ProcedureCoordinator coordinator = null; // if started this should be non-null
ZooKeeperWatcher watcher;
String procedureType;
String coordName;
/**
* @param watcher zookeeper watcher. Owned by <tt>this</tt> and closed via {@link #close()}
* @param procedureClass procedure type name is a category for when there are multiple kinds of
* procedures.-- this becomes a znode so be aware of the naming restrictions
* @param coordName name of the node running the coordinator
* @throws KeeperException if an unexpected zk error occurs
*/
public ZKProcedureCoordinatorRpcs(ZooKeeperWatcher watcher,
String procedureClass, String coordName) throws KeeperException {
this.watcher = watcher;
this.procedureType = procedureClass;
this.coordName = coordName;
}
/**
* The "acquire" phase. The coordinator creates a new procType/acquired/ znode dir. If znodes
* appear, first acquire to relevant listener or sets watch waiting for notification of
* the acquire node
*
* @param proc the Procedure
* @param info data to be stored in the acquire node
* @param nodeNames children of the acquire phase
* @throws IOException if any failure occurs.
*/
@Override
final public void sendGlobalBarrierAcquire(Procedure proc, byte[] info, List<String> nodeNames)
throws IOException, IllegalArgumentException {
String procName = proc.getName();
// start watching for the abort node
String abortNode = zkProc.getAbortZNode(procName);
try {
// check to see if the abort node already exists
if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), abortNode)) {
abort(abortNode);
}
// If we get an abort node watch triggered here, we'll go complete creating the acquired
// znode but then handle the acquire znode and bail out
} catch (KeeperException e) {
LOG.error("Failed to watch abort", e);
throw new IOException("Failed while watching abort node:" + abortNode, e);
}
// create the acquire barrier
String acquire = zkProc.getAcquiredBarrierNode(procName);
LOG.debug("Creating acquire znode:" + acquire);
try {
// notify all the procedure listeners to look for the acquire node
byte[] data = ProtobufUtil.prependPBMagic(info);
ZKUtil.createWithParents(zkProc.getWatcher(), acquire, data);
// loop through all the children of the acquire phase and watch for them
for (String node : nodeNames) {
String znode = ZKUtil.joinZNode(acquire, node);
LOG.debug("Watching for acquire node:" + znode);
if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), znode)) {
coordinator.memberAcquiredBarrier(procName, node);
}
}
} catch (KeeperException e) {
throw new IOException("Failed while creating acquire node:" + acquire, e);
}
}
@Override
public void sendGlobalBarrierReached(Procedure proc, List<String> nodeNames) throws IOException {
String procName = proc.getName();
String reachedNode = zkProc.getReachedBarrierNode(procName);
LOG.debug("Creating reached barrier zk node:" + reachedNode);
try {
// create the reached znode and watch for the reached znodes
ZKUtil.createWithParents(zkProc.getWatcher(), reachedNode);
// loop through all the children of the acquire phase and watch for them
for (String node : nodeNames) {
String znode = ZKUtil.joinZNode(reachedNode, node);
if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), znode)) {
coordinator.memberFinishedBarrier(procName, node);
}
}
} catch (KeeperException e) {
throw new IOException("Failed while creating reached node:" + reachedNode, e);
}
}
/**
* Delete znodes that are no longer in use.
*/
@Override
final public void resetMembers(Procedure proc) throws IOException {
String procName = proc.getName();
boolean stillGettingNotifications = false;
do {
try {
LOG.debug("Attempting to clean out zk node for op:" + procName);
zkProc.clearZNodes(procName);
stillGettingNotifications = false;
} catch (KeeperException.NotEmptyException e) {
// recursive delete isn't transactional (yet) so we need to deal with cases where we get
// children trickling in
stillGettingNotifications = true;
} catch (KeeperException e) {
throw new IOException("Failed to complete reset procedure " + procName, e);
}
} while (stillGettingNotifications);
}
/**
* Start monitoring znodes in ZK - subclass hook to start monitoring znodes they are about.
* @return true if succeed, false if encountered initialization errors.
*/
final public boolean start(final ProcedureCoordinator coordinator) {
if (this.coordinator != null) {
throw new IllegalStateException(
"ZKProcedureCoordinator already started and already has listener installed");
}
this.coordinator = coordinator;
try {
this.zkProc = new ZKProcedureUtil(watcher, procedureType, coordName) {
@Override
public void nodeCreated(String path) {
if (!isInProcedurePath(path)) return;
LOG.debug("Node created: " + path);
logZKTree(this.baseZNode);
if (isAcquiredPathNode(path)) {
// node wasn't present when we created the watch so zk event triggers acquire
coordinator.memberAcquiredBarrier(ZKUtil.getNodeName(ZKUtil.getParent(path)),
ZKUtil.getNodeName(path));
} else if (isReachedPathNode(path)) {
// node was absent when we created the watch so zk event triggers the finished barrier.
// TODO Nothing enforces that acquire and reached znodes from showing up in wrong order.
coordinator.memberFinishedBarrier(ZKUtil.getNodeName(ZKUtil.getParent(path)),
ZKUtil.getNodeName(path));
} else if (isAbortPathNode(path)) {
abort(path);
}
}
};
zkProc.clearChildZNodes();
} catch (KeeperException e) {
LOG.error("Unable to start the ZK-based Procedure Coordinator rpcs.", e);
return false;
}
LOG.debug("Starting the controller for procedure member:" + zkProc.getMemberName());
return true;
}
/**
* This is the abort message being sent by the coordinator to member
*
* TODO this code isn't actually used but can be used to issue a cancellation from the
* coordinator.
*/
@Override
final public void sendAbortToMembers(Procedure proc, ForeignException ee) {
String procName = proc.getName();
LOG.debug("Aborting procedure '" + procName + "' in zk");
String procAbortNode = zkProc.getAbortZNode(procName);
try {
LOG.debug("Creating abort znode:" + procAbortNode);
String source = (ee.getSource() == null) ? coordName : ee.getSource();
byte[] errorInfo = ProtobufUtil.prependPBMagic(ForeignException.serialize(source, ee));
// first create the znode for the procedure
ZKUtil.createAndFailSilent(zkProc.getWatcher(), procAbortNode, errorInfo);
LOG.debug("Finished creating abort node:" + procAbortNode);
} catch (KeeperException e) {
// possible that we get this error for the procedure if we already reset the zk state, but in
// that case we should still get an error for that procedure anyways
zkProc.logZKTree(zkProc.baseZNode);
coordinator.rpcConnectionFailure("Failed to post zk node:" + procAbortNode
+ " to abort procedure '" + procName + "'", new IOException(e));
}
}
/**
* Receive a notification and propagate it to the local coordinator
* @param abortNode full znode path to the failed procedure information
*/
protected void abort(String abortNode) {
String procName = ZKUtil.getNodeName(abortNode);
ForeignException ee = null;
try {
byte[] data = ZKUtil.getData(zkProc.getWatcher(), abortNode);
if (!ProtobufUtil.isPBMagicPrefix(data)) {
LOG.warn("Got an error notification for op:" + abortNode
+ " but we can't read the information. Killing the procedure.");
// we got a remote exception, but we can't describe it
ee = new ForeignException(coordName, "Data in abort node is illegally formatted. ignoring content.");
} else {
data = Arrays.copyOfRange(data, ProtobufUtil.lengthOfPBMagic(), data.length);
ee = ForeignException.deserialize(data);
}
} catch (InvalidProtocolBufferException e) {
LOG.warn("Got an error notification for op:" + abortNode
+ " but we can't read the information. Killing the procedure.");
// we got a remote exception, but we can't describe it
ee = new ForeignException(coordName, e);
} catch (KeeperException e) {
coordinator.rpcConnectionFailure("Failed to get data for abort node:" + abortNode
+ zkProc.getAbortZnode(), new IOException(e));
}
coordinator.abortProcedure(procName, ee);
}
@Override
final public void close() throws IOException {
zkProc.close();
}
/**
* Used in testing
*/
final ZKProcedureUtil getZkProcedureUtil() {
return zkProc;
}
}

View File

@ -0,0 +1,350 @@
/**
* 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.procedure;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* ZooKeeper based controller for a procedure member.
* <p>
* There can only be one {@link ZKProcedureMemberRpcs} per procedure type per member,
* since each procedure type is bound to a single set of znodes. You can have multiple
* {@link ZKProcedureMemberRpcs} on the same server, each serving a different member
* name, but each individual rpcs is still bound to a single member name (and since they are
* used to determine global progress, its important to not get this wrong).
* <p>
* To make this slightly more confusing, you can run multiple, concurrent procedures at the same
* time (as long as they have different types), from the same controller, but the same node name
* must be used for each procedure (though there is no conflict between the two procedure as long
* as they have distinct names).
* <p>
* There is no real error recovery with this mechanism currently -- if any the coordinator fails,
* its re-initialization will delete the znodes and require all in progress subprocedures to start
* anew.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs {
private static final Log LOG = LogFactory.getLog(ZKProcedureMemberRpcs.class);
private final String memberName;
protected ProcedureMember member;
private ZKProcedureUtil zkController;
/**
* Must call {@link #start(ProcedureMember)} before this can be used.
* @param watcher {@link ZooKeeperWatcher} to be owned by <tt>this</tt>. Closed via
* {@link #close()}.
* @param procType name of the znode describing the procedure type
* @param memberName name of the member to join the procedure
* @throws KeeperException if we can't reach zookeeper
*/
public ZKProcedureMemberRpcs(ZooKeeperWatcher watcher,
String procType, String memberName) throws KeeperException {
this.zkController = new ZKProcedureUtil(watcher, procType, memberName) {
@Override
public void nodeCreated(String path) {
if (!isInProcedurePath(path)) {
return;
}
LOG.info("Received created event:" + path);
// if it is a simple start/end/abort then we just rewatch the node
if (isAcquiredNode(path)) {
waitForNewProcedures();
return;
} else if (isAbortNode(path)) {
watchForAbortedProcedures();
return;
}
String parent = ZKUtil.getParent(path);
// if its the end barrier, the procedure can be completed
if (isReachedNode(parent)) {
receivedReachedGlobalBarrier(path);
return;
} else if (isAbortNode(parent)) {
abort(path);
return;
} else if (isAcquiredNode(parent)) {
startNewSubprocedure(path);
} else {
LOG.debug("Ignoring created notification for node:" + path);
}
}
@Override
public void nodeChildrenChanged(String path) {
if (path.equals(this.acquiredZnode)) {
LOG.info("Received procedure start children changed event: " + path);
waitForNewProcedures();
} else if (path.equals(this.abortZnode)) {
LOG.info("Received procedure abort children changed event: " + path);
watchForAbortedProcedures();
}
}
};
this.memberName = memberName;
}
public ZKProcedureUtil getZkController() {
return zkController;
}
@Override
public String getMemberName() {
return memberName;
}
/**
* Pass along the procedure global barrier notification to any listeners
* @param path full znode path that cause the notification
*/
private void receivedReachedGlobalBarrier(String path) {
LOG.debug("Recieved reached global barrier:" + path);
String procName = ZKUtil.getNodeName(path);
this.member.receivedReachedGlobalBarrier(procName);
}
private void watchForAbortedProcedures() {
LOG.debug("Checking for aborted procedures on node: '" + zkController.getAbortZnode() + "'");
try {
// this is the list of the currently aborted procedues
for (String node : ZKUtil.listChildrenAndWatchForNewChildren(zkController.getWatcher(),
zkController.getAbortZnode())) {
String abortNode = ZKUtil.joinZNode(zkController.getAbortZnode(), node);
abort(abortNode);
}
} catch (KeeperException e) {
member.controllerConnectionFailure("Failed to list children for abort node:"
+ zkController.getAbortZnode(), new IOException(e));
}
}
private void waitForNewProcedures() {
// watch for new procedues that we need to start subprocedures for
LOG.debug("Looking for new procedures under znode:'" + zkController.getAcquiredBarrier() + "'");
List<String> runningProcedures = null;
try {
runningProcedures = ZKUtil.listChildrenAndWatchForNewChildren(zkController.getWatcher(),
zkController.getAcquiredBarrier());
if (runningProcedures == null) {
LOG.debug("No running procedures.");
return;
}
} catch (KeeperException e) {
member.controllerConnectionFailure("General failure when watching for new procedures",
new IOException(e));
}
if (runningProcedures == null) {
LOG.debug("No running procedures.");
return;
}
for (String procName : runningProcedures) {
// then read in the procedure information
String path = ZKUtil.joinZNode(zkController.getAcquiredBarrier(), procName);
startNewSubprocedure(path);
}
}
/**
* Kick off a new sub-procedure on the listener with the data stored in the passed znode.
* <p>
* Will attempt to create the same procedure multiple times if an procedure znode with the same
* name is created. It is left up the coordinator to ensure this doesn't occur.
* @param path full path to the znode for the procedure to start
*/
private synchronized void startNewSubprocedure(String path) {
LOG.debug("Found procedure znode: " + path);
String opName = ZKUtil.getNodeName(path);
// start watching for an abort notification for the procedure
String abortZNode = zkController.getAbortZNode(opName);
try {
if (ZKUtil.watchAndCheckExists(zkController.getWatcher(), abortZNode)) {
LOG.debug("Not starting:" + opName + " because we already have an abort notification.");
return;
}
} catch (KeeperException e) {
member.controllerConnectionFailure("Failed to get the abort znode (" + abortZNode
+ ") for procedure :" + opName, new IOException(e));
return;
}
// get the data for the procedure
Subprocedure subproc = null;
try {
byte[] data = ZKUtil.getData(zkController.getWatcher(), path);
LOG.debug("start proc data length is " + data.length);
if (!ProtobufUtil.isPBMagicPrefix(data)) {
String msg = "Data in for starting procuedure " + opName + " is illegally formatted. "
+ "Killing the procedure.";
LOG.error(msg);
throw new IllegalArgumentException(msg);
}
data = Arrays.copyOfRange(data, ProtobufUtil.lengthOfPBMagic(), data.length);
LOG.debug("Found data for znode:" + path);
subproc = member.createSubprocedure(opName, data);
member.submitSubprocedure(subproc);
} catch (IllegalArgumentException iae ) {
LOG.error("Illegal argument exception", iae);
sendMemberAborted(subproc, new ForeignException(getMemberName(), iae));
} catch (IllegalStateException ise) {
LOG.error("Illegal state exception ", ise);
sendMemberAborted(subproc, new ForeignException(getMemberName(), ise));
} catch (KeeperException e) {
member.controllerConnectionFailure("Failed to get data for new procedure:" + opName,
new IOException(e));
}
}
/**
* This attempts to create an acquired state znode for the procedure (snapshot name).
*
* It then looks for the reached znode to trigger in-barrier execution. If not present we
* have a watcher, if present then trigger the in-barrier action.
*/
@Override
public void sendMemberAcquired(Subprocedure sub) throws IOException {
String procName = sub.getName();
try {
LOG.debug("Member: '" + memberName + "' joining acquired barrier for procedure (" + procName
+ ") in zk");
String acquiredZNode = ZKUtil.joinZNode(ZKProcedureUtil.getAcquireBarrierNode(
zkController, procName), memberName);
ZKUtil.createAndFailSilent(zkController.getWatcher(), acquiredZNode);
// watch for the complete node for this snapshot
String reachedBarrier = zkController.getReachedBarrierNode(procName);
LOG.debug("Watch for global barrier reached:" + reachedBarrier);
if (ZKUtil.watchAndCheckExists(zkController.getWatcher(), reachedBarrier)) {
receivedReachedGlobalBarrier(reachedBarrier);
}
} catch (KeeperException e) {
member.controllerConnectionFailure("Failed to acquire barrier for procedure: "
+ procName + " and member: " + memberName, new IOException(e));
}
}
/**
* This acts as the ack for a completed snapshot
*/
@Override
public void sendMemberCompleted(Subprocedure sub) throws IOException {
String procName = sub.getName();
LOG.debug("Marking procedure '" + procName + "' completed for member '" + memberName
+ "' in zk");
String joinPath = ZKUtil.joinZNode(zkController.getReachedBarrierNode(procName), memberName);
try {
ZKUtil.createAndFailSilent(zkController.getWatcher(), joinPath);
} catch (KeeperException e) {
member.controllerConnectionFailure("Failed to post zk node:" + joinPath
+ " to join procedure barrier.", new IOException(e));
}
}
/**
* This should be called by the member and should write a serialized root cause exception as
* to the abort znode.
*/
@Override
public void sendMemberAborted(Subprocedure sub, ForeignException ee) {
if (sub == null) {
LOG.error("Failed due to null subprocedure", ee);
return;
}
String procName = sub.getName();
LOG.debug("Aborting procedure (" + procName + ") in zk");
String procAbortZNode = zkController.getAbortZNode(procName);
try {
String source = (ee.getSource() == null) ? memberName: ee.getSource();
byte[] errorInfo = ProtobufUtil.prependPBMagic(ForeignException.serialize(source, ee));
ZKUtil.createAndFailSilent(zkController.getWatcher(), procAbortZNode, errorInfo);
LOG.debug("Finished creating abort znode:" + procAbortZNode);
} catch (KeeperException e) {
// possible that we get this error for the procedure if we already reset the zk state, but in
// that case we should still get an error for that procedure anyways
zkController.logZKTree(zkController.getBaseZnode());
member.controllerConnectionFailure("Failed to post zk node:" + procAbortZNode
+ " to abort procedure", new IOException(e));
}
}
/**
* Pass along the found abort notification to the listener
* @param abortZNode full znode path to the failed procedure information
*/
protected void abort(String abortZNode) {
LOG.debug("Aborting procedure member for znode " + abortZNode);
String opName = ZKUtil.getNodeName(abortZNode);
try {
byte[] data = ZKUtil.getData(zkController.getWatcher(), abortZNode);
// figure out the data we need to pass
ForeignException ee;
try {
if (!ProtobufUtil.isPBMagicPrefix(data)) {
String msg = "Illegally formatted data in abort node for proc " + opName
+ ". Killing the procedure.";
LOG.error(msg);
// we got a remote exception, but we can't describe it so just return exn from here
ee = new ForeignException(getMemberName(), new IllegalArgumentException(msg));
} else {
data = Arrays.copyOfRange(data, ProtobufUtil.lengthOfPBMagic(), data.length);
ee = ForeignException.deserialize(data);
}
} catch (InvalidProtocolBufferException e) {
LOG.warn("Got an error notification for op:" + opName
+ " but we can't read the information. Killing the procedure.");
// we got a remote exception, but we can't describe it so just return exn from here
ee = new ForeignException(getMemberName(), e);
}
this.member.receiveAbortProcedure(opName, ee);
} catch (KeeperException e) {
member.controllerConnectionFailure("Failed to get data for abort znode:" + abortZNode
+ zkController.getAbortZnode(), new IOException(e));
}
}
public void start(ProcedureMember listener) {
LOG.debug("Starting procedure member '" + this.memberName + "'");
this.member = listener;
watchForAbortedProcedures();
waitForNewProcedures();
}
@Override
public void close() throws IOException {
zkController.close();
}
}

View File

@ -0,0 +1,285 @@
/**
* 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.procedure;
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
* This is a shared ZooKeeper-based znode management utils for distributed procedure. All znode
* operations should go through the provided methods in coordinators and members.
*
* Layout of nodes in ZK is
* /hbase/[op name]/acquired/
* [op instance] - op data/
* /[nodes that have acquired]
* /reached/
* [op instance]/
* /[nodes that have completed]
* /abort/
* [op instance] - failure data
*
* NOTE: while acquired and completed are znode dirs, abort is actually just a znode.
*
* Assumption here that procedure names are unique
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public abstract class ZKProcedureUtil
extends ZooKeeperListener implements Closeable {
private static final Log LOG = LogFactory.getLog(ZKProcedureUtil.class);
public static final String ACQUIRED_BARRIER_ZNODE_DEFAULT = "acquired";
public static final String REACHED_BARRIER_ZNODE_DEFAULT = "reached";
public static final String ABORT_ZNODE_DEFAULT = "abort";
public final String baseZNode;
protected final String acquiredZnode;
protected final String reachedZnode;
protected final String abortZnode;
protected final String memberName;
/**
* Top-level watcher/controller for procedures across the cluster.
* <p>
* On instantiation, this ensures the procedure znodes exist. This however requires the passed in
* watcher has been started.
* @param watcher watcher for the cluster ZK. Owned by <tt>this</tt> and closed via
* {@link #close()}
* @param procDescription name of the znode describing the procedure to run
* @param memberName name of the member from which we are interacting with running procedures
* @throws KeeperException when the procedure znodes cannot be created
*/
public ZKProcedureUtil(ZooKeeperWatcher watcher, String procDescription,
String memberName) throws KeeperException {
super(watcher);
this.memberName = memberName;
// make sure we are listening for events
watcher.registerListener(this);
// setup paths for the zknodes used in procedures
this.baseZNode = ZKUtil.joinZNode(watcher.baseZNode, procDescription);
acquiredZnode = ZKUtil.joinZNode(baseZNode, ACQUIRED_BARRIER_ZNODE_DEFAULT);
reachedZnode = ZKUtil.joinZNode(baseZNode, REACHED_BARRIER_ZNODE_DEFAULT);
abortZnode = ZKUtil.joinZNode(baseZNode, ABORT_ZNODE_DEFAULT);
// first make sure all the ZK nodes exist
// make sure all the parents exist (sometimes not the case in tests)
ZKUtil.createWithParents(watcher, acquiredZnode);
// regular create because all the parents exist
ZKUtil.createAndFailSilent(watcher, reachedZnode);
ZKUtil.createAndFailSilent(watcher, abortZnode);
}
@Override
public void close() throws IOException {
// the watcher is passed from either Master or Region Server
// watcher.close() will be called by the owner so no need to call close() here
}
public String getAcquiredBarrierNode(String opInstanceName) {
return ZKProcedureUtil.getAcquireBarrierNode(this, opInstanceName);
}
public String getReachedBarrierNode(String opInstanceName) {
return ZKProcedureUtil.getReachedBarrierNode(this, opInstanceName);
}
public String getAbortZNode(String opInstanceName) {
return ZKProcedureUtil.getAbortNode(this, opInstanceName);
}
public String getAbortZnode() {
return abortZnode;
}
public String getBaseZnode() {
return baseZNode;
}
public String getAcquiredBarrier() {
return acquiredZnode;
}
public String getMemberName() {
return memberName;
}
/**
* Get the full znode path for the node used by the coordinator to trigger a global barrier
* acquire on each subprocedure.
* @param controller controller running the procedure
* @param opInstanceName name of the running procedure instance (not the procedure description).
* @return full znode path to the prepare barrier/start node
*/
public static String getAcquireBarrierNode(ZKProcedureUtil controller,
String opInstanceName) {
return ZKUtil.joinZNode(controller.acquiredZnode, opInstanceName);
}
/**
* Get the full znode path for the node used by the coordinator to trigger a global barrier
* execution and release on each subprocedure.
* @param controller controller running the procedure
* @param opInstanceName name of the running procedure instance (not the procedure description).
* @return full znode path to the commit barrier
*/
public static String getReachedBarrierNode(ZKProcedureUtil controller,
String opInstanceName) {
return ZKUtil.joinZNode(controller.reachedZnode, opInstanceName);
}
/**
* Get the full znode path for the node used by the coordinator or member to trigger an abort
* of the global barrier acquisition or execution in subprocedures.
* @param controller controller running the procedure
* @param opInstanceName name of the running procedure instance (not the procedure description).
* @return full znode path to the abort znode
*/
public static String getAbortNode(ZKProcedureUtil controller, String opInstanceName) {
return ZKUtil.joinZNode(controller.abortZnode, opInstanceName);
}
public ZooKeeperWatcher getWatcher() {
return watcher;
}
/**
* Is this a procedure related znode path?
*
* TODO: this is not strict, can return true if had name just starts with same prefix but is
* different zdir.
*
* @return true if starts with baseZnode
*/
boolean isInProcedurePath(String path) {
return path.startsWith(baseZNode);
}
/**
* Is this the exact procedure barrier acquired znode
*/
boolean isAcquiredNode(String path) {
return path.equals(acquiredZnode);
}
/**
* Is this in the procedure barrier acquired znode path
*/
boolean isAcquiredPathNode(String path) {
return path.startsWith(this.acquiredZnode) && !path.equals(acquiredZnode);
}
/**
* Is this the exact procedure barrier reached znode
*/
boolean isReachedNode(String path) {
return path.equals(reachedZnode);
}
/**
* Is this in the procedure barrier reached znode path
*/
boolean isReachedPathNode(String path) {
return path.startsWith(this.reachedZnode) && !path.equals(reachedZnode);
}
/**
* Is this in the procedure barrier abort znode path
*/
boolean isAbortNode(String path) {
return path.equals(abortZnode);
}
/**
* Is this in the procedure barrier abort znode path
*/
public boolean isAbortPathNode(String path) {
return path.startsWith(this.abortZnode) && !path.equals(abortZnode);
}
// --------------------------------------------------------------------------
// internal debugging methods
// --------------------------------------------------------------------------
/**
* Recursively print the current state of ZK (non-transactional)
* @param root name of the root directory in zk to print
* @throws KeeperException
*/
void logZKTree(String root) {
if (!LOG.isDebugEnabled()) return;
LOG.debug("Current zk system:");
String prefix = "|-";
LOG.debug(prefix + root);
try {
logZKTree(root, prefix);
} catch (KeeperException e) {
throw new RuntimeException(e);
}
}
/**
* Helper method to print the current state of the ZK tree.
* @see #logZKTree(String)
* @throws KeeperException if an unexpected exception occurs
*/
protected void logZKTree(String root, String prefix) throws KeeperException {
List<String> children = ZKUtil.listChildrenNoWatch(watcher, root);
if (children == null) return;
for (String child : children) {
LOG.debug(prefix + child);
String node = ZKUtil.joinZNode(root.equals("/") ? "" : root, child);
logZKTree(node, prefix + "---");
}
}
public void clearChildZNodes() throws KeeperException {
// TODO This is potentially racy since not atomic. update when we support zk that has multi
LOG.info("Clearing all procedure znodes: " + acquiredZnode + " " + reachedZnode + " "
+ abortZnode);
// If the coordinator was shutdown mid-procedure, then we are going to lose
// an procedure that was previously started by cleaning out all the previous state. Its much
// harder to figure out how to keep an procedure going and the subject of HBASE-5487.
ZKUtil.deleteChildrenRecursively(watcher, acquiredZnode);
ZKUtil.deleteChildrenRecursively(watcher, reachedZnode);
ZKUtil.deleteChildrenRecursively(watcher, abortZnode);
}
public void clearZNodes(String procedureName) throws KeeperException {
// TODO This is potentially racy since not atomic. update when we support zk that has multi
LOG.info("Clearing all znodes for procedure " + procedureName + "including nodes "
+ acquiredZnode + " " + reachedZnode + " " + abortZnode);
ZKUtil.deleteNodeRecursively(watcher, getAcquiredBarrierNode(procedureName));
ZKUtil.deleteNodeRecursively(watcher, getReachedBarrierNode(procedureName));
ZKUtil.deleteNodeRecursively(watcher, getAbortZNode(procedureName));
}
}

View File

@ -970,7 +970,7 @@ public final class ProtobufUtil {
* Convert a delete KeyValue type to protocol buffer DeleteType.
*
* @param type
* @return a DeleteType
* @return protocol buffer DeleteType
* @throws IOException
*/
public static DeleteType toDeleteType(

View File

@ -97,6 +97,7 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Row;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.Filter;
@ -113,6 +114,7 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl.WriteEntry;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
@ -120,6 +122,7 @@ import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.ClassSize;
@ -134,7 +137,6 @@ import org.apache.hadoop.util.StringUtils;
import org.cliffc.high_scale_lib.Counter;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.io.Closeables;
@ -2450,8 +2452,71 @@ public class HRegion implements HeapSize { // , Writable{
}
/**
* Replaces any KV timestamps set to {@link HConstants#LATEST_TIMESTAMP}
* with the provided current timestamp.
* Complete taking the snapshot on the region. Writes the region info and adds references to the
* working snapshot directory.
*
* TODO for api consistency, consider adding another version with no {@link ForeignExceptionSnare}
* arg. (In the future other cancellable HRegion methods could eventually add a
* {@link ForeignExceptionSnare}, or we could do something fancier).
*
* @param desc snasphot description object
* @param exnSnare ForeignExceptionSnare that captures external exeptions in case we need to
* bail out. This is allowed to be null and will just be ignored in that case.
* @throws IOException if there is an external or internal error causing the snapshot to fail
*/
public void addRegionToSnapshot(SnapshotDescription desc,
ForeignExceptionSnare exnSnare) throws IOException {
// This should be "fast" since we don't rewrite store files but instead
// back up the store files by creating a reference
Path rootDir = FSUtils.getRootDir(this.rsServices.getConfiguration());
Path snapshotRegionDir = TakeSnapshotUtils.getRegionSnapshotDirectory(desc, rootDir,
regionInfo.getEncodedName());
// 1. dump region meta info into the snapshot directory
LOG.debug("Storing region-info for snapshot.");
checkRegioninfoOnFilesystem(snapshotRegionDir);
// 2. iterate through all the stores in the region
LOG.debug("Creating references for hfiles");
// This ensures that we have an atomic view of the directory as long as we have < ls limit
// (batch size of the files in a directory) on the namenode. Otherwise, we get back the files in
// batches and may miss files being added/deleted. This could be more robust (iteratively
// checking to see if we have all the files until we are sure), but the limit is currently 1000
// files/batch, far more than the number of store files under a single column family.
for (Store store : stores.values()) {
// 2.1. build the snapshot reference directory for the store
Path dstStoreDir = TakeSnapshotUtils.getStoreSnapshotDirectory(snapshotRegionDir,
Bytes.toString(store.getFamily().getName()));
List<StoreFile> storeFiles = new ArrayList<StoreFile>(store.getStorefiles());
if (LOG.isDebugEnabled()) {
LOG.debug("Adding snapshot references for " + storeFiles + " hfiles");
}
// 2.2. iterate through all the store's files and create "references".
int sz = storeFiles.size();
for (int i = 0; i < sz; i++) {
if (exnSnare != null) {
exnSnare.rethrowException();
}
Path file = storeFiles.get(i).getPath();
// create "reference" to this store file. It is intentionally an empty file -- all
// necessary infomration is captured by its fs location and filename. This allows us to
// only figure out what needs to be done via a single nn operation (instead of having to
// open and read the files as well).
LOG.debug("Creating reference for file (" + (i+1) + "/" + sz + ") : " + file);
Path referenceFile = new Path(dstStoreDir, file.getName());
boolean success = fs.createNewFile(referenceFile);
if (!success) {
throw new IOException("Failed to create reference file:" + referenceFile);
}
}
}
}
/**
* Replaces any KV timestamps set to {@link HConstants#LATEST_TIMESTAMP} with the provided current
* timestamp.
*/
void updateKVTimestamps(
final Iterable<List<KeyValue>> keyLists, final byte[] now) {
@ -4373,10 +4438,10 @@ public class HRegion implements HeapSize { // , Writable{
}
// delete out the 'A' region
HFileArchiver.archiveRegion(a.getBaseConf(), fs,
HFileArchiver.archiveRegion(fs,
FSUtils.getRootDir(a.getBaseConf()), a.getTableDir(), a.getRegionDir());
// delete out the 'B' region
HFileArchiver.archiveRegion(a.getBaseConf(), fs,
HFileArchiver.archiveRegion(fs,
FSUtils.getRootDir(b.getBaseConf()), b.getTableDir(), b.getRegionDir());
LOG.info("merge completed. New region is " + dstRegion);

View File

@ -178,6 +178,7 @@ import org.apache.hadoop.hbase.regionserver.handler.CloseRootHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenRootHandler;
import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
@ -425,6 +426,9 @@ public class HRegionServer implements ClientProtocol,
private RegionServerCoprocessorHost rsHost;
/** Handle all the snapshot requests to this server */
RegionServerSnapshotManager snapshotManager;
/**
* Starts a HRegionServer at the default location
*
@ -763,6 +767,13 @@ public class HRegionServer implements ClientProtocol,
} catch (KeeperException e) {
this.abort("Failed to retrieve Cluster ID",e);
}
// watch for snapshots
try {
this.snapshotManager = new RegionServerSnapshotManager(this);
} catch (KeeperException e) {
this.abort("Failed to reach zk cluster when creating snapshot handler.");
}
}
/**
@ -849,6 +860,9 @@ public class HRegionServer implements ClientProtocol,
}
}
// start the snapshot handler, since the server is ready to run
this.snapshotManager.start();
// We registered with the Master. Go into run mode.
long lastMsg = 0;
long oldRequestCount = -1;
@ -930,6 +944,12 @@ public class HRegionServer implements ClientProtocol,
this.healthCheckChore.interrupt();
}
try {
if (snapshotManager != null) snapshotManager.stop(this.abortRequested);
} catch (IOException e) {
LOG.warn("Failed to close snapshot handler cleanly", e);
}
if (this.killed) {
// Just skip out w/o closing regions. Used when testing.
} else if (abortRequested) {
@ -946,6 +966,13 @@ public class HRegionServer implements ClientProtocol,
// handlers are stuck waiting on meta or root.
if (this.catalogTracker != null) this.catalogTracker.stop();
// stop the snapshot handler, forcefully killing all running tasks
try {
if (snapshotManager != null) snapshotManager.stop(this.abortRequested || this.killed);
} catch (IOException e) {
LOG.warn("Failed to close snapshot handler cleanly", e);
}
// Closing the compactSplit thread before closing meta regions
if (!this.killed && containsMetaTableRegions()) {
if (!abortRequested || this.fsOk) {
@ -3745,7 +3772,8 @@ public class HRegionServer implements ClientProtocol,
*
* @param region
* @param mutate
* @return the Result
* @return result to return to client if default operation should be
* bypassed as indicated by RegionObserver, null otherwise
* @throws IOException
*/
protected Result append(final HRegion region,

View File

@ -46,24 +46,24 @@ import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.HalfStoreFileReader;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
import org.apache.hadoop.hbase.util.ChecksumType;
import org.apache.hadoop.hbase.util.BloomFilter;
import org.apache.hadoop.hbase.util.BloomFilterFactory;
import org.apache.hadoop.hbase.util.BloomFilterWriter;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ChecksumType;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.RawComparator;
@ -183,13 +183,26 @@ public class StoreFile {
*/
private Map<byte[], byte[]> metadataMap;
/*
* Regex that will work for straight filenames and for reference names.
* If reference, then the regex has more than just one group. Group 1 is
* this files id. Group 2 the referenced region name, etc.
/**
* A non-capture group, for hfiles, so that this can be embedded.
* HFiles are uuid ([0-9a-z]+). Bulk loaded hfiles has (_SeqId_[0-9]+_) has suffix.
*/
private static final Pattern REF_NAME_PARSER =
Pattern.compile("^([0-9a-f]+(?:_SeqId_[0-9]+_)?)(?:\\.(.+))?$");
public static final String HFILE_NAME_REGEX = "[0-9a-f]+(?:_SeqId_[0-9]+_)?";
/** Regex that will work for hfiles */
private static final Pattern HFILE_NAME_PATTERN =
Pattern.compile("^(" + HFILE_NAME_REGEX + ")");
/**
* Regex that will work for straight reference names (<hfile>.<parentEncRegion>)
* and hfilelink reference names (<table>=<region>-<hfile>.<parentEncRegion>)
* If reference, then the regex has more than just one group.
* Group 1, hfile/hfilelink pattern, is this file's id.
* Group 2 '(.+)' is the reference's parent region name.
*/
private static final Pattern REF_NAME_PATTERN =
Pattern.compile(String.format("^(%s|%s)\\.(.+)$",
HFILE_NAME_REGEX, HFileLink.LINK_NAME_REGEX));
// StoreFile.Reader
private volatile Reader reader;
@ -239,7 +252,13 @@ public class StoreFile {
} else if (isReference(p)) {
this.reference = Reference.read(fs, p);
this.referencePath = getReferredToFile(this.path);
LOG.debug("Store file " + p + " is a reference");
if (HFileLink.isHFileLink(this.referencePath)) {
this.link = new HFileLink(conf, this.referencePath);
}
LOG.debug("Store file " + p + " is a " + reference.getFileRegion() +
" reference to " + this.referencePath);
} else if (!isHFile(p)) {
throw new IOException("path=" + path + " doesn't look like a valid StoreFile");
}
if (BloomFilterFactory.isGeneralBloomEnabled(conf)) {
@ -257,7 +276,6 @@ public class StoreFile {
} else {
this.modificationTimeStamp = 0;
}
}
/**
@ -286,7 +304,12 @@ public class StoreFile {
* @return <tt>true</tt> if this StoreFile is an HFileLink
*/
boolean isLink() {
return this.link != null;
return this.link != null && this.reference == null;
}
private static boolean isHFile(final Path path) {
Matcher m = HFILE_NAME_PATTERN.matcher(path.getName());
return m.matches() && m.groupCount() > 0;
}
/**
@ -294,22 +317,16 @@ public class StoreFile {
* @return True if the path has format of a HStoreFile reference.
*/
public static boolean isReference(final Path p) {
return !p.getName().startsWith("_") &&
isReference(p, REF_NAME_PARSER.matcher(p.getName()));
return isReference(p.getName());
}
/**
* @param p Path to check.
* @param m Matcher to use.
* @param name file name to check.
* @return True if the path has format of a HStoreFile reference.
*/
public static boolean isReference(final Path p, final Matcher m) {
if (m == null || !m.matches()) {
LOG.warn("Failed match of store file name " + p.toString());
throw new RuntimeException("Failed match of store file name " +
p.toString());
}
return m.groupCount() > 1 && m.group(2) != null;
public static boolean isReference(final String name) {
Matcher m = REF_NAME_PATTERN.matcher(name);
return m.matches() && m.groupCount() > 1;
}
/*
@ -317,20 +334,23 @@ public class StoreFile {
* hierarchy of <code>${hbase.rootdir}/tablename/regionname/familyname</code>.
* @param p Path to a Reference file.
* @return Calculated path to parent region file.
* @throws IOException
* @throws IllegalArgumentException when path regex fails to match.
*/
public static Path getReferredToFile(final Path p) {
Matcher m = REF_NAME_PARSER.matcher(p.getName());
Matcher m = REF_NAME_PATTERN.matcher(p.getName());
if (m == null || !m.matches()) {
LOG.warn("Failed match of store file name " + p.toString());
throw new RuntimeException("Failed match of store file name " +
throw new IllegalArgumentException("Failed match of store file name " +
p.toString());
}
// Other region name is suffix on the passed Reference file name
String otherRegion = m.group(2);
// Tabledir is up two directories from where Reference was written.
Path tableDir = p.getParent().getParent().getParent();
String nameStrippedOfSuffix = m.group(1);
LOG.debug("reference '" + p + "' to region=" + otherRegion + " hfile=" + nameStrippedOfSuffix);
// Build up new path with the referenced region in place of our current
// region in the reference path. Also strip regionname suffix from name.
return new Path(new Path(new Path(tableDir, otherRegion),
@ -437,16 +457,15 @@ public class StoreFile {
* If this estimate isn't good enough, we can improve it later.
* @param fs The FileSystem
* @param reference The reference
* @param reference The referencePath
* @param status The reference FileStatus
* @return HDFS blocks distribution
*/
static private HDFSBlocksDistribution computeRefFileHDFSBlockDistribution(
FileSystem fs, Reference reference, Path referencePath) throws IOException {
if ( referencePath == null) {
FileSystem fs, Reference reference, FileStatus status) throws IOException {
if (status == null) {
return null;
}
FileStatus status = fs.getFileStatus(referencePath);
long start = 0;
long length = 0;
@ -460,35 +479,19 @@ public class StoreFile {
return FSUtils.computeHDFSBlocksDistribution(fs, status, start, length);
}
/**
* helper function to compute HDFS blocks distribution of a given file.
* For reference file, it is an estimate
* @param fs The FileSystem
* @param p The path of the file
* @return HDFS blocks distribution
*/
static public HDFSBlocksDistribution computeHDFSBlockDistribution(
FileSystem fs, Path p) throws IOException {
if (isReference(p)) {
Reference reference = Reference.read(fs, p);
Path referencePath = getReferredToFile(p);
return computeRefFileHDFSBlockDistribution(fs, reference, referencePath);
} else {
if (HFileLink.isHFileLink(p)) p = HFileLink.getReferencedPath(fs, p);
FileStatus status = fs.getFileStatus(p);
long length = status.getLen();
return FSUtils.computeHDFSBlocksDistribution(fs, status, 0, length);
}
}
/**
* compute HDFS block distribution, for reference file, it is an estimate
*/
private void computeHDFSBlockDistribution() throws IOException {
if (isReference()) {
FileStatus status;
if (this.link != null) {
status = this.link.getFileStatus(fs);
} else {
status = fs.getFileStatus(this.referencePath);
}
this.hdfsBlocksDistribution = computeRefFileHDFSBlockDistribution(
this.fs, this.reference, this.referencePath);
this.fs, this.reference, status);
} else {
FileStatus status;
if (isLink()) {
@ -513,13 +516,17 @@ public class StoreFile {
throw new IllegalAccessError("Already open");
}
if (isReference()) {
this.reader = new HalfStoreFileReader(this.fs, this.referencePath,
this.cacheConf, this.reference,
dataBlockEncoder.getEncodingInCache());
if (this.link != null) {
this.reader = new HalfStoreFileReader(this.fs, this.referencePath, this.link,
this.cacheConf, this.reference, dataBlockEncoder.getEncodingInCache());
} else {
this.reader = new HalfStoreFileReader(this.fs, this.referencePath,
this.cacheConf, this.reference, dataBlockEncoder.getEncodingInCache());
}
} else if (isLink()) {
long size = link.getFileStatus(fs).getLen();
this.reader = new Reader(this.fs, this.path, link, size, this.cacheConf,
dataBlockEncoder.getEncodingInCache(), true);
dataBlockEncoder.getEncodingInCache(), true);
} else {
this.reader = new Reader(this.fs, this.path, this.cacheConf,
dataBlockEncoder.getEncodingInCache());
@ -901,6 +908,8 @@ public class StoreFile {
public static boolean validateStoreFileName(String fileName) {
if (HFileLink.isHFileLink(fileName))
return true;
if (isReference(fileName))
return true;
return !fileName.contains("-");
}
@ -926,7 +935,7 @@ public class StoreFile {
Reference r =
top? Reference.createTopReference(splitRow): Reference.createBottomReference(splitRow);
// Add the referred-to regions name as a dot separated suffix.
// See REF_NAME_PARSER regex above. The referred-to regions name is
// See REF_NAME_REGEX regex above. The referred-to regions name is
// up in the path of the passed in <code>f</code> -- parentdir is family,
// then the directory above is the region name.
String parentRegionName = f.getPath().getParent().getParent().getName();

View File

@ -0,0 +1,161 @@
/**
* 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.regionserver.snapshot;
import java.util.List;
import java.util.concurrent.Callable;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.procedure.ProcedureMember;
import org.apache.hadoop.hbase.procedure.Subprocedure;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager.SnapshotSubprocedurePool;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
/**
* This online snapshot implementation uses the distributed procedure framework to force a
* store flush and then records the hfiles. Its enter stage does nothing. Its leave stage then
* flushes the memstore, builds the region server's snapshot manifest from its hfiles list, and
* copies .regioninfos into the snapshot working directory. At the master side, there is an atomic
* rename of the working dir into the proper snapshot directory.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class FlushSnapshotSubprocedure extends Subprocedure {
private static final Log LOG = LogFactory.getLog(FlushSnapshotSubprocedure.class);
private final List<HRegion> regions;
private final SnapshotDescription snapshot;
private final SnapshotSubprocedurePool taskManager;
public FlushSnapshotSubprocedure(ProcedureMember member,
ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout,
List<HRegion> regions, SnapshotDescription snapshot,
SnapshotSubprocedurePool taskManager) {
super(member, snapshot.getName(), errorListener, wakeFrequency, timeout);
this.snapshot = snapshot;
this.regions = regions;
this.taskManager = taskManager;
}
/**
* Callable for adding files to snapshot manifest working dir. Ready for multithreading.
*/
private class RegionSnapshotTask implements Callable<Void> {
HRegion region;
RegionSnapshotTask(HRegion region) {
this.region = region;
}
@Override
public Void call() throws Exception {
// Taking the region read lock prevents the individual region from being closed while a
// snapshot is in progress. This is helpful but not sufficient for preventing races with
// snapshots that involve multiple regions and regionservers. It is still possible to have
// an interleaving such that globally regions are missing, so we still need the verification
// step.
LOG.debug("Starting region operation on " + region);
region.startRegionOperation();
try {
LOG.debug("Flush Snapshotting region " + region.toString() + " started...");
region.flushcache();
region.addRegionToSnapshot(snapshot, monitor);
LOG.debug("... Flush Snapshotting region " + region.toString() + " completed.");
} finally {
LOG.debug("Closing region operation on " + region);
region.closeRegionOperation();
}
return null;
}
}
private void flushSnapshot() throws ForeignException {
if (regions.isEmpty()) {
// No regions on this RS, we are basically done.
return;
}
monitor.rethrowException();
// assert that the taskManager is empty.
if (taskManager.hasTasks()) {
throw new IllegalStateException("Attempting to take snapshot "
+ SnapshotDescriptionUtils.toString(snapshot)
+ " but we currently have outstanding tasks");
}
// Add all hfiles already existing in region.
for (HRegion region : regions) {
// submit one task per region for parallelize by region.
taskManager.submitTask(new RegionSnapshotTask(region));
monitor.rethrowException();
}
// wait for everything to complete.
LOG.debug("Flush Snapshot Tasks submitted for " + regions.size() + " regions");
try {
taskManager.waitForOutstandingTasks();
} catch (InterruptedException e) {
throw new ForeignException(getMemberName(), e);
}
}
/**
* do nothing, core of snapshot is executed in {@link #insideBarrier} step.
*/
@Override
public void acquireBarrier() throws ForeignException {
// NO OP
}
/**
* do a flush snapshot of every region on this rs from the target table.
*/
@Override
public void insideBarrier() throws ForeignException {
flushSnapshot();
}
/**
* Cancel threads if they haven't finished.
*/
@Override
public void cleanup(Exception e) {
LOG.info("Aborting all online FLUSH snapshot subprocedure task threads for '"
+ snapshot.getName() + "' due to error", e);
try {
taskManager.cancelTasks();
} catch (InterruptedException e1) {
Thread.currentThread().interrupt();
}
}
/**
* Hooray!
*/
public void releaseBarrier() {
// NO OP
}
}

View File

@ -0,0 +1,377 @@
/**
* 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.regionserver.snapshot;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.DaemonThreadFactory;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.master.snapshot.MasterSnapshotVerifier;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.procedure.ProcedureMember;
import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
import org.apache.hadoop.hbase.procedure.Subprocedure;
import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* This manager class handles the work dealing with snapshots for a {@link HRegionServer}.
* <p>
* This provides the mechanism necessary to kick off a online snapshot specific
* {@link Subprocedure} that is responsible for the regions being served by this region server.
* If any failures occur with the subprocedure, the RegionSeverSnapshotManager's subprocedure
* handler, {@link ProcedureMember}, notifies the master's ProcedureCoordinator to abort all
* others.
* <p>
* On startup, requires {@link #start()} to be called.
* <p>
* On shutdown, requires {@link #stop(boolean)} to be called
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class RegionServerSnapshotManager {
private static final Log LOG = LogFactory.getLog(RegionServerSnapshotManager.class);
/** Maximum number of snapshot region tasks that can run concurrently */
private static final String CONCURENT_SNAPSHOT_TASKS_KEY = "hbase.snapshot.region.concurrentTasks";
private static final int DEFAULT_CONCURRENT_SNAPSHOT_TASKS = 3;
/** Conf key for number of request threads to start snapshots on regionservers */
public static final String SNAPSHOT_REQUEST_THREADS_KEY = "hbase.snapshot.region.pool.threads";
/** # of threads for snapshotting regions on the rs. */
public static final int SNAPSHOT_REQUEST_THREADS_DEFAULT = 10;
/** Conf key for max time to keep threads in snapshot request pool waiting */
public static final String SNAPSHOT_TIMEOUT_MILLIS_KEY = "hbase.snapshot.region.timeout";
/** Keep threads alive in request pool for max of 60 seconds */
public static final long SNAPSHOT_TIMEOUT_MILLIS_DEFAULT = 60000;
/** Conf key for millis between checks to see if snapshot completed or if there are errors*/
public static final String SNAPSHOT_REQUEST_WAKE_MILLIS_KEY = "hbase.snapshot.region.wakefrequency";
/** Default amount of time to check for errors while regions finish snapshotting */
private static final long SNAPSHOT_REQUEST_WAKE_MILLIS_DEFAULT = 500;
private final RegionServerServices rss;
private final ProcedureMemberRpcs memberRpcs;
private final ProcedureMember member;
/**
* Exposed for testing.
* @param conf HBase configuration.
* @param parent parent running the snapshot handler
* @param memberRpc use specified memberRpc instance
* @param procMember use specified ProcedureMember
*/
RegionServerSnapshotManager(Configuration conf, HRegionServer parent,
ProcedureMemberRpcs memberRpc, ProcedureMember procMember) {
this.rss = parent;
this.memberRpcs = memberRpc;
this.member = procMember;
}
/**
* Create a default snapshot handler - uses a zookeeper based member controller.
* @param rss region server running the handler
* @throws KeeperException if the zookeeper cluster cannot be reached
*/
public RegionServerSnapshotManager(RegionServerServices rss)
throws KeeperException {
this.rss = rss;
ZooKeeperWatcher zkw = rss.getZooKeeper();
String nodeName = rss.getServerName().toString();
this.memberRpcs = new ZKProcedureMemberRpcs(zkw,
SnapshotManager.ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION, nodeName);
// read in the snapshot request configuration properties
Configuration conf = rss.getConfiguration();
long wakeMillis = conf.getLong(SNAPSHOT_REQUEST_WAKE_MILLIS_KEY, SNAPSHOT_REQUEST_WAKE_MILLIS_DEFAULT);
long keepAlive = conf.getLong(SNAPSHOT_TIMEOUT_MILLIS_KEY, SNAPSHOT_TIMEOUT_MILLIS_DEFAULT);
int opThreads = conf.getInt(SNAPSHOT_REQUEST_THREADS_KEY, SNAPSHOT_REQUEST_THREADS_DEFAULT);
// create the actual snapshot procedure member
ThreadPoolExecutor pool = ProcedureMember.defaultPool(wakeMillis, keepAlive, opThreads, nodeName);
this.member = new ProcedureMember(memberRpcs, pool, new SnapshotSubprocedureBuilder());
}
/**
* Start accepting snapshot requests.
*/
public void start() {
this.memberRpcs.start(member);
}
/**
* Close <tt>this</tt> and all running snapshot tasks
* @param force forcefully stop all running tasks
* @throws IOException
*/
public void stop(boolean force) throws IOException {
String mode = force ? "abruptly" : "gracefully";
LOG.info("Stopping RegionServerSnapshotManager " + mode + ".");
try {
this.member.close();
} finally {
this.memberRpcs.close();
}
}
/**
* If in a running state, creates the specified subprocedure for handling an online snapshot.
*
* Because this gets the local list of regions to snapshot and not the set the master had,
* there is a possibility of a race where regions may be missed. This detected by the master in
* the snapshot verification step.
*
* @param snapshot
* @return Subprocedure to submit to the ProcedureMemeber.
*/
public Subprocedure buildSubprocedure(SnapshotDescription snapshot) {
// don't run a snapshot if the parent is stop(ping)
if (rss.isStopping() || rss.isStopped()) {
throw new IllegalStateException("Can't start snapshot on RS: " + rss.getServerName()
+ ", because stopping/stopped!");
}
// check to see if this server is hosting any regions for the snapshots
// check to see if we have regions for the snapshot
List<HRegion> involvedRegions;
try {
involvedRegions = getRegionsToSnapshot(snapshot);
} catch (IOException e1) {
throw new IllegalStateException("Failed to figure out if we should handle a snapshot - "
+ "something has gone awry with the online regions.", e1);
}
// We need to run the subprocedure even if we have no relevant regions. The coordinator
// expects participation in the procedure and without sending message the snapshot attempt
// will hang and fail.
LOG.debug("Launching subprocedure for snapshot " + snapshot.getName() + " from table "
+ snapshot.getTable());
ForeignExceptionDispatcher exnDispatcher = new ForeignExceptionDispatcher();
Configuration conf = rss.getConfiguration();
long timeoutMillis = conf.getLong(SNAPSHOT_TIMEOUT_MILLIS_KEY,
SNAPSHOT_TIMEOUT_MILLIS_DEFAULT);
long wakeMillis = conf.getLong(SNAPSHOT_REQUEST_WAKE_MILLIS_KEY,
SNAPSHOT_REQUEST_WAKE_MILLIS_DEFAULT);
switch (snapshot.getType()) {
case FLUSH:
SnapshotSubprocedurePool taskManager =
new SnapshotSubprocedurePool(rss.getServerName().toString(), conf);
return new FlushSnapshotSubprocedure(member, exnDispatcher, wakeMillis,
timeoutMillis, involvedRegions, snapshot, taskManager);
default:
throw new UnsupportedOperationException("Unrecognized snapshot type:" + snapshot.getType());
}
}
/**
* Determine if the snapshot should be handled on this server
*
* NOTE: This is racy -- the master expects a list of regionservers.
* This means if a region moves somewhere between the calls we'll miss some regions.
* For example, a region move during a snapshot could result in a region to be skipped or done
* twice. This is manageable because the {@link MasterSnapshotVerifier} will double check the
* region lists after the online portion of the snapshot completes and will explicitly fail the
* snapshot.
*
* @param snapshot
* @return the list of online regions. Empty list is returned if no regions are responsible for
* the given snapshot.
* @throws IOException
*/
private List<HRegion> getRegionsToSnapshot(SnapshotDescription snapshot) throws IOException {
byte[] table = Bytes.toBytes(snapshot.getTable());
return rss.getOnlineRegions(table);
}
/**
* Build the actual snapshot runner that will do all the 'hard' work
*/
public class SnapshotSubprocedureBuilder implements SubprocedureFactory {
@Override
public Subprocedure buildSubprocedure(String name, byte[] data) {
try {
// unwrap the snapshot information
SnapshotDescription snapshot = SnapshotDescription.parseFrom(data);
return RegionServerSnapshotManager.this.buildSubprocedure(snapshot);
} catch (InvalidProtocolBufferException e) {
throw new IllegalArgumentException("Could not read snapshot information from request.");
}
}
}
/**
* We use the SnapshotSubprocedurePool, a class specific thread pool instead of
* {@link org.apache.hadoop.hbase.executor.ExecutorService}.
*
* It uses a {@link java.util.concurrent.ExecutorCompletionService} which provides queuing of
* completed tasks which lets us efficiently cancel pending tasks upon the earliest operation
* failures.
*
* HBase's ExecutorService (different from {@link java.util.concurrent.ExecutorService}) isn't
* really built for coordinated tasks where multiple threads as part of one larger task. In
* RS's the HBase Executor services are only used for open and close and not other threadpooled
* operations such as compactions and replication sinks.
*/
static class SnapshotSubprocedurePool {
private final ExecutorCompletionService<Void> taskPool;
private final ThreadPoolExecutor executor;
private volatile boolean stopped;
private final List<Future<Void>> futures = new ArrayList<Future<Void>>();
private final String name;
SnapshotSubprocedurePool(String name, Configuration conf) {
// configure the executor service
long keepAlive = conf.getLong(
RegionServerSnapshotManager.SNAPSHOT_TIMEOUT_MILLIS_KEY,
RegionServerSnapshotManager.SNAPSHOT_TIMEOUT_MILLIS_DEFAULT);
int threads = conf.getInt(CONCURENT_SNAPSHOT_TASKS_KEY, DEFAULT_CONCURRENT_SNAPSHOT_TASKS);
this.name = name;
executor = new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.MILLISECONDS,
new LinkedBlockingQueue<Runnable>(), new DaemonThreadFactory("rs("
+ name + ")-snapshot-pool"));
taskPool = new ExecutorCompletionService<Void>(executor);
}
boolean hasTasks() {
return futures.size() != 0;
}
/**
* Submit a task to the pool.
*
* NOTE: all must be submitted before you can safely {@link #waitForOutstandingTasks()}. This
* version does not support issuing tasks from multiple concurrent table snapshots requests.
*/
void submitTask(final Callable<Void> task) {
Future<Void> f = this.taskPool.submit(task);
futures.add(f);
}
/**
* Wait for all of the currently outstanding tasks submitted via {@link #submitTask(Callable)}.
* This *must* be called after all tasks are submitted via submitTask.
*
* @return <tt>true</tt> on success, <tt>false</tt> otherwise
* @throws InterruptedException
* @throws SnapshotCreationException if the snapshot failed while we were waiting
*/
boolean waitForOutstandingTasks() throws ForeignException, InterruptedException {
LOG.debug("Waiting for local region snapshots to finish.");
int sz = futures.size();
try {
// Using the completion service to process the futures that finish first first.
for (int i = 0; i < sz; i++) {
Future<Void> f = taskPool.take();
f.get();
if (!futures.remove(f)) {
LOG.warn("unexpected future" + f);
}
LOG.debug("Completed " + (i+1) + "/" + sz + " local region snapshots.");
}
LOG.debug("Completed " + sz + " local region snapshots.");
return true;
} catch (InterruptedException e) {
LOG.warn("Got InterruptedException in SnapshotSubprocedurePool", e);
if (!stopped) {
Thread.currentThread().interrupt();
throw new ForeignException("SnapshotSubprocedurePool", e);
}
// we are stopped so we can just exit.
} catch (ExecutionException e) {
if (e.getCause() instanceof ForeignException) {
LOG.warn("Rethrowing ForeignException from SnapshotSubprocedurePool", e);
throw (ForeignException)e.getCause();
}
LOG.warn("Got Exception in SnapshotSubprocedurePool", e);
throw new ForeignException(name, e.getCause());
} finally {
cancelTasks();
}
return false;
}
/**
* This attempts to cancel out all pending and in progress tasks (interruptions issues)
* @throws InterruptedException
*/
void cancelTasks() throws InterruptedException {
Collection<Future<Void>> tasks = futures;
LOG.debug("cancelling " + tasks.size() + " tasks for snapshot " + name);
for (Future<Void> f: tasks) {
// TODO Ideally we'd interrupt hbase threads when we cancel. However it seems that there
// are places in the HBase code where row/region locks are taken and not released in a
// finally block. Thus we cancel without interrupting. Cancellations will be slower to
// complete but we won't suffer from unreleased locks due to poor code discipline.
f.cancel(false);
}
// evict remaining tasks and futures from taskPool.
LOG.debug(taskPool);
while (!futures.isEmpty()) {
// block to remove cancelled futures;
LOG.warn("Removing cancelled elements from taskPool");
futures.remove(taskPool.take());
}
stop();
}
/**
* Abruptly shutdown the thread pool. Call when exiting a region server.
*/
void stop() {
if (this.stopped) return;
this.stopped = true;
this.executor.shutdownNow();
}
}
}

View File

@ -59,7 +59,7 @@ public interface HLog {
*/
static final String RECOVERED_EDITS_DIR = "recovered.edits";
static final Pattern EDITFILES_NAME_PATTERN = Pattern.compile("-?[0-9]+");
static final String RECOVERED_LOG_TMPFILE_SUFFIX = ".temp";
public static final String RECOVERED_LOG_TMPFILE_SUFFIX = ".temp";
public interface Reader {
void init(FileSystem fs, Path path, Configuration c) throws IOException;

View File

@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
@ -708,6 +709,55 @@ public class AccessController extends BaseRegionObserver
AccessControlLists.init(ctx.getEnvironment().getMasterServices());
}
@Override
public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
requirePermission("snapshot", Permission.Action.ADMIN);
}
@Override
public void postSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
requirePermission("clone", Permission.Action.ADMIN);
}
@Override
public void postCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
requirePermission("restore", Permission.Action.ADMIN);
}
@Override
public void postRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
throws IOException {
}
@Override
public void preDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot) throws IOException {
requirePermission("deleteSnapshot", Permission.Action.ADMIN);
}
@Override
public void postDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot) throws IOException {
}
/* ---- RegionObserver implementation ---- */

View File

@ -0,0 +1,90 @@
/**
* 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.snapshot;
import java.io.IOException;
import java.util.NavigableSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
/**
* Copy over each of the files in a region's recovered.edits directory to the region's snapshot
* directory.
* <p>
* This is a serial operation over each of the files in the recovered.edits directory and also
* streams all the bytes to the client and then back to the filesystem, so the files being copied
* should be <b>small</b> or it will (a) suck up a lot of bandwidth, and (b) take a long time.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class CopyRecoveredEditsTask extends SnapshotTask {
private static final Log LOG = LogFactory.getLog(CopyRecoveredEditsTask.class);
private final FileSystem fs;
private final Path regiondir;
private final Path outputDir;
/**
* @param snapshot Snapshot being taken
* @param monitor error monitor for the snapshot
* @param fs {@link FileSystem} where the snapshot is being taken
* @param regionDir directory for the region to examine for edits
* @param snapshotRegionDir directory for the region in the snapshot
*/
public CopyRecoveredEditsTask(SnapshotDescription snapshot, ForeignExceptionDispatcher monitor,
FileSystem fs, Path regionDir, Path snapshotRegionDir) {
super(snapshot, monitor);
this.fs = fs;
this.regiondir = regionDir;
this.outputDir = HLogUtil.getRegionDirRecoveredEditsDir(snapshotRegionDir);
}
@Override
public Void call() throws IOException {
NavigableSet<Path> files = HLogUtil.getSplitEditFilesSorted(this.fs, regiondir);
if (files == null || files.size() == 0) return null;
// copy over each file.
// this is really inefficient (could be trivially parallelized), but is
// really simple to reason about.
for (Path source : files) {
// check to see if the file is zero length, in which case we can skip it
FileStatus stat = fs.getFileStatus(source);
if (stat.getLen() <= 0) continue;
// its not zero length, so copy over the file
Path out = new Path(outputDir, source.getName());
LOG.debug("Copying " + source + " to " + out);
FileUtil.copy(fs, source, fs, out, true, fs.getConf());
// check for errors to the running operation after each file
this.rethrowException();
}
return null;
}
}

View File

@ -0,0 +1,56 @@
/**
* 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.snapshot;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
/**
* Exception thrown when the found snapshot info from the filesystem is not valid
*/
@SuppressWarnings("serial")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class CorruptedSnapshotException extends HBaseSnapshotException {
/**
* @param message message describing the exception
* @param e cause
*/
public CorruptedSnapshotException(String message, Exception e) {
super(message, e);
}
/**
* Snapshot was corrupt for some reason
* @param message full description of the failure
* @param snapshot snapshot that was expected
*/
public CorruptedSnapshotException(String message, SnapshotDescription snapshot) {
super(message, snapshot);
}
/**
* @param message message describing the exception
*/
public CorruptedSnapshotException(String message) {
super(message, (SnapshotDescription)null);
}
}

View File

@ -0,0 +1,686 @@
/**
* 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.snapshot;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.LinkedList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.HLogLink;
import org.apache.hadoop.hbase.mapreduce.JobUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
/**
* Export the specified snapshot to a given FileSystem.
*
* The .snapshot/name folder is copied to the destination cluster
* and then all the hfiles/hlogs are copied using a Map-Reduce Job in the .archive/ location.
* When everything is done, the second cluster can restore the snapshot.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class ExportSnapshot extends Configured implements Tool {
private static final Log LOG = LogFactory.getLog(ExportSnapshot.class);
private static final String CONF_FILES_USER = "snapshot.export.files.attributes.user";
private static final String CONF_FILES_GROUP = "snapshot.export.files.attributes.group";
private static final String CONF_FILES_MODE = "snapshot.export.files.attributes.mode";
private static final String CONF_CHECKSUM_VERIFY = "snapshot.export.checksum.verify";
private static final String CONF_OUTPUT_ROOT = "snapshot.export.output.root";
private static final String CONF_INPUT_ROOT = "snapshot.export.input.root";
private static final String INPUT_FOLDER_PREFIX = "export-files.";
// Export Map-Reduce Counters, to keep track of the progress
public enum Counter { MISSING_FILES, COPY_FAILED, BYTES_EXPECTED, BYTES_COPIED };
private static class ExportMapper extends Mapper<Text, NullWritable, NullWritable, NullWritable> {
final static int REPORT_SIZE = 1 * 1024 * 1024;
final static int BUFFER_SIZE = 64 * 1024;
private boolean verifyChecksum;
private String filesGroup;
private String filesUser;
private short filesMode;
private FileSystem outputFs;
private Path outputArchive;
private Path outputRoot;
private FileSystem inputFs;
private Path inputArchive;
private Path inputRoot;
@Override
public void setup(Context context) {
Configuration conf = context.getConfiguration();
verifyChecksum = conf.getBoolean(CONF_CHECKSUM_VERIFY, true);
filesGroup = conf.get(CONF_FILES_GROUP);
filesUser = conf.get(CONF_FILES_USER);
filesMode = (short)conf.getInt(CONF_FILES_MODE, 0);
outputRoot = new Path(conf.get(CONF_OUTPUT_ROOT));
inputRoot = new Path(conf.get(CONF_INPUT_ROOT));
inputArchive = new Path(inputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY);
outputArchive = new Path(outputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY);
try {
inputFs = FileSystem.get(inputRoot.toUri(), conf);
} catch (IOException e) {
throw new RuntimeException("Could not get the input FileSystem with root=" + inputRoot, e);
}
try {
outputFs = FileSystem.get(outputRoot.toUri(), conf);
} catch (IOException e) {
throw new RuntimeException("Could not get the output FileSystem with root="+ outputRoot, e);
}
}
@Override
public void map(Text key, NullWritable value, Context context)
throws InterruptedException, IOException {
Path inputPath = new Path(key.toString());
Path outputPath = getOutputPath(inputPath);
LOG.info("copy file input=" + inputPath + " output=" + outputPath);
if (copyFile(context, inputPath, outputPath)) {
LOG.info("copy completed for input=" + inputPath + " output=" + outputPath);
}
}
/**
* Returns the location where the inputPath will be copied.
* - hfiles are encoded as hfile links hfile-region-table
* - logs are encoded as serverName/logName
*/
private Path getOutputPath(final Path inputPath) throws IOException {
Path path;
if (HFileLink.isHFileLink(inputPath)) {
String family = inputPath.getParent().getName();
String table = HFileLink.getReferencedTableName(inputPath.getName());
String region = HFileLink.getReferencedRegionName(inputPath.getName());
String hfile = HFileLink.getReferencedHFileName(inputPath.getName());
path = new Path(table, new Path(region, new Path(family, hfile)));
} else if (isHLogLinkPath(inputPath)) {
String logName = inputPath.getName();
path = new Path(new Path(outputRoot, HConstants.HREGION_OLDLOGDIR_NAME), logName);
} else {
path = inputPath;
}
return new Path(outputArchive, path);
}
private boolean copyFile(final Context context, final Path inputPath, final Path outputPath)
throws IOException {
FSDataInputStream in = openSourceFile(inputPath);
if (in == null) {
context.getCounter(Counter.MISSING_FILES).increment(1);
return false;
}
try {
// Verify if the input file exists
FileStatus inputStat = getFileStatus(inputFs, inputPath);
if (inputStat == null) return false;
// Verify if the output file exists and is the same that we want to copy
FileStatus outputStat = getFileStatus(outputFs, outputPath);
if (outputStat != null && sameFile(inputStat, outputStat)) {
LOG.info("Skip copy " + inputPath + " to " + outputPath + ", same file.");
return true;
}
context.getCounter(Counter.BYTES_EXPECTED).increment(inputStat.getLen());
// Ensure that the output folder is there and copy the file
outputFs.mkdirs(outputPath.getParent());
FSDataOutputStream out = outputFs.create(outputPath, true);
try {
if (!copyData(context, inputPath, in, outputPath, out, inputStat.getLen()))
return false;
} finally {
out.close();
}
// Preserve attributes
return preserveAttributes(outputPath, inputStat);
} finally {
in.close();
}
}
/**
* Preserve the files attribute selected by the user copying them from the source file
*/
private boolean preserveAttributes(final Path path, final FileStatus refStat) {
FileStatus stat;
try {
stat = outputFs.getFileStatus(path);
} catch (IOException e) {
LOG.warn("Unable to get the status for file=" + path);
return false;
}
try {
if (filesMode > 0 && stat.getPermission().toShort() != filesMode) {
outputFs.setPermission(path, new FsPermission(filesMode));
} else if (!stat.getPermission().equals(refStat.getPermission())) {
outputFs.setPermission(path, refStat.getPermission());
}
} catch (IOException e) {
LOG.error("Unable to set the permission for file=" + path, e);
return false;
}
try {
String user = (filesUser != null) ? filesUser : refStat.getOwner();
String group = (filesGroup != null) ? filesGroup : refStat.getGroup();
if (!(user.equals(stat.getOwner()) && group.equals(stat.getGroup()))) {
outputFs.setOwner(path, user, group);
}
} catch (IOException e) {
LOG.error("Unable to set the owner/group for file=" + path, e);
return false;
}
return true;
}
private boolean copyData(final Context context,
final Path inputPath, final FSDataInputStream in,
final Path outputPath, final FSDataOutputStream out,
final long inputFileSize) {
final String statusMessage = "copied %s/" + StringUtils.humanReadableInt(inputFileSize) +
" (%.3f%%) from " + inputPath + " to " + outputPath;
try {
byte[] buffer = new byte[BUFFER_SIZE];
long totalBytesWritten = 0;
int reportBytes = 0;
int bytesRead;
while ((bytesRead = in.read(buffer)) > 0) {
out.write(buffer, 0, bytesRead);
totalBytesWritten += bytesRead;
reportBytes += bytesRead;
if (reportBytes >= REPORT_SIZE) {
context.getCounter(Counter.BYTES_COPIED).increment(reportBytes);
context.setStatus(String.format(statusMessage,
StringUtils.humanReadableInt(totalBytesWritten),
reportBytes/(float)inputFileSize));
reportBytes = 0;
}
}
context.getCounter(Counter.BYTES_COPIED).increment(reportBytes);
context.setStatus(String.format(statusMessage,
StringUtils.humanReadableInt(totalBytesWritten),
reportBytes/(float)inputFileSize));
// Verify that the written size match
if (totalBytesWritten != inputFileSize) {
LOG.error("number of bytes copied not matching copied=" + totalBytesWritten +
" expected=" + inputFileSize + " for file=" + inputPath);
context.getCounter(Counter.COPY_FAILED).increment(1);
return false;
}
return true;
} catch (IOException e) {
LOG.error("Error copying " + inputPath + " to " + outputPath, e);
context.getCounter(Counter.COPY_FAILED).increment(1);
return false;
}
}
private FSDataInputStream openSourceFile(final Path path) {
try {
if (HFileLink.isHFileLink(path)) {
return new HFileLink(inputRoot, inputArchive, path).open(inputFs);
} else if (isHLogLinkPath(path)) {
String serverName = path.getParent().getName();
String logName = path.getName();
return new HLogLink(inputRoot, serverName, logName).open(inputFs);
}
return inputFs.open(path);
} catch (IOException e) {
LOG.error("Unable to open source file=" + path, e);
return null;
}
}
private FileStatus getFileStatus(final FileSystem fs, final Path path) {
try {
if (HFileLink.isHFileLink(path)) {
HFileLink link = new HFileLink(inputRoot, inputArchive, path);
return link.getFileStatus(fs);
} else if (isHLogLinkPath(path)) {
String serverName = path.getParent().getName();
String logName = path.getName();
return new HLogLink(inputRoot, serverName, logName).getFileStatus(fs);
}
return fs.getFileStatus(path);
} catch (IOException e) {
LOG.warn("Unable to get the status for file=" + path);
return null;
}
}
private FileChecksum getFileChecksum(final FileSystem fs, final Path path) {
try {
return fs.getFileChecksum(path);
} catch (IOException e) {
LOG.warn("Unable to get checksum for file=" + path, e);
return null;
}
}
/**
* Check if the two files are equal by looking at the file length,
* and at the checksum (if user has specified the verifyChecksum flag).
*/
private boolean sameFile(final FileStatus inputStat, final FileStatus outputStat) {
// Not matching length
if (inputStat.getLen() != outputStat.getLen()) return false;
// Mark files as equals, since user asked for no checksum verification
if (!verifyChecksum) return true;
// If checksums are not available, files are not the same.
FileChecksum inChecksum = getFileChecksum(inputFs, inputStat.getPath());
if (inChecksum == null) return false;
FileChecksum outChecksum = getFileChecksum(outputFs, outputStat.getPath());
if (outChecksum == null) return false;
return inChecksum.equals(outChecksum);
}
/**
* HLog files are encoded as serverName/logName
* and since all the other files should be in /hbase/table/..path..
* we can rely on the depth, for now.
*/
private static boolean isHLogLinkPath(final Path path) {
return path.depth() == 2;
}
}
/**
* Extract the list of files (HFiles/HLogs) to copy using Map-Reduce.
* @return list of files referenced by the snapshot (pair of path and size)
*/
private List<Pair<Path, Long>> getSnapshotFiles(final FileSystem fs, final Path snapshotDir)
throws IOException {
SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
final List<Pair<Path, Long>> files = new ArrayList<Pair<Path, Long>>();
final String table = snapshotDesc.getTable();
final Configuration conf = getConf();
// Get snapshot files
SnapshotReferenceUtil.visitReferencedFiles(fs, snapshotDir,
new SnapshotReferenceUtil.FileVisitor() {
public void storeFile (final String region, final String family, final String hfile)
throws IOException {
Path path = new Path(family, HFileLink.createHFileLinkName(table, region, hfile));
long size = new HFileLink(conf, path).getFileStatus(fs).getLen();
files.add(new Pair<Path, Long>(path, size));
}
public void recoveredEdits (final String region, final String logfile)
throws IOException {
// copied with the snapshot referenecs
}
public void logFile (final String server, final String logfile)
throws IOException {
long size = new HLogLink(conf, server, logfile).getFileStatus(fs).getLen();
files.add(new Pair<Path, Long>(new Path(server, logfile), size));
}
});
return files;
}
/**
* Given a list of file paths and sizes, create around ngroups in as balanced a way as possible.
* The groups created will have similar amounts of bytes.
* <p>
* The algorithm used is pretty straightforward; the file list is sorted by size,
* and then each group fetch the bigger file available, iterating through groups
* alternating the direction.
*/
static List<List<Path>> getBalancedSplits(final List<Pair<Path, Long>> files, int ngroups) {
// Sort files by size, from small to big
Collections.sort(files, new Comparator<Pair<Path, Long>>() {
public int compare(Pair<Path, Long> a, Pair<Path, Long> b) {
long r = a.getSecond() - b.getSecond();
return (r < 0) ? -1 : ((r > 0) ? 1 : 0);
}
});
// create balanced groups
List<List<Path>> fileGroups = new LinkedList<List<Path>>();
long[] sizeGroups = new long[ngroups];
int hi = files.size() - 1;
int lo = 0;
List<Path> group;
int dir = 1;
int g = 0;
while (hi >= lo) {
if (g == fileGroups.size()) {
group = new LinkedList<Path>();
fileGroups.add(group);
} else {
group = fileGroups.get(g);
}
Pair<Path, Long> fileInfo = files.get(hi--);
// add the hi one
sizeGroups[g] += fileInfo.getSecond();
group.add(fileInfo.getFirst());
// change direction when at the end or the beginning
g += dir;
if (g == ngroups) {
dir = -1;
g = ngroups - 1;
} else if (g < 0) {
dir = 1;
g = 0;
}
}
if (LOG.isDebugEnabled()) {
for (int i = 0; i < sizeGroups.length; ++i) {
LOG.debug("export split=" + i + " size=" + StringUtils.humanReadableInt(sizeGroups[i]));
}
}
return fileGroups;
}
private static Path getInputFolderPath(Configuration conf)
throws IOException, InterruptedException {
Path stagingDir = JobUtil.getStagingDir(conf);
return new Path(stagingDir, INPUT_FOLDER_PREFIX +
String.valueOf(EnvironmentEdgeManager.currentTimeMillis()));
}
/**
* Create the input files, with the path to copy, for the MR job.
* Each input files contains n files, and each input file has a similar amount data to copy.
* The number of input files created are based on the number of mappers provided as argument
* and the number of the files to copy.
*/
private static Path[] createInputFiles(final Configuration conf,
final List<Pair<Path, Long>> snapshotFiles, int mappers)
throws IOException, InterruptedException {
Path inputFolderPath = getInputFolderPath(conf);
FileSystem fs = inputFolderPath.getFileSystem(conf);
LOG.debug("Input folder location: " + inputFolderPath);
List<List<Path>> splits = getBalancedSplits(snapshotFiles, mappers);
Path[] inputFiles = new Path[splits.size()];
Text key = new Text();
for (int i = 0; i < inputFiles.length; i++) {
List<Path> files = splits.get(i);
inputFiles[i] = new Path(inputFolderPath, String.format("export-%d.seq", i));
SequenceFile.Writer writer = SequenceFile.createWriter(fs, conf, inputFiles[i],
Text.class, NullWritable.class);
LOG.debug("Input split: " + i);
try {
for (Path file: files) {
LOG.debug(file.toString());
key.set(file.toString());
writer.append(key, NullWritable.get());
}
} finally {
writer.close();
}
}
return inputFiles;
}
/**
* Run Map-Reduce Job to perform the files copy.
*/
private boolean runCopyJob(final Path inputRoot, final Path outputRoot,
final List<Pair<Path, Long>> snapshotFiles, final boolean verifyChecksum,
final String filesUser, final String filesGroup, final int filesMode,
final int mappers) throws IOException, InterruptedException, ClassNotFoundException {
Configuration conf = getConf();
if (filesGroup != null) conf.set(CONF_FILES_GROUP, filesGroup);
if (filesUser != null) conf.set(CONF_FILES_USER, filesUser);
conf.setInt(CONF_FILES_MODE, filesMode);
conf.setBoolean(CONF_CHECKSUM_VERIFY, verifyChecksum);
conf.set(CONF_OUTPUT_ROOT, outputRoot.toString());
conf.set(CONF_INPUT_ROOT, inputRoot.toString());
conf.setInt("mapreduce.job.maps", mappers);
Job job = new Job(conf);
job.setJobName("ExportSnapshot");
job.setJarByClass(ExportSnapshot.class);
job.setMapperClass(ExportMapper.class);
job.setInputFormatClass(SequenceFileInputFormat.class);
job.setOutputFormatClass(NullOutputFormat.class);
job.setMapSpeculativeExecution(false);
job.setNumReduceTasks(0);
for (Path path: createInputFiles(conf, snapshotFiles, mappers)) {
LOG.debug("Add Input Path=" + path);
SequenceFileInputFormat.addInputPath(job, path);
}
return job.waitForCompletion(true);
}
/**
* Execute the export snapshot by copying the snapshot metadata, hfiles and hlogs.
* @return 0 on success, and != 0 upon failure.
*/
@Override
public int run(String[] args) throws Exception {
boolean verifyChecksum = true;
String snapshotName = null;
String filesGroup = null;
String filesUser = null;
Path outputRoot = null;
int filesMode = 0;
int mappers = getConf().getInt("mapreduce.job.maps", 1);
// Process command line args
for (int i = 0; i < args.length; i++) {
String cmd = args[i];
try {
if (cmd.equals("-snapshot")) {
snapshotName = args[++i];
} else if (cmd.equals("-copy-to")) {
outputRoot = new Path(args[++i]);
} else if (cmd.equals("-no-checksum-verify")) {
verifyChecksum = false;
} else if (cmd.equals("-mappers")) {
mappers = Integer.parseInt(args[++i]);
} else if (cmd.equals("-chuser")) {
filesUser = args[++i];
} else if (cmd.equals("-chgroup")) {
filesGroup = args[++i];
} else if (cmd.equals("-chmod")) {
filesMode = Integer.parseInt(args[++i], 8);
} else if (cmd.equals("-h") || cmd.equals("--help")) {
printUsageAndExit();
} else {
System.err.println("UNEXPECTED: " + cmd);
printUsageAndExit();
}
} catch (Exception e) {
printUsageAndExit();
}
}
// Check user options
if (snapshotName == null) {
System.err.println("Snapshot name not provided.");
printUsageAndExit();
}
if (outputRoot == null) {
System.err.println("Destination file-system not provided.");
printUsageAndExit();
}
Configuration conf = getConf();
Path inputRoot = FSUtils.getRootDir(conf);
FileSystem inputFs = FileSystem.get(conf);
FileSystem outputFs = FileSystem.get(outputRoot.toUri(), conf);
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, inputRoot);
Path snapshotTmpDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshotName, outputRoot);
Path outputSnapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, outputRoot);
// Check if the snapshot already exists
if (outputFs.exists(outputSnapshotDir)) {
System.err.println("The snapshot '" + snapshotName +
"' already exists in the destination: " + outputSnapshotDir);
return 1;
}
// Check if the snapshot already in-progress
if (outputFs.exists(snapshotTmpDir)) {
System.err.println("A snapshot with the same name '" + snapshotName + "' is in-progress");
return 1;
}
// Step 0 - Extract snapshot files to copy
final List<Pair<Path, Long>> files = getSnapshotFiles(inputFs, snapshotDir);
// Step 1 - Copy fs1:/.snapshot/<snapshot> to fs2:/.snapshot/.tmp/<snapshot>
// The snapshot references must be copied before the hfiles otherwise the cleaner
// will remove them because they are unreferenced.
try {
FileUtil.copy(inputFs, snapshotDir, outputFs, snapshotTmpDir, false, false, conf);
} catch (IOException e) {
System.err.println("Failed to copy the snapshot directory: from=" + snapshotDir +
" to=" + snapshotTmpDir);
e.printStackTrace(System.err);
return 1;
}
// Step 2 - Start MR Job to copy files
// The snapshot references must be copied before the files otherwise the files gets removed
// by the HFileArchiver, since they have no references.
try {
if (!runCopyJob(inputRoot, outputRoot, files, verifyChecksum,
filesUser, filesGroup, filesMode, mappers)) {
throw new ExportSnapshotException("Snapshot export failed!");
}
// Step 3 - Rename fs2:/.snapshot/.tmp/<snapshot> fs2:/.snapshot/<snapshot>
if (!outputFs.rename(snapshotTmpDir, outputSnapshotDir)) {
System.err.println("Snapshot export failed!");
System.err.println("Unable to rename snapshot directory from=" +
snapshotTmpDir + " to=" + outputSnapshotDir);
return 1;
}
return 0;
} catch (Exception e) {
System.err.println("Snapshot export failed!");
e.printStackTrace(System.err);
outputFs.delete(outputSnapshotDir, true);
return 1;
}
}
// ExportSnapshot
private void printUsageAndExit() {
System.err.printf("Usage: bin/hbase %s [options]%n", getClass().getName());
System.err.println(" where [options] are:");
System.err.println(" -h|-help Show this help and exit.");
System.err.println(" -snapshot NAME Snapshot to restore.");
System.err.println(" -copy-to NAME Remote destination hdfs://");
System.err.println(" -no-checksum-verify Do not verify checksum.");
System.err.println(" -chuser USERNAME Change the owner of the files to the specified one.");
System.err.println(" -chgroup GROUP Change the group of the files to the specified one.");
System.err.println(" -chmod MODE Change the permission of the files to the specified one.");
System.err.println(" -mappers Number of mappers to use during the copy (mapreduce.job.maps).");
System.err.println();
System.err.println("Examples:");
System.err.println(" hbase " + getClass() + " \\");
System.err.println(" -snapshot MySnapshot -copy-to hdfs:///srv2:8082/hbase \\");
System.err.println(" -chuser MyUser -chgroup MyGroup -chmod 700 -mappers 16");
System.exit(1);
}
/**
* The guts of the {@link #main} method.
* Call this method to avoid the {@link #main(String[])} System.exit.
* @param args
* @return errCode
* @throws Exception
*/
static int innerMain(final Configuration conf, final String [] args) throws Exception {
return ToolRunner.run(conf, new ExportSnapshot(), args);
}
public static void main(String[] args) throws Exception {
System.exit(innerMain(HBaseConfiguration.create(), args));
}
}

View File

@ -0,0 +1,43 @@
/**
* 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.snapshot;
import org.apache.hadoop.classification.InterfaceAudience;
/**
* Thrown when a snapshot could not be exported due to an error during the operation.
*/
@InterfaceAudience.Public
@SuppressWarnings("serial")
public class ExportSnapshotException extends HBaseSnapshotException {
/**
* @param msg message describing the exception
*/
public ExportSnapshotException(String msg) {
super(msg);
}
/**
* @param message message describing the exception
* @param e cause
*/
public ExportSnapshotException(String message, Exception e) {
super(message, e);
}
}

View File

@ -0,0 +1,77 @@
/**
* 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.snapshot;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
/**
* General exception base class for when a snapshot fails
*/
@SuppressWarnings("serial")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class HBaseSnapshotException extends HBaseIOException {
private SnapshotDescription description;
/**
* Some exception happened for a snapshot and don't even know the snapshot that it was about
* @param msg Full description of the failure
*/
public HBaseSnapshotException(String msg) {
super(msg);
}
/**
* Exception for the given snapshot that has no previous root cause
* @param msg reason why the snapshot failed
* @param desc description of the snapshot that is being failed
*/
public HBaseSnapshotException(String msg, SnapshotDescription desc) {
super(msg);
this.description = desc;
}
/**
* Exception for the given snapshot due to another exception
* @param msg reason why the snapshot failed
* @param cause root cause of the failure
* @param desc description of the snapshot that is being failed
*/
public HBaseSnapshotException(String msg, Throwable cause, SnapshotDescription desc) {
super(msg, cause);
this.description = desc;
}
/**
* Exception when the description of the snapshot cannot be determined, due to some root other
* root cause
* @param message description of what caused the failure
* @param e root cause
*/
public HBaseSnapshotException(String message, Exception e) {
super(message, e);
}
public SnapshotDescription getSnapshotDescription() {
return this.description;
}
}

View File

@ -0,0 +1,127 @@
/**
* 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.snapshot;
import java.io.IOException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* Reference all the hfiles in a region for a snapshot.
* <p>
* Doesn't take into acccount if the hfiles are valid or not, just keeps track of what's in the
* region's directory.
*/
public class ReferenceRegionHFilesTask extends SnapshotTask {
public static final Log LOG = LogFactory.getLog(ReferenceRegionHFilesTask.class);
private final Path regiondir;
private final FileSystem fs;
private final PathFilter fileFilter;
private final Path snapshotDir;
/**
* Reference all the files in the given region directory
* @param snapshot snapshot for which to add references
* @param monitor to check/send error
* @param regionDir region directory to look for errors
* @param fs {@link FileSystem} where the snapshot/region live
* @param regionSnapshotDir directory in the snapshot to store region files
*/
public ReferenceRegionHFilesTask(final SnapshotDescription snapshot,
ForeignExceptionDispatcher monitor, Path regionDir, final FileSystem fs, Path regionSnapshotDir) {
super(snapshot, monitor);
this.regiondir = regionDir;
this.fs = fs;
this.fileFilter = new PathFilter() {
@Override
public boolean accept(Path path) {
try {
return fs.isFile(path);
} catch (IOException e) {
LOG.error("Failed to reach fs to check file:" + path + ", marking as not file");
ReferenceRegionHFilesTask.this.snapshotFailure("Failed to reach fs to check file status",
e);
return false;
}
}
};
this.snapshotDir = regionSnapshotDir;
}
@Override
public Void call() throws IOException {
FileStatus[] families = FSUtils.listStatus(fs, regiondir, new FSUtils.FamilyDirFilter(fs));
// if no families, then we are done again
if (families == null || families.length == 0) {
LOG.info("No families under region directory:" + regiondir
+ ", not attempting to add references.");
return null;
}
// snapshot directories to store the hfile reference
List<Path> snapshotFamilyDirs = TakeSnapshotUtils.getFamilySnapshotDirectories(snapshot,
snapshotDir, families);
LOG.debug("Add hfile references to snapshot directories:" + snapshotFamilyDirs);
for (int i = 0; i < families.length; i++) {
FileStatus family = families[i];
Path familyDir = family.getPath();
// get all the hfiles in the family
FileStatus[] hfiles = FSUtils.listStatus(fs, familyDir, fileFilter);
// if no hfiles, then we are done with this family
if (hfiles == null || hfiles.length == 0) {
LOG.debug("Not hfiles found for family: " + familyDir + ", skipping.");
continue;
}
// make the snapshot's family directory
Path snapshotFamilyDir = snapshotFamilyDirs.get(i);
fs.mkdirs(snapshotFamilyDir);
// create a reference for each hfile
for (FileStatus hfile : hfiles) {
// references are 0-length files, relying on file name.
Path referenceFile = new Path(snapshotFamilyDir, hfile.getPath().getName());
LOG.debug("Creating reference for:" + hfile.getPath() + " at " + referenceFile);
if (!fs.createNewFile(referenceFile)) {
throw new IOException("Failed to create reference file:" + referenceFile);
}
}
}
if (LOG.isDebugEnabled()) {
LOG.debug("Finished referencing hfiles, current region state:");
FSUtils.logFileSystemState(fs, regiondir, LOG);
LOG.debug("and the snapshot directory:");
FSUtils.logFileSystemState(fs, snapshotDir, LOG);
}
return null;
}
}

View File

@ -0,0 +1,108 @@
/**
* 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.snapshot;
import java.io.IOException;
import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* Reference all the WAL files under a server's WAL directory
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class ReferenceServerWALsTask extends SnapshotTask {
private static final Log LOG = LogFactory.getLog(ReferenceServerWALsTask.class);
private final FileSystem fs;
private final Configuration conf;
private final String serverName;
private Path logDir;
/**
* @param snapshot snapshot being run
* @param failureListener listener to check for errors while running the operation and to
* propagate errors found while running the task
* @param logDir log directory for the server. Name of the directory is taken as the name of the
* server
* @param conf {@link Configuration} to extract filesystem information
* @param fs filesystem where the log files are stored and should be referenced
*/
public ReferenceServerWALsTask(SnapshotDescription snapshot,
ForeignExceptionDispatcher failureListener, final Path logDir, final Configuration conf,
final FileSystem fs) {
super(snapshot, failureListener);
this.fs = fs;
this.conf = conf;
this.serverName = logDir.getName();
this.logDir = logDir;
}
/**
* Create reference files (empty files with the same path and file name as original).
* @throws IOException exception from hdfs or network problems
* @throws ForeignException exception from an external procedure
*/
@Override
public Void call() throws IOException, ForeignException {
// TODO switch to using a single file to reference all required WAL files
// Iterate through each of the log files and add a reference to it.
// assumes that all the files under the server's logs directory is a log
FileStatus[] serverLogs = FSUtils.listStatus(fs, logDir, null);
if (serverLogs == null) {
LOG.debug("No logs for server directory:" + logDir + ", done referencing files.");
return null;
}
if (LOG.isDebugEnabled()) {
LOG.debug("Adding references for WAL files:" + Arrays.toString(serverLogs));
}
for (FileStatus file : serverLogs) {
this.rethrowException();
// add the reference to the file. ex: hbase/.snapshots/.logs/<serverName>/<hlog>
Path rootDir = FSUtils.getRootDir(conf);
Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(this.snapshot, rootDir);
Path snapshotLogDir = TakeSnapshotUtils.getSnapshotHLogsDir(snapshotDir, serverName);
// actually store the reference on disk (small file)
Path ref = new Path(snapshotLogDir, file.getPath().getName());
if (!fs.createNewFile(ref)) {
if (!fs.exists(ref)) {
throw new IOException("Couldn't create reference for:" + file.getPath());
}
}
LOG.debug("Completed WAL referencing for: " + file.getPath() + " to " + ref);
}
LOG.debug("Successfully completed WAL referencing for ALL files");
return null;
}
}

View File

@ -0,0 +1,43 @@
/**
* 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.snapshot;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
/**
* Thrown when a snapshot could not be restored due to a server-side error when restoring it.
*/
@SuppressWarnings("serial")
public class RestoreSnapshotException extends HBaseSnapshotException {
public RestoreSnapshotException(String msg, SnapshotDescription desc) {
super(msg, desc);
}
public RestoreSnapshotException(String msg, Throwable cause, SnapshotDescription desc) {
super(msg, cause, desc);
}
public RestoreSnapshotException(String msg) {
super(msg);
}
public RestoreSnapshotException(String message, Exception e) {
super(message, e);
}
}

View File

@ -0,0 +1,588 @@
/**
* 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.snapshot;
import java.io.InputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.FSVisitor;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.io.IOUtils;
/**
* Helper to Restore/Clone a Snapshot
*
* <p>The helper assumes that a table is already created, and by calling restore()
* the content present in the snapshot will be restored as the new content of the table.
*
* <p>Clone from Snapshot: If the target table is empty, the restore operation
* is just a "clone operation", where the only operations are:
* <ul>
* <li>for each region in the snapshot create a new region
* (note that the region will have a different name, since the encoding contains the table name)
* <li>for each file in the region create a new HFileLink to point to the original file.
* <li>restore the logs, if any
* </ul>
*
* <p>Restore from Snapshot:
* <ul>
* <li>for each region in the table verify which are available in the snapshot and which are not
* <ul>
* <li>if the region is not present in the snapshot, remove it.
* <li>if the region is present in the snapshot
* <ul>
* <li>for each file in the table region verify which are available in the snapshot
* <ul>
* <li>if the hfile is not present in the snapshot, remove it
* <li>if the hfile is present, keep it (nothing to do)
* </ul>
* <li>for each file in the snapshot region but not in the table
* <ul>
* <li>create a new HFileLink that point to the original file
* </ul>
* </ul>
* </ul>
* <li>for each region in the snapshot not present in the current table state
* <ul>
* <li>create a new region and for each file in the region create a new HFileLink
* (This is the same as the clone operation)
* </ul>
* <li>restore the logs, if any
* </ul>
*/
@InterfaceAudience.Private
public class RestoreSnapshotHelper {
private static final Log LOG = LogFactory.getLog(RestoreSnapshotHelper.class);
private final Map<byte[], byte[]> regionsMap =
new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
private final ForeignExceptionDispatcher monitor;
private final SnapshotDescription snapshotDesc;
private final Path snapshotDir;
private final HTableDescriptor tableDesc;
private final Path tableDir;
private final Configuration conf;
private final FileSystem fs;
public RestoreSnapshotHelper(final Configuration conf, final FileSystem fs,
final SnapshotDescription snapshotDescription, final Path snapshotDir,
final HTableDescriptor tableDescriptor, final Path tableDir,
final ForeignExceptionDispatcher monitor)
{
this.fs = fs;
this.conf = conf;
this.snapshotDesc = snapshotDescription;
this.snapshotDir = snapshotDir;
this.tableDesc = tableDescriptor;
this.tableDir = tableDir;
this.monitor = monitor;
}
/**
* Restore the on-disk table to a specified snapshot state.
* @return the set of regions touched by the restore operation
*/
public RestoreMetaChanges restoreHdfsRegions() throws IOException {
LOG.debug("starting restore");
Set<String> snapshotRegionNames = SnapshotReferenceUtil.getSnapshotRegionNames(fs, snapshotDir);
if (snapshotRegionNames == null) {
LOG.warn("Nothing to restore. Snapshot " + snapshotDesc + " looks empty");
return null;
}
RestoreMetaChanges metaChanges = new RestoreMetaChanges();
// Identify which region are still available and which not.
// NOTE: we rely upon the region name as: "table name, start key, end key"
List<HRegionInfo> tableRegions = getTableRegions();
if (tableRegions != null) {
monitor.rethrowException();
for (HRegionInfo regionInfo: tableRegions) {
String regionName = regionInfo.getEncodedName();
if (snapshotRegionNames.contains(regionName)) {
LOG.info("region to restore: " + regionName);
snapshotRegionNames.remove(regionName);
metaChanges.addRegionToRestore(regionInfo);
} else {
LOG.info("region to remove: " + regionName);
metaChanges.addRegionToRemove(regionInfo);
}
}
// Restore regions using the snapshot data
monitor.rethrowException();
restoreHdfsRegions(metaChanges.getRegionsToRestore());
// Remove regions from the current table
monitor.rethrowException();
removeHdfsRegions(metaChanges.getRegionsToRemove());
}
// Regions to Add: present in the snapshot but not in the current table
if (snapshotRegionNames.size() > 0) {
List<HRegionInfo> regionsToAdd = new LinkedList<HRegionInfo>();
monitor.rethrowException();
for (String regionName: snapshotRegionNames) {
LOG.info("region to add: " + regionName);
Path regionDir = new Path(snapshotDir, regionName);
regionsToAdd.add(HRegion.loadDotRegionInfoFileContent(fs, regionDir));
}
// Create new regions cloning from the snapshot
monitor.rethrowException();
HRegionInfo[] clonedRegions = cloneHdfsRegions(regionsToAdd);
metaChanges.setNewRegions(clonedRegions);
}
// Restore WALs
monitor.rethrowException();
restoreWALs();
return metaChanges;
}
/**
* Describe the set of operations needed to update META after restore.
*/
public static class RestoreMetaChanges {
private List<HRegionInfo> regionsToRestore = null;
private List<HRegionInfo> regionsToRemove = null;
private List<HRegionInfo> regionsToAdd = null;
/**
* @return true if there're new regions
*/
public boolean hasRegionsToAdd() {
return this.regionsToAdd != null && this.regionsToAdd.size() > 0;
}
/**
* Returns the list of new regions added during the on-disk restore.
* The caller is responsible to add the regions to META.
* e.g MetaEditor.addRegionsToMeta(...)
* @return the list of regions to add to META
*/
public List<HRegionInfo> getRegionsToAdd() {
return this.regionsToAdd;
}
/**
* @return true if there're regions to restore
*/
public boolean hasRegionsToRestore() {
return this.regionsToRestore != null && this.regionsToRestore.size() > 0;
}
/**
* Returns the list of 'restored regions' during the on-disk restore.
* The caller is responsible to add the regions to META if not present.
* @return the list of regions restored
*/
public List<HRegionInfo> getRegionsToRestore() {
return this.regionsToRestore;
}
/**
* @return true if there're regions to remove
*/
public boolean hasRegionsToRemove() {
return this.regionsToRemove != null && this.regionsToRemove.size() > 0;
}
/**
* Returns the list of regions removed during the on-disk restore.
* The caller is responsible to remove the regions from META.
* e.g. MetaEditor.deleteRegions(...)
* @return the list of regions to remove from META
*/
public List<HRegionInfo> getRegionsToRemove() {
return this.regionsToRemove;
}
void setNewRegions(final HRegionInfo[] hris) {
if (hris != null) {
regionsToAdd = Arrays.asList(hris);
} else {
regionsToAdd = null;
}
}
void addRegionToRemove(final HRegionInfo hri) {
if (regionsToRemove == null) {
regionsToRemove = new LinkedList<HRegionInfo>();
}
regionsToRemove.add(hri);
}
void addRegionToRestore(final HRegionInfo hri) {
if (regionsToRestore == null) {
regionsToRestore = new LinkedList<HRegionInfo>();
}
regionsToRestore.add(hri);
}
}
/**
* Remove specified regions from the file-system, using the archiver.
*/
private void removeHdfsRegions(final List<HRegionInfo> regions) throws IOException {
if (regions != null && regions.size() > 0) {
for (HRegionInfo hri: regions) {
HFileArchiver.archiveRegion(conf, fs, hri);
}
}
}
/**
* Restore specified regions by restoring content to the snapshot state.
*/
private void restoreHdfsRegions(final List<HRegionInfo> regions) throws IOException {
if (regions == null || regions.size() == 0) return;
for (HRegionInfo hri: regions) restoreRegion(hri);
}
/**
* Restore region by removing files not in the snapshot
* and adding the missing ones from the snapshot.
*/
private void restoreRegion(HRegionInfo regionInfo) throws IOException {
Path snapshotRegionDir = new Path(snapshotDir, regionInfo.getEncodedName());
Map<String, List<String>> snapshotFiles =
SnapshotReferenceUtil.getRegionHFileReferences(fs, snapshotRegionDir);
Path regionDir = new Path(tableDir, regionInfo.getEncodedName());
String tableName = tableDesc.getNameAsString();
// Restore families present in the table
for (Path familyDir: FSUtils.getFamilyDirs(fs, regionDir)) {
byte[] family = Bytes.toBytes(familyDir.getName());
Set<String> familyFiles = getTableRegionFamilyFiles(familyDir);
List<String> snapshotFamilyFiles = snapshotFiles.remove(familyDir.getName());
if (snapshotFamilyFiles != null) {
List<String> hfilesToAdd = new LinkedList<String>();
for (String hfileName: snapshotFamilyFiles) {
if (familyFiles.contains(hfileName)) {
// HFile already present
familyFiles.remove(hfileName);
} else {
// HFile missing
hfilesToAdd.add(hfileName);
}
}
// Restore Missing files
for (String hfileName: hfilesToAdd) {
LOG.trace("Adding HFileLink " + hfileName +
" to region=" + regionInfo.getEncodedName() + " table=" + tableName);
restoreStoreFile(familyDir, regionInfo, hfileName);
}
// Remove hfiles not present in the snapshot
for (String hfileName: familyFiles) {
Path hfile = new Path(familyDir, hfileName);
LOG.trace("Removing hfile=" + hfile +
" from region=" + regionInfo.getEncodedName() + " table=" + tableName);
HFileArchiver.archiveStoreFile(fs, regionInfo, conf, tableDir, family, hfile);
}
} else {
// Family doesn't exists in the snapshot
LOG.trace("Removing family=" + Bytes.toString(family) +
" from region=" + regionInfo.getEncodedName() + " table=" + tableName);
HFileArchiver.archiveFamily(fs, conf, regionInfo, tableDir, family);
fs.delete(familyDir, true);
}
}
// Add families not present in the table
for (Map.Entry<String, List<String>> familyEntry: snapshotFiles.entrySet()) {
Path familyDir = new Path(regionDir, familyEntry.getKey());
if (!fs.mkdirs(familyDir)) {
throw new IOException("Unable to create familyDir=" + familyDir);
}
for (String hfileName: familyEntry.getValue()) {
LOG.trace("Adding HFileLink " + hfileName + " to table=" + tableName);
restoreStoreFile(familyDir, regionInfo, hfileName);
}
}
}
/**
* @return The set of files in the specified family directory.
*/
private Set<String> getTableRegionFamilyFiles(final Path familyDir) throws IOException {
Set<String> familyFiles = new HashSet<String>();
FileStatus[] hfiles = FSUtils.listStatus(fs, familyDir);
if (hfiles == null) return familyFiles;
for (FileStatus hfileRef: hfiles) {
String hfileName = hfileRef.getPath().getName();
familyFiles.add(hfileName);
}
return familyFiles;
}
/**
* Clone specified regions. For each region create a new region
* and create a HFileLink for each hfile.
*/
private HRegionInfo[] cloneHdfsRegions(final List<HRegionInfo> regions) throws IOException {
if (regions == null || regions.size() == 0) return null;
final Map<String, HRegionInfo> snapshotRegions =
new HashMap<String, HRegionInfo>(regions.size());
// clone region info (change embedded tableName with the new one)
HRegionInfo[] clonedRegionsInfo = new HRegionInfo[regions.size()];
for (int i = 0; i < clonedRegionsInfo.length; ++i) {
// clone the region info from the snapshot region info
HRegionInfo snapshotRegionInfo = regions.get(i);
clonedRegionsInfo[i] = cloneRegionInfo(snapshotRegionInfo);
// add the region name mapping between snapshot and cloned
String snapshotRegionName = snapshotRegionInfo.getEncodedName();
String clonedRegionName = clonedRegionsInfo[i].getEncodedName();
regionsMap.put(Bytes.toBytes(snapshotRegionName), Bytes.toBytes(clonedRegionName));
LOG.info("clone region=" + snapshotRegionName + " as " + clonedRegionName);
// Add mapping between cloned region name and snapshot region info
snapshotRegions.put(clonedRegionName, snapshotRegionInfo);
}
// create the regions on disk
ModifyRegionUtils.createRegions(conf, tableDir.getParent(),
tableDesc, clonedRegionsInfo, new ModifyRegionUtils.RegionFillTask() {
public void fillRegion(final HRegion region) throws IOException {
cloneRegion(region, snapshotRegions.get(region.getRegionInfo().getEncodedName()));
}
});
return clonedRegionsInfo;
}
/**
* Clone region directory content from the snapshot info.
*
* Each region is encoded with the table name, so the cloned region will have
* a different region name.
*
* Instead of copying the hfiles a HFileLink is created.
*
* @param region {@link HRegion} cloned
* @param snapshotRegionInfo
*/
private void cloneRegion(final HRegion region, final HRegionInfo snapshotRegionInfo)
throws IOException {
final Path snapshotRegionDir = new Path(snapshotDir, snapshotRegionInfo.getEncodedName());
final Path regionDir = new Path(tableDir, region.getRegionInfo().getEncodedName());
final String tableName = tableDesc.getNameAsString();
SnapshotReferenceUtil.visitRegionStoreFiles(fs, snapshotRegionDir,
new FSVisitor.StoreFileVisitor() {
public void storeFile (final String region, final String family, final String hfile)
throws IOException {
LOG.info("Adding HFileLink " + hfile + " to table=" + tableName);
Path familyDir = new Path(regionDir, family);
restoreStoreFile(familyDir, snapshotRegionInfo, hfile);
}
});
}
/**
* Create a new {@link HFileLink} to reference the store file.
* <p>The store file in the snapshot can be a simple hfile, an HFileLink or a reference.
* <ul>
* <li>hfile: abc -> table=region-abc
* <li>reference: abc.1234 -> table=region-abc.1234
* <li>hfilelink: table=region-hfile -> table=region-hfile
* </ul>
* @param familyDir destination directory for the store file
* @param regionInfo destination region info for the table
* @param hfileName store file name (can be a Reference, HFileLink or simple HFile)
*/
private void restoreStoreFile(final Path familyDir, final HRegionInfo regionInfo,
final String hfileName) throws IOException {
if (HFileLink.isHFileLink(hfileName)) {
HFileLink.createFromHFileLink(conf, fs, familyDir, hfileName);
} else if (StoreFile.isReference(hfileName)) {
restoreReferenceFile(familyDir, regionInfo, hfileName);
} else {
HFileLink.create(conf, fs, familyDir, regionInfo, hfileName);
}
}
/**
* Create a new {@link Reference} as copy of the source one.
* <p><blockquote><pre>
* The source table looks like:
* 1234/abc (original file)
* 5678/abc.1234 (reference file)
*
* After the clone operation looks like:
* wxyz/table=1234-abc
* stuv/table=1234-abc.wxyz
*
* NOTE that the region name in the clone changes (md5 of regioninfo)
* and the reference should reflect that change.
* </pre></blockquote>
* @param familyDir destination directory for the store file
* @param regionInfo destination region info for the table
* @param hfileName reference file name
*/
private void restoreReferenceFile(final Path familyDir, final HRegionInfo regionInfo,
final String hfileName) throws IOException {
// Extract the referred information (hfile name and parent region)
String tableName = snapshotDesc.getTable();
Path refPath = StoreFile.getReferredToFile(new Path(new Path(new Path(tableName,
regionInfo.getEncodedName()), familyDir.getName()), hfileName));
String snapshotRegionName = refPath.getParent().getParent().getName();
String fileName = refPath.getName();
// The new reference should have the cloned region name as parent, if it is a clone.
String clonedRegionName = Bytes.toString(regionsMap.get(Bytes.toBytes(snapshotRegionName)));
if (clonedRegionName == null) clonedRegionName = snapshotRegionName;
// The output file should be a reference link table=snapshotRegion-fileName.clonedRegionName
String refLink = fileName;
if (!HFileLink.isHFileLink(fileName)) {
refLink = HFileLink.createHFileLinkName(tableName, snapshotRegionName, fileName);
}
Path outPath = new Path(familyDir, refLink + '.' + clonedRegionName);
// Create the new reference
Path linkPath = new Path(familyDir,
HFileLink.createHFileLinkName(tableName, regionInfo.getEncodedName(), hfileName));
InputStream in = new HFileLink(conf, linkPath).open(fs);
OutputStream out = fs.create(outPath);
IOUtils.copyBytes(in, out, conf);
}
/**
* Create a new {@link HRegionInfo} from the snapshot region info.
* Keep the same startKey, endKey, regionId and split information but change
* the table name.
*
* @param snapshotRegionInfo Info for region to clone.
* @return the new HRegion instance
*/
public HRegionInfo cloneRegionInfo(final HRegionInfo snapshotRegionInfo) {
return new HRegionInfo(tableDesc.getName(),
snapshotRegionInfo.getStartKey(), snapshotRegionInfo.getEndKey(),
snapshotRegionInfo.isSplit(), snapshotRegionInfo.getRegionId());
}
/**
* Restore snapshot WALs.
*
* Global Snapshot keep a reference to region servers logs present during the snapshot.
* (/hbase/.snapshot/snapshotName/.logs/hostName/logName)
*
* Since each log contains different tables data, logs must be split to
* extract the table that we are interested in.
*/
private void restoreWALs() throws IOException {
final SnapshotLogSplitter logSplitter = new SnapshotLogSplitter(conf, fs, tableDir,
Bytes.toBytes(snapshotDesc.getTable()), regionsMap);
try {
// Recover.Edits
SnapshotReferenceUtil.visitRecoveredEdits(fs, snapshotDir,
new FSVisitor.RecoveredEditsVisitor() {
public void recoveredEdits (final String region, final String logfile) throws IOException {
Path path = SnapshotReferenceUtil.getRecoveredEdits(snapshotDir, region, logfile);
logSplitter.splitRecoveredEdit(path);
}
});
// Region Server Logs
SnapshotReferenceUtil.visitLogFiles(fs, snapshotDir, new FSVisitor.LogFileVisitor() {
public void logFile (final String server, final String logfile) throws IOException {
logSplitter.splitLog(server, logfile);
}
});
} finally {
logSplitter.close();
}
}
/**
* @return the set of the regions contained in the table
*/
private List<HRegionInfo> getTableRegions() throws IOException {
LOG.debug("get table regions: " + tableDir);
FileStatus[] regionDirs = FSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs));
if (regionDirs == null) return null;
List<HRegionInfo> regions = new LinkedList<HRegionInfo>();
for (FileStatus regionDir: regionDirs) {
HRegionInfo hri = HRegion.loadDotRegionInfoFileContent(fs, regionDir.getPath());
regions.add(hri);
}
LOG.debug("found " + regions.size() + " regions for table=" + tableDesc.getNameAsString());
return regions;
}
/**
* Create a new table descriptor cloning the snapshot table schema.
*
* @param snapshotTableDescriptor
* @param tableName
* @return cloned table descriptor
* @throws IOException
*/
public static HTableDescriptor cloneTableSchema(final HTableDescriptor snapshotTableDescriptor,
final byte[] tableName) throws IOException {
HTableDescriptor htd = new HTableDescriptor(tableName);
for (HColumnDescriptor hcd: snapshotTableDescriptor.getColumnFamilies()) {
htd.addFamily(hcd);
}
return htd;
}
}

View File

@ -0,0 +1,54 @@
/**
* 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.snapshot;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
/**
* Thrown when a snapshot could not be created due to a server-side error when taking the snapshot.
*/
@SuppressWarnings("serial")
public class SnapshotCreationException extends HBaseSnapshotException {
/**
* Used internally by the RPC engine to pass the exception back to the client.
* @param msg error message to pass back
*/
public SnapshotCreationException(String msg) {
super(msg);
}
/**
* Failure to create the specified snapshot
* @param msg reason why the snapshot couldn't be completed
* @param desc description of the snapshot attempted
*/
public SnapshotCreationException(String msg, SnapshotDescription desc) {
super(msg, desc);
}
/**
* Failure to create the specified snapshot due to an external cause
* @param msg reason why the snapshot couldn't be completed
* @param cause root cause of the failure
* @param desc description of the snapshot attempted
*/
public SnapshotCreationException(String msg, Throwable cause, SnapshotDescription desc) {
super(msg, cause, desc);
}
}

View File

@ -0,0 +1,360 @@
/**
* 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.snapshot;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* Utility class to help manage {@link SnapshotDescription SnapshotDesriptions}.
* <p>
* Snapshots are laid out on disk like this:
*
* <pre>
* /hbase/.snapshots
* /.tmp <---- working directory
* /[snapshot name] <----- completed snapshot
* </pre>
*
* A completed snapshot named 'completed' then looks like (multiple regions, servers, files, etc.
* signified by '...' on the same directory depth).
*
* <pre>
* /hbase/.snapshots/completed
* .snapshotinfo <--- Description of the snapshot
* .tableinfo <--- Copy of the tableinfo
* /.logs
* /[server_name]
* /... [log files]
* ...
* /[region name] <---- All the region's information
* .regioninfo <---- Copy of the HRegionInfo
* /[column family name]
* /[hfile name] <--- name of the hfile in the real region
* ...
* ...
* ...
* </pre>
*
* Utility methods in this class are useful for getting the correct locations for different parts of
* the snapshot, as well as moving completed snapshots into place (see
* {@link #completeSnapshot}, and writing the
* {@link SnapshotDescription} to the working snapshot directory.
*/
public class SnapshotDescriptionUtils {
/**
* Filter that only accepts completed snapshot directories
*/
public static class CompletedSnaphotDirectoriesFilter extends FSUtils.DirFilter {
/**
* @param fs
*/
public CompletedSnaphotDirectoriesFilter(FileSystem fs) {
super(fs);
}
@Override
public boolean accept(Path path) {
// only accept directories that aren't the tmp directory
if (super.accept(path)) {
return !path.getName().equals(SNAPSHOT_TMP_DIR_NAME);
}
return false;
}
}
private static final Log LOG = LogFactory.getLog(SnapshotDescriptionUtils.class);
/**
* Version of the fs layout for a snapshot. Future snapshots may have different file layouts,
* which we may need to read in differently.
*/
public static final int SNAPSHOT_LAYOUT_VERSION = 0;
// snapshot directory constants
/**
* The file contains the snapshot basic information and it is under the directory of a snapshot.
*/
public static final String SNAPSHOTINFO_FILE = ".snapshotinfo";
/** Temporary directory under the snapshot directory to store in-progress snapshots */
public static final String SNAPSHOT_TMP_DIR_NAME = ".tmp";
// snapshot operation values
/** Default value if no start time is specified */
public static final long NO_SNAPSHOT_START_TIME_SPECIFIED = 0;
public static final String MASTER_SNAPSHOT_TIMEOUT_MILLIS = "hbase.snapshot.master.timeout.millis";
/** By default, wait 60 seconds for a snapshot to complete */
public static final long DEFAULT_MAX_WAIT_TIME = 60000;
private SnapshotDescriptionUtils() {
// private constructor for utility class
}
/**
* Check to make sure that the description of the snapshot requested is valid
* @param snapshot description of the snapshot
* @throws IllegalArgumentException if the name of the snapshot or the name of the table to
* snapshot are not valid names.
*/
public static void assertSnapshotRequestIsValid(SnapshotDescription snapshot)
throws IllegalArgumentException {
// FIXME these method names is really bad - trunk will probably change
// .META. and -ROOT- snapshots are not allowed
if (HTableDescriptor.isMetaTable(Bytes.toBytes(snapshot.getTable()))) {
throw new IllegalArgumentException(".META. and -ROOT- snapshots are not allowed");
}
// make sure the snapshot name is valid
HTableDescriptor.isLegalTableName(Bytes.toBytes(snapshot.getName()));
// make sure the table name is valid
HTableDescriptor.isLegalTableName(Bytes.toBytes(snapshot.getTable()));
}
/**
* @param conf {@link Configuration} from which to check for the timeout
* @param type type of snapshot being taken
* @param defaultMaxWaitTime Default amount of time to wait, if none is in the configuration
* @return the max amount of time the master should wait for a snapshot to complete
*/
public static long getMaxMasterTimeout(Configuration conf, SnapshotDescription.Type type,
long defaultMaxWaitTime) {
String confKey;
switch (type) {
case DISABLED:
default:
confKey = MASTER_SNAPSHOT_TIMEOUT_MILLIS;
}
return conf.getLong(confKey, defaultMaxWaitTime);
}
/**
* Get the snapshot root directory. All the snapshots are kept under this directory, i.e.
* ${hbase.rootdir}/.snapshot
* @param rootDir hbase root directory
* @return the base directory in which all snapshots are kept
*/
public static Path getSnapshotRootDir(final Path rootDir) {
return new Path(rootDir, HConstants.SNAPSHOT_DIR_NAME);
}
/**
* Get the directory for a specified snapshot. This directory is a sub-directory of snapshot root
* directory and all the data files for a snapshot are kept under this directory.
* @param snapshot snapshot being taken
* @param rootDir hbase root directory
* @return the final directory for the completed snapshot
*/
public static Path getCompletedSnapshotDir(final SnapshotDescription snapshot, final Path rootDir) {
return getCompletedSnapshotDir(snapshot.getName(), rootDir);
}
/**
* Get the directory for a completed snapshot. This directory is a sub-directory of snapshot root
* directory and all the data files for a snapshot are kept under this directory.
* @param snapshotName name of the snapshot being taken
* @param rootDir hbase root directory
* @return the final directory for the completed snapshot
*/
public static Path getCompletedSnapshotDir(final String snapshotName, final Path rootDir) {
return getCompletedSnapshotDir(getSnapshotsDir(rootDir), snapshotName);
}
/**
* Get the general working directory for snapshots - where they are built, where they are
* temporarily copied on export, etc.
* @param rootDir root directory of the HBase installation
* @return Path to the snapshot tmp directory, relative to the passed root directory
*/
public static Path getWorkingSnapshotDir(final Path rootDir) {
return new Path(getSnapshotsDir(rootDir), SNAPSHOT_TMP_DIR_NAME);
}
/**
* Get the directory to build a snapshot, before it is finalized
* @param snapshot snapshot that will be built
* @param rootDir root directory of the hbase installation
* @return {@link Path} where one can build a snapshot
*/
public static Path getWorkingSnapshotDir(SnapshotDescription snapshot, final Path rootDir) {
return getCompletedSnapshotDir(getWorkingSnapshotDir(rootDir), snapshot.getName());
}
/**
* Get the directory to build a snapshot, before it is finalized
* @param snapshotName name of the snapshot
* @param rootDir root directory of the hbase installation
* @return {@link Path} where one can build a snapshot
*/
public static Path getWorkingSnapshotDir(String snapshotName, final Path rootDir) {
return getCompletedSnapshotDir(getWorkingSnapshotDir(rootDir), snapshotName);
}
/**
* Get the directory to store the snapshot instance
* @param snapshotsDir hbase-global directory for storing all snapshots
* @param snapshotName name of the snapshot to take
* @return
*/
private static final Path getCompletedSnapshotDir(final Path snapshotsDir, String snapshotName) {
return new Path(snapshotsDir, snapshotName);
}
/**
* @param rootDir hbase root directory
* @return the directory for all completed snapshots;
*/
public static final Path getSnapshotsDir(Path rootDir) {
return new Path(rootDir, HConstants.SNAPSHOT_DIR_NAME);
}
/**
* Convert the passed snapshot description into a 'full' snapshot description based on default
* parameters, if none have been supplied. This resolves any 'optional' parameters that aren't
* supplied to their default values.
* @param snapshot general snapshot descriptor
* @param conf Configuration to read configured snapshot defaults if snapshot is not complete
* @return a valid snapshot description
* @throws IllegalArgumentException if the {@link SnapshotDescription} is not a complete
* {@link SnapshotDescription}.
*/
public static SnapshotDescription validate(SnapshotDescription snapshot, Configuration conf)
throws IllegalArgumentException {
if (!snapshot.hasTable()) {
throw new IllegalArgumentException(
"Descriptor doesn't apply to a table, so we can't build it.");
}
// set the creation time, if one hasn't been set
long time = snapshot.getCreationTime();
if (time == SnapshotDescriptionUtils.NO_SNAPSHOT_START_TIME_SPECIFIED) {
time = EnvironmentEdgeManager.currentTimeMillis();
LOG.debug("Creation time not specified, setting to:" + time + " (current time:"
+ EnvironmentEdgeManager.currentTimeMillis() + ").");
SnapshotDescription.Builder builder = snapshot.toBuilder();
builder.setCreationTime(time);
snapshot = builder.build();
}
return snapshot;
}
/**
* Write the snapshot description into the working directory of a snapshot
* @param snapshot description of the snapshot being taken
* @param workingDir working directory of the snapshot
* @param fs {@link FileSystem} on which the snapshot should be taken
* @throws IOException if we can't reach the filesystem and the file cannot be cleaned up on
* failure
*/
public static void writeSnapshotInfo(SnapshotDescription snapshot, Path workingDir, FileSystem fs)
throws IOException {
FsPermission perms = FSUtils.getFilePermissions(fs, fs.getConf(),
HConstants.DATA_FILE_UMASK_KEY);
Path snapshotInfo = new Path(workingDir, SnapshotDescriptionUtils.SNAPSHOTINFO_FILE);
try {
FSDataOutputStream out = FSUtils.create(fs, snapshotInfo, perms, true);
try {
snapshot.writeTo(out);
} finally {
out.close();
}
} catch (IOException e) {
// if we get an exception, try to remove the snapshot info
if (!fs.delete(snapshotInfo, false)) {
String msg = "Couldn't delete snapshot info file: " + snapshotInfo;
LOG.error(msg);
throw new IOException(msg);
}
}
}
/**
* Read in the {@link SnapshotDescription} stored for the snapshot in the passed directory
* @param fs filesystem where the snapshot was taken
* @param snapshotDir directory where the snapshot was stored
* @return the stored snapshot description
* @throws CorruptedSnapshotException if the snapshot cannot be read
*/
public static SnapshotDescription readSnapshotInfo(FileSystem fs, Path snapshotDir)
throws CorruptedSnapshotException {
Path snapshotInfo = new Path(snapshotDir, SNAPSHOTINFO_FILE);
try {
FSDataInputStream in = null;
try {
in = fs.open(snapshotInfo);
return SnapshotDescription.parseFrom(in);
} finally {
if (in != null) in.close();
}
} catch (IOException e) {
throw new CorruptedSnapshotException("Couldn't read snapshot info from:" + snapshotInfo, e);
}
}
/**
* Move the finished snapshot to its final, publicly visible directory - this marks the snapshot
* as 'complete'.
* @param snapshot description of the snapshot being tabken
* @param rootdir root directory of the hbase installation
* @param workingDir directory where the in progress snapshot was built
* @param fs {@link FileSystem} where the snapshot was built
* @throws SnapshotCreationException if the snapshot could not be moved
* @throws IOException the filesystem could not be reached
*/
public static void completeSnapshot(SnapshotDescription snapshot, Path rootdir, Path workingDir,
FileSystem fs) throws SnapshotCreationException, IOException {
Path finishedDir = getCompletedSnapshotDir(snapshot, rootdir);
LOG.debug("Snapshot is done, just moving the snapshot from " + workingDir + " to "
+ finishedDir);
if (!fs.rename(workingDir, finishedDir)) {
throw new SnapshotCreationException("Failed to move working directory(" + workingDir
+ ") to completed directory(" + finishedDir + ").", snapshot);
}
}
/**
* Returns a single line (no \n) representation of snapshot metadata. Use this instead of
* {@link SnapshotDescription#toString()}. We don't replace SnapshotDescrpition's toString
* because it is auto-generated by protoc.
* @param ssd
* @return Single line string with a summary of the snapshot parameters
*/
public static String toString(SnapshotDescription ssd) {
if (ssd == null) {
return null;
}
return "{ ss=" + ssd.getName() + " table=" + ssd.getTable()
+ " type=" + ssd.getType() + " }";
}
}

View File

@ -0,0 +1,45 @@
/**
* 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.snapshot;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
/**
* Thrown when the server is looking for a snapshot but can't find the snapshot on the filesystem
*/
@SuppressWarnings("serial")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class SnapshotDoesNotExistException extends HBaseSnapshotException {
/**
* @param msg full description of the failure
*/
public SnapshotDoesNotExistException(String msg) {
super(msg);
}
/**
* @param desc expected snapshot to find
*/
public SnapshotDoesNotExistException(SnapshotDescription desc) {
super("Snapshot '" + desc.getName() +"' doesn't exist on the filesystem", desc);
}
}

View File

@ -0,0 +1,40 @@
/**
* 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.snapshot;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
/**
* Thrown when a snapshot exists but should not
*/
@SuppressWarnings("serial")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class SnapshotExistsException extends HBaseSnapshotException {
/**
* Failure due to the snapshot already existing
* @param msg full description of the failure
* @param desc snapshot that was attempted
*/
public SnapshotExistsException(String msg, SnapshotDescription desc) {
super(msg, desc);
}
}

View File

@ -0,0 +1,303 @@
/**
* 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.snapshot;
import java.io.IOException;
import java.io.FileNotFoundException;
import java.text.SimpleDateFormat;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.Date;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.HLogLink;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
/**
* Tool for dumping snapshot information.
* <ol>
* <li> Table Descriptor
* <li> Snapshot creation time, type, format version, ...
* <li> List of hfiles and hlogs
* <li> Stats about hfiles and logs sizes, percentage of shared with the source table, ...
* </ol>
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class SnapshotInfo extends Configured implements Tool {
private static final Log LOG = LogFactory.getLog(SnapshotInfo.class);
private FileSystem fs;
private Path rootDir;
private HTableDescriptor snapshotTableDesc;
private SnapshotDescription snapshotDesc;
private Path snapshotDir;
@Override
public int run(String[] args) throws IOException, InterruptedException {
String snapshotName = null;
boolean showSchema = false;
boolean showFiles = false;
boolean showStats = false;
// Process command line args
for (int i = 0; i < args.length; i++) {
String cmd = args[i];
try {
if (cmd.equals("-snapshot")) {
snapshotName = args[++i];
} else if (cmd.equals("-files")) {
showFiles = true;
} else if (cmd.equals("-stats")) {
showStats = true;
} else if (cmd.equals("-schema")) {
showSchema = true;
} else if (cmd.equals("-h") || cmd.equals("--help")) {
printUsageAndExit();
} else {
System.err.println("UNEXPECTED: " + cmd);
printUsageAndExit();
}
} catch (Exception e) {
printUsageAndExit();
}
}
if (snapshotName == null) {
System.err.println("Missing snapshot name!");
printUsageAndExit();
return 1;
}
Configuration conf = getConf();
fs = FileSystem.get(conf);
rootDir = FSUtils.getRootDir(conf);
// Load snapshot information
if (!loadSnapshotInfo(snapshotName)) {
System.err.println("Snapshot '" + snapshotName + "' not found!");
return 1;
}
printInfo();
if (showSchema) printSchema();
if (showFiles || showStats) printFiles(showFiles);
return 0;
}
/**
* Load snapshot info and table descriptor for the specified snapshot
* @param snapshotName name of the snapshot to load
* @return false if snapshot is not found
*/
private boolean loadSnapshotInfo(final String snapshotName) throws IOException {
snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
if (!fs.exists(snapshotDir)) {
LOG.warn("Snapshot '" + snapshotName + "' not found in: " + snapshotDir);
return false;
}
snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
snapshotTableDesc = FSTableDescriptors.getTableDescriptor(fs, snapshotDir);
return true;
}
/**
* Dump the {@link SnapshotDescription}
*/
private void printInfo() {
SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss");
System.out.println("Snapshot Info");
System.out.println("----------------------------------------");
System.out.println(" Name: " + snapshotDesc.getName());
System.out.println(" Type: " + snapshotDesc.getType());
System.out.println(" Table: " + snapshotDesc.getTable());
System.out.println(" Format: " + snapshotDesc.getVersion());
System.out.println("Created: " + df.format(new Date(snapshotDesc.getCreationTime())));
System.out.println();
}
/**
* Dump the {@link HTableDescriptor}
*/
private void printSchema() {
System.out.println("Table Descriptor");
System.out.println("----------------------------------------");
System.out.println(snapshotTableDesc.toString());
System.out.println();
}
/**
* Collect the hfiles and logs statistics of the snapshot and
* dump the file list if requested and the collected information.
*/
private void printFiles(final boolean showFiles) throws IOException {
final String table = snapshotDesc.getTable();
final Configuration conf = getConf();
if (showFiles) {
System.out.println("Snapshot Files");
System.out.println("----------------------------------------");
}
// Collect information about hfiles and logs in the snapshot
final AtomicInteger hfileArchiveCount = new AtomicInteger();
final AtomicInteger hfilesMissing = new AtomicInteger();
final AtomicInteger hfilesCount = new AtomicInteger();
final AtomicInteger logsMissing = new AtomicInteger();
final AtomicInteger logsCount = new AtomicInteger();
final AtomicLong hfileArchiveSize = new AtomicLong();
final AtomicLong hfileSize = new AtomicLong();
final AtomicLong logSize = new AtomicLong();
SnapshotReferenceUtil.visitReferencedFiles(fs, snapshotDir,
new SnapshotReferenceUtil.FileVisitor() {
public void storeFile (final String region, final String family, final String hfile)
throws IOException {
Path path = new Path(family, HFileLink.createHFileLinkName(table, region, hfile));
HFileLink link = new HFileLink(conf, path);
boolean inArchive = false;
long size = -1;
try {
if ((inArchive = fs.exists(link.getArchivePath()))) {
size = fs.getFileStatus(link.getArchivePath()).getLen();
hfileArchiveSize.addAndGet(size);
hfileArchiveCount.addAndGet(1);
} else {
size = link.getFileStatus(fs).getLen();
hfileSize.addAndGet(size);
hfilesCount.addAndGet(1);
}
} catch (FileNotFoundException e) {
hfilesMissing.addAndGet(1);
}
if (showFiles) {
System.out.printf("%8s %s/%s/%s/%s %s%n",
(size < 0 ? "-" : StringUtils.humanReadableInt(size)),
table, region, family, hfile,
(inArchive ? "(archive)" : (size < 0) ? "(NOT FOUND)" : ""));
}
}
public void recoveredEdits (final String region, final String logfile)
throws IOException {
Path path = SnapshotReferenceUtil.getRecoveredEdits(snapshotDir, region, logfile);
long size = fs.getFileStatus(path).getLen();
logSize.addAndGet(size);
logsCount.addAndGet(1);
if (showFiles) {
System.out.printf("%8s recovered.edits %s on region %s%n",
StringUtils.humanReadableInt(size), logfile, region);
}
}
public void logFile (final String server, final String logfile)
throws IOException {
HLogLink logLink = new HLogLink(conf, server, logfile);
long size = -1;
try {
size = logLink.getFileStatus(fs).getLen();
logSize.addAndGet(size);
logsCount.addAndGet(1);
} catch (FileNotFoundException e) {
logsMissing.addAndGet(1);
}
if (showFiles) {
System.out.printf("%8s log %s on server %s %s%n",
(size < 0 ? "-" : StringUtils.humanReadableInt(size)),
logfile, server,
(size < 0 ? "(NOT FOUND)" : ""));
}
}
});
// Dump the stats
System.out.println();
if (hfilesMissing.get() > 0 || logsMissing.get() > 0) {
System.out.println("**************************************************************");
System.out.printf("BAD SNAPSHOT: %d hfile(s) and %d log(s) missing.%n",
hfilesMissing.get(), logsMissing.get());
System.out.println("**************************************************************");
}
System.out.printf("%d HFiles (%d in archive), total size %s (%.2f%% %s shared with the source table)%n",
hfilesCount.get() + hfileArchiveCount.get(), hfileArchiveCount.get(),
StringUtils.humanReadableInt(hfileSize.get() + hfileArchiveSize.get()),
((float)hfileSize.get() / (hfileSize.get() + hfileArchiveSize.get())) * 100,
StringUtils.humanReadableInt(hfileSize.get())
);
System.out.printf("%d Logs, total size %s%n",
logsCount.get(), StringUtils.humanReadableInt(logSize.get()));
System.out.println();
}
private void printUsageAndExit() {
System.err.printf("Usage: bin/hbase %s [options]%n", getClass().getName());
System.err.println(" where [options] are:");
System.err.println(" -h|-help Show this help and exit.");
System.err.println(" -snapshot NAME Snapshot to examine.");
System.err.println(" -files Files and logs list.");
System.err.println(" -stats Files and logs stats.");
System.err.println(" -schema Describe the snapshotted table.");
System.err.println();
System.err.println("Examples:");
System.err.println(" hbase " + getClass() + " \\");
System.err.println(" -snapshot MySnapshot -files");
System.exit(1);
}
/**
* The guts of the {@link #main} method.
* Call this method to avoid the {@link #main(String[])} System.exit.
* @param args
* @return errCode
* @throws Exception
*/
static int innerMain(final String [] args) throws Exception {
return ToolRunner.run(HBaseConfiguration.create(), new SnapshotInfo(), args);
}
public static void main(String[] args) throws Exception {
System.exit(innerMain(args));
}
}

View File

@ -0,0 +1,198 @@
/**
* 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.snapshot;
import java.io.Closeable;
import java.io.IOException;
import java.util.Map;
import java.util.TreeMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.HLogLink;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
import org.apache.hadoop.hbase.util.Bytes;
/**
* If the snapshot has references to one or more log files,
* those must be split (each log contains multiple tables and regions)
* and must be placed in the region/recovered.edits folder.
* (recovered.edits files will be played on region startup)
*
* In case of Restore: the log can just be split in the recovered.edits folder.
* In case of Clone: each entry in the log must be modified to use the new region name.
* (region names are encoded with: tableName, startKey, regionIdTimeStamp)
*
* We can't use the normal split code, because the HLogKey contains the
* table name and the region name, and in case of "clone from snapshot"
* region name and table name will be different and must be replaced in
* the recovered.edits.
*/
@InterfaceAudience.Private
class SnapshotLogSplitter implements Closeable {
static final Log LOG = LogFactory.getLog(SnapshotLogSplitter.class);
private final class LogWriter implements Closeable {
private HLog.Writer writer;
private Path logFile;
private long seqId;
public LogWriter(final Configuration conf, final FileSystem fs,
final Path logDir, long seqId) throws IOException {
logFile = new Path(logDir, logFileName(seqId, true));
this.writer = HLogFactory.createWriter(fs, logFile, conf);
this.seqId = seqId;
}
public void close() throws IOException {
writer.close();
Path finalFile = new Path(logFile.getParent(), logFileName(seqId, false));
LOG.debug("LogWriter tmpLogFile=" + logFile + " -> logFile=" + finalFile);
fs.rename(logFile, finalFile);
}
public void append(final HLog.Entry entry) throws IOException {
writer.append(entry);
if (seqId < entry.getKey().getLogSeqNum()) {
seqId = entry.getKey().getLogSeqNum();
}
}
private String logFileName(long seqId, boolean temp) {
String fileName = String.format("%019d", seqId);
if (temp) fileName += HLog.RECOVERED_LOG_TMPFILE_SUFFIX;
return fileName;
}
}
private final Map<byte[], LogWriter> regionLogWriters =
new TreeMap<byte[], LogWriter>(Bytes.BYTES_COMPARATOR);
private final Map<byte[], byte[]> regionsMap;
private final Configuration conf;
private final byte[] snapshotTableName;
private final byte[] tableName;
private final Path tableDir;
private final FileSystem fs;
/**
* @params tableName snapshot table name
* @params regionsMap maps original region names to the new ones.
*/
public SnapshotLogSplitter(final Configuration conf, final FileSystem fs,
final Path tableDir, final byte[] snapshotTableName,
final Map<byte[], byte[]> regionsMap) {
this.regionsMap = regionsMap;
this.snapshotTableName = snapshotTableName;
this.tableName = Bytes.toBytes(tableDir.getName());
this.tableDir = tableDir;
this.conf = conf;
this.fs = fs;
}
public void close() throws IOException {
for (LogWriter writer: regionLogWriters.values()) {
writer.close();
}
}
public void splitLog(final String serverName, final String logfile) throws IOException {
LOG.debug("Restore log=" + logfile + " server=" + serverName +
" for snapshotTable=" + Bytes.toString(snapshotTableName) +
" to table=" + Bytes.toString(tableName));
splitLog(new HLogLink(conf, serverName, logfile).getAvailablePath(fs));
}
public void splitRecoveredEdit(final Path editPath) throws IOException {
LOG.debug("Restore recover.edits=" + editPath +
" for snapshotTable=" + Bytes.toString(snapshotTableName) +
" to table=" + Bytes.toString(tableName));
splitLog(editPath);
}
/**
* Split the snapshot HLog reference into regions recovered.edits.
*
* The HLogKey contains the table name and the region name,
* and they must be changed to the restored table names.
*
* @param logPath Snapshot HLog reference path
*/
public void splitLog(final Path logPath) throws IOException {
HLog.Reader log = HLogFactory.createReader(fs, logPath, conf);
try {
HLog.Entry entry;
LogWriter writer = null;
byte[] regionName = null;
byte[] newRegionName = null;
while ((entry = log.next()) != null) {
HLogKey key = entry.getKey();
// We're interested only in the snapshot table that we're restoring
if (!Bytes.equals(key.getTablename(), snapshotTableName)) continue;
// Writer for region.
if (!Bytes.equals(regionName, key.getEncodedRegionName())) {
regionName = key.getEncodedRegionName().clone();
// Get the new region name in case of clone, or use the original one
newRegionName = regionsMap.get(regionName);
if (newRegionName == null) newRegionName = regionName;
writer = getOrCreateWriter(newRegionName, key.getLogSeqNum());
LOG.debug("+ regionName=" + Bytes.toString(regionName));
}
// Append Entry
key = new HLogKey(newRegionName, tableName,
key.getLogSeqNum(), key.getWriteTime(), key.getClusterId());
writer.append(new HLog.Entry(key, entry.getEdit()));
}
} catch (IOException e) {
LOG.warn("Something wrong during the log split", e);
} finally {
log.close();
}
}
/**
* Create a LogWriter for specified region if not already created.
*/
private LogWriter getOrCreateWriter(final byte[] regionName, long seqId) throws IOException {
LogWriter writer = regionLogWriters.get(regionName);
if (writer == null) {
Path regionDir = HRegion.getRegionDir(tableDir, Bytes.toString(regionName));
Path dir = HLogUtil.getRegionDirRecoveredEditsDir(regionDir);
fs.mkdirs(dir);
writer = new LogWriter(conf, fs, dir, seqId);
regionLogWriters.put(regionName, writer);
}
return(writer);
}
}

View File

@ -0,0 +1,245 @@
/**
* 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.snapshot;
import java.io.IOException;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.FSVisitor;
/**
* Utility methods for interacting with the snapshot referenced files.
*/
@InterfaceAudience.Private
public final class SnapshotReferenceUtil {
public interface FileVisitor extends FSVisitor.StoreFileVisitor,
FSVisitor.RecoveredEditsVisitor, FSVisitor.LogFileVisitor {
}
private SnapshotReferenceUtil() {
// private constructor for utility class
}
/**
* Get log directory for a server in a snapshot.
*
* @param snapshotDir directory where the specific snapshot is stored
* @param serverName name of the parent regionserver for the log files
* @return path to the log home directory for the archive files.
*/
public static Path getLogsDir(Path snapshotDir, String serverName) {
return new Path(snapshotDir, HLogUtil.getHLogDirectoryName(serverName));
}
/**
* Get the snapshotted recovered.edits dir for the specified region.
*
* @param snapshotDir directory where the specific snapshot is stored
* @param regionName name of the region
* @return path to the recovered.edits directory for the specified region files.
*/
public static Path getRecoveredEditsDir(Path snapshotDir, String regionName) {
return HLogUtil.getRegionDirRecoveredEditsDir(new Path(snapshotDir, regionName));
}
/**
* Get the snapshot recovered.edits file
*
* @param snapshotDir directory where the specific snapshot is stored
* @param regionName name of the region
* @param logfile name of the edit file
* @return full path of the log file for the specified region files.
*/
public static Path getRecoveredEdits(Path snapshotDir, String regionName, String logfile) {
return new Path(getRecoveredEditsDir(snapshotDir, regionName), logfile);
}
/**
* Iterate over the snapshot store files, restored.edits and logs
*
* @param fs {@link FileSystem}
* @param snapshotDir {@link Path} to the Snapshot directory
* @param visitor callback object to get the referenced files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitReferencedFiles(final FileSystem fs, final Path snapshotDir,
final FileVisitor visitor) throws IOException {
visitTableStoreFiles(fs, snapshotDir, visitor);
visitRecoveredEdits(fs, snapshotDir, visitor);
visitLogFiles(fs, snapshotDir, visitor);
}
/**
* Iterate over the snapshot store files
*
* @param fs {@link FileSystem}
* @param snapshotDir {@link Path} to the Snapshot directory
* @param visitor callback object to get the store files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitTableStoreFiles(final FileSystem fs, final Path snapshotDir,
final FSVisitor.StoreFileVisitor visitor) throws IOException {
FSVisitor.visitTableStoreFiles(fs, snapshotDir, visitor);
}
/**
* Iterate over the snapshot store files in the specified region
*
* @param fs {@link FileSystem}
* @param regionDir {@link Path} to the Snapshot region directory
* @param visitor callback object to get the store files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitRegionStoreFiles(final FileSystem fs, final Path regionDir,
final FSVisitor.StoreFileVisitor visitor) throws IOException {
FSVisitor.visitRegionStoreFiles(fs, regionDir, visitor);
}
/**
* Iterate over the snapshot recovered.edits
*
* @param fs {@link FileSystem}
* @param snapshotDir {@link Path} to the Snapshot directory
* @param visitor callback object to get the recovered.edits files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitRecoveredEdits(final FileSystem fs, final Path snapshotDir,
final FSVisitor.RecoveredEditsVisitor visitor) throws IOException {
FSVisitor.visitTableRecoveredEdits(fs, snapshotDir, visitor);
}
/**
* Iterate over the snapshot log files
*
* @param fs {@link FileSystem}
* @param snapshotDir {@link Path} to the Snapshot directory
* @param visitor callback object to get the log files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitLogFiles(final FileSystem fs, final Path snapshotDir,
final FSVisitor.LogFileVisitor visitor) throws IOException {
FSVisitor.visitLogFiles(fs, snapshotDir, visitor);
}
/**
* Returns the set of region names available in the snapshot.
*
* @param fs {@link FileSystem}
* @param snapshotDir {@link Path} to the Snapshot directory
* @throws IOException if an error occurred while scanning the directory
* @return the set of the regions contained in the snapshot
*/
public static Set<String> getSnapshotRegionNames(final FileSystem fs, final Path snapshotDir)
throws IOException {
FileStatus[] regionDirs = FSUtils.listStatus(fs, snapshotDir, new FSUtils.RegionDirFilter(fs));
if (regionDirs == null) return null;
Set<String> regions = new HashSet<String>();
for (FileStatus regionDir: regionDirs) {
regions.add(regionDir.getPath().getName());
}
return regions;
}
/**
* Get the list of hfiles for the specified snapshot region.
* NOTE: The current implementation keeps one empty file per HFile in the region.
* The file name matches the one in the original table, and by reconstructing
* the path you can quickly jump to the referenced file.
*
* @param fs {@link FileSystem}
* @param snapshotRegionDir {@link Path} to the Snapshot region directory
* @return Map of hfiles per family, the key is the family name and values are hfile names
* @throws IOException if an error occurred while scanning the directory
*/
public static Map<String, List<String>> getRegionHFileReferences(final FileSystem fs,
final Path snapshotRegionDir) throws IOException {
final Map<String, List<String>> familyFiles = new TreeMap<String, List<String>>();
visitRegionStoreFiles(fs, snapshotRegionDir,
new FSVisitor.StoreFileVisitor() {
public void storeFile (final String region, final String family, final String hfile)
throws IOException {
List<String> hfiles = familyFiles.get(family);
if (hfiles == null) {
hfiles = new LinkedList<String>();
familyFiles.put(family, hfiles);
}
hfiles.add(hfile);
}
});
return familyFiles;
}
/**
* Returns the store file names in the snapshot.
*
* @param fs {@link FileSystem}
* @param snapshotDir {@link Path} to the Snapshot directory
* @throws IOException if an error occurred while scanning the directory
* @return the names of hfiles in the specified snaphot
*/
public static Set<String> getHFileNames(final FileSystem fs, final Path snapshotDir)
throws IOException {
final Set<String> names = new HashSet<String>();
visitTableStoreFiles(fs, snapshotDir, new FSVisitor.StoreFileVisitor() {
public void storeFile (final String region, final String family, final String hfile)
throws IOException {
if (HFileLink.isHFileLink(hfile)) {
names.add(HFileLink.getReferencedHFileName(hfile));
} else {
names.add(hfile);
}
}
});
return names;
}
/**
* Returns the log file names available in the snapshot.
*
* @param fs {@link FileSystem}
* @param snapshotDir {@link Path} to the Snapshot directory
* @throws IOException if an error occurred while scanning the directory
* @return the names of hlogs in the specified snaphot
*/
public static Set<String> getHLogNames(final FileSystem fs, final Path snapshotDir)
throws IOException {
final Set<String> names = new HashSet<String>();
visitLogFiles(fs, snapshotDir, new FSVisitor.LogFileVisitor() {
public void logFile (final String server, final String logfile) throws IOException {
names.add(logfile);
}
});
return names;
}
}

View File

@ -0,0 +1,67 @@
/**
* 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.snapshot;
import java.util.concurrent.Callable;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
/**
* General snapshot operation taken on a regionserver
*/
@InterfaceAudience.Private
public abstract class SnapshotTask implements ForeignExceptionSnare, Callable<Void>{
protected final SnapshotDescription snapshot;
protected final ForeignExceptionDispatcher errorMonitor;
/**
* @param snapshot Description of the snapshot we are going to operate on
* @param monitor listener interested in failures to the snapshot caused by this operation
*/
public SnapshotTask(SnapshotDescription snapshot, ForeignExceptionDispatcher monitor) {
assert monitor != null : "ForeignExceptionDispatcher must not be null!";
assert snapshot != null : "SnapshotDescription must not be null!";
this.snapshot = snapshot;
this.errorMonitor = monitor;
}
public void snapshotFailure(String message, Exception e) {
ForeignException ee = new ForeignException(message, e);
errorMonitor.receive(ee);
}
@Override
public void rethrowException() throws ForeignException {
this.errorMonitor.rethrowException();
}
@Override
public boolean hasException() {
return this.errorMonitor.hasException();
}
@Override
public ForeignException getException() {
return this.errorMonitor.getException();
}
}

View File

@ -0,0 +1,73 @@
/**
* 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.snapshot;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
/**
* Copy the table info into the snapshot directory
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class TableInfoCopyTask extends SnapshotTask {
public static final Log LOG = LogFactory.getLog(TableInfoCopyTask.class);
private final FileSystem fs;
private final Path rootDir;
/**
* Copy the table info for the given table into the snapshot
* @param monitor listen for errors while running the snapshot
* @param snapshot snapshot for which we are copying the table info
* @param fs {@link FileSystem} where the tableinfo is stored (and where the copy will be written)
* @param rootDir root of the {@link FileSystem} where the tableinfo is stored
*/
public TableInfoCopyTask(ForeignExceptionDispatcher monitor,
SnapshotDescription snapshot, FileSystem fs, Path rootDir) {
super(snapshot, monitor);
this.rootDir = rootDir;
this.fs = fs;
}
@Override
public Void call() throws Exception {
LOG.debug("Running table info copy.");
this.rethrowException();
LOG.debug("Attempting to copy table info for snapshot:"
+ SnapshotDescriptionUtils.toString(this.snapshot));
// get the HTable descriptor
HTableDescriptor orig = FSTableDescriptors.getTableDescriptor(fs, rootDir,
Bytes.toBytes(this.snapshot.getTable()));
this.rethrowException();
// write a copy of descriptor to the snapshot directory
Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
FSTableDescriptors.createTableDescriptorForTableDirectory(fs, snapshotDir, orig, false);
LOG.debug("Finished copying tableinfo.");
return null;
}
}

View File

@ -0,0 +1,51 @@
/**
* 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.snapshot;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Thrown if a table should be online/offline but is partially open
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class TablePartiallyOpenException extends IOException {
private static final long serialVersionUID = 3571982660065058361L;
public TablePartiallyOpenException() {
super();
}
/**
* @param s message
*/
public TablePartiallyOpenException(String s) {
super(s);
}
/**
* @param tableName Name of table that is partial open
*/
public TablePartiallyOpenException(byte[] tableName) {
this(Bytes.toString(tableName));
}
}

View File

@ -0,0 +1,322 @@
/**
* 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.snapshot;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map.Entry;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionListener;
import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.Multimap;
/**
* Utilities for useful when taking a snapshot
*/
public class TakeSnapshotUtils {
private static final Log LOG = LogFactory.getLog(TakeSnapshotUtils.class);
private TakeSnapshotUtils() {
// private constructor for util class
}
/**
* Get the per-region snapshot description location.
* <p>
* Under the per-snapshot directory, specific files per-region are kept in a similar layout as per
* the current directory layout.
* @param desc description of the snapshot
* @param rootDir root directory for the hbase installation
* @param regionName encoded name of the region (see {@link HRegionInfo#encodeRegionName(byte[])})
* @return path to the per-region directory for the snapshot
*/
public static Path getRegionSnapshotDirectory(SnapshotDescription desc, Path rootDir,
String regionName) {
Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir);
return HRegion.getRegionDir(snapshotDir, regionName);
}
/**
* Get the home directory for store-level snapshot files.
* <p>
* Specific files per store are kept in a similar layout as per the current directory layout.
* @param regionDir snapshot directory for the parent region, <b>not</b> the standard region
* directory. See {@link #getRegionSnapshotDirectory}
* @param family name of the store to snapshot
* @return path to the snapshot home directory for the store/family
*/
public static Path getStoreSnapshotDirectory(Path regionDir, String family) {
return HStore.getStoreHomedir(regionDir, Bytes.toBytes(family));
}
/**
* Get the snapshot directory for each family to be added to the the snapshot
* @param snapshot description of the snapshot being take
* @param snapshotRegionDir directory in the snapshot where the region directory information
* should be stored
* @param families families to be added (can be null)
* @return paths to the snapshot directory for each family, in the same order as the families
* passed in
*/
public static List<Path> getFamilySnapshotDirectories(SnapshotDescription snapshot,
Path snapshotRegionDir, FileStatus[] families) {
if (families == null || families.length == 0) return Collections.emptyList();
List<Path> familyDirs = new ArrayList<Path>(families.length);
for (FileStatus family : families) {
// build the reference directory name
familyDirs.add(getStoreSnapshotDirectory(snapshotRegionDir, family.getPath().getName()));
}
return familyDirs;
}
/**
* Create a snapshot timer for the master which notifies the monitor when an error occurs
* @param snapshot snapshot to monitor
* @param conf configuration to use when getting the max snapshot life
* @param monitor monitor to notify when the snapshot life expires
* @return the timer to use update to signal the start and end of the snapshot
*/
public static TimeoutExceptionInjector getMasterTimerAndBindToMonitor(SnapshotDescription snapshot,
Configuration conf, ForeignExceptionListener monitor) {
long maxTime = SnapshotDescriptionUtils.getMaxMasterTimeout(conf, snapshot.getType(),
SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
return new TimeoutExceptionInjector(monitor, maxTime);
}
/**
* Verify that all the expected logs got referenced
* @param fs filesystem where the logs live
* @param logsDir original logs directory
* @param serverNames names of the servers that involved in the snapshot
* @param snapshot description of the snapshot being taken
* @param snapshotLogDir directory for logs in the snapshot
* @throws IOException
*/
public static void verifyAllLogsGotReferenced(FileSystem fs, Path logsDir,
Set<String> serverNames, SnapshotDescription snapshot, Path snapshotLogDir)
throws IOException {
assertTrue(snapshot, "Logs directory doesn't exist in snapshot", fs.exists(logsDir));
// for each of the server log dirs, make sure it matches the main directory
Multimap<String, String> snapshotLogs = getMapOfServersAndLogs(fs, snapshotLogDir, serverNames);
Multimap<String, String> realLogs = getMapOfServersAndLogs(fs, logsDir, serverNames);
if (realLogs != null) {
assertNotNull(snapshot, "No server logs added to snapshot", snapshotLogs);
} else {
assertNull(snapshot, "Snapshotted server logs that don't exist", snapshotLogs);
}
// check the number of servers
Set<Entry<String, Collection<String>>> serverEntries = realLogs.asMap().entrySet();
Set<Entry<String, Collection<String>>> snapshotEntries = snapshotLogs.asMap().entrySet();
assertEquals(snapshot, "Not the same number of snapshot and original server logs directories",
serverEntries.size(), snapshotEntries.size());
// verify we snapshotted each of the log files
for (Entry<String, Collection<String>> serverLogs : serverEntries) {
// if the server is not the snapshot, skip checking its logs
if (!serverNames.contains(serverLogs.getKey())) continue;
Collection<String> snapshotServerLogs = snapshotLogs.get(serverLogs.getKey());
assertNotNull(snapshot, "Snapshots missing logs for server:" + serverLogs.getKey(),
snapshotServerLogs);
// check each of the log files
assertEquals(snapshot,
"Didn't reference all the log files for server:" + serverLogs.getKey(), serverLogs
.getValue().size(), snapshotServerLogs.size());
for (String log : serverLogs.getValue()) {
assertTrue(snapshot, "Snapshot logs didn't include " + log,
snapshotServerLogs.contains(log));
}
}
}
/**
* Verify one of a snapshot's region's recovered.edits, has been at the surface (file names,
* length), match the original directory.
* @param fs filesystem on which the snapshot had been taken
* @param rootDir full path to the root hbase directory
* @param regionInfo info for the region
* @param snapshot description of the snapshot that was taken
* @throws IOException if there is an unexpected error talking to the filesystem
*/
public static void verifyRecoveredEdits(FileSystem fs, Path rootDir, HRegionInfo regionInfo,
SnapshotDescription snapshot) throws IOException {
Path regionDir = HRegion.getRegionDir(rootDir, regionInfo);
Path editsDir = HLogUtil.getRegionDirRecoveredEditsDir(regionDir);
Path snapshotRegionDir = TakeSnapshotUtils.getRegionSnapshotDirectory(snapshot, rootDir,
regionInfo.getEncodedName());
Path snapshotEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(snapshotRegionDir);
FileStatus[] edits = FSUtils.listStatus(fs, editsDir);
FileStatus[] snapshotEdits = FSUtils.listStatus(fs, snapshotEditsDir);
if (edits == null) {
assertNull(snapshot, "Snapshot has edits but table doesn't", snapshotEdits);
return;
}
assertNotNull(snapshot, "Table has edits, but snapshot doesn't", snapshotEdits);
// check each of the files
assertEquals(snapshot, "Not same number of edits in snapshot as table", edits.length,
snapshotEdits.length);
// make sure we have a file with the same name as the original
// it would be really expensive to verify the content matches the original
for (FileStatus edit : edits) {
for (FileStatus sEdit : snapshotEdits) {
if (sEdit.getPath().equals(edit.getPath())) {
assertEquals(snapshot, "Snapshot file" + sEdit.getPath()
+ " length not equal to the original: " + edit.getPath(), edit.getLen(),
sEdit.getLen());
break;
}
}
assertTrue(snapshot, "No edit in snapshot with name:" + edit.getPath(), false);
}
}
private static void assertNull(SnapshotDescription snapshot, String msg, Object isNull)
throws CorruptedSnapshotException {
if (isNull != null) {
throw new CorruptedSnapshotException(msg + ", Expected " + isNull + " to be null.", snapshot);
}
}
private static void assertNotNull(SnapshotDescription snapshot, String msg, Object notNull)
throws CorruptedSnapshotException {
if (notNull == null) {
throw new CorruptedSnapshotException(msg + ", Expected object to not be null, but was null.",
snapshot);
}
}
private static void assertTrue(SnapshotDescription snapshot, String msg, boolean isTrue)
throws CorruptedSnapshotException {
if (!isTrue) {
throw new CorruptedSnapshotException(msg + ", Expected true, but was false", snapshot);
}
}
/**
* Assert that the expect matches the gotten amount
* @param msg message to add the to exception
* @param expected
* @param gotten
* @throws CorruptedSnapshotException thrown if the two elements don't match
*/
private static void assertEquals(SnapshotDescription snapshot, String msg, int expected,
int gotten) throws CorruptedSnapshotException {
if (expected != gotten) {
throw new CorruptedSnapshotException(msg + ". Expected:" + expected + ", got:" + gotten,
snapshot);
}
}
/**
* Assert that the expect matches the gotten amount
* @param msg message to add the to exception
* @param expected
* @param gotten
* @throws CorruptedSnapshotException thrown if the two elements don't match
*/
private static void assertEquals(SnapshotDescription snapshot, String msg, long expected,
long gotten) throws CorruptedSnapshotException {
if (expected != gotten) {
throw new CorruptedSnapshotException(msg + ". Expected:" + expected + ", got:" + gotten,
snapshot);
}
}
/**
* @param logdir
* @param toInclude list of servers to include. If empty or null, returns all servers
* @return maps of servers to all their log files. If there is no log directory, returns
* <tt>null</tt>
*/
private static Multimap<String, String> getMapOfServersAndLogs(FileSystem fs, Path logdir,
Collection<String> toInclude) throws IOException {
// create a path filter based on the passed directories to include
PathFilter filter = toInclude == null || toInclude.size() == 0 ? null
: new MatchesDirectoryNames(toInclude);
// get all the expected directories
FileStatus[] serverLogDirs = FSUtils.listStatus(fs, logdir, filter);
if (serverLogDirs == null) return null;
// map those into a multimap of servername -> [log files]
Multimap<String, String> map = HashMultimap.create();
for (FileStatus server : serverLogDirs) {
FileStatus[] serverLogs = FSUtils.listStatus(fs, server.getPath(), null);
if (serverLogs == null) continue;
for (FileStatus log : serverLogs) {
map.put(server.getPath().getName(), log.getPath().getName());
}
}
return map;
}
/**
* Path filter that only accepts paths where that have a {@link Path#getName()} that is contained
* in the specified collection.
*/
private static class MatchesDirectoryNames implements PathFilter {
Collection<String> paths;
public MatchesDirectoryNames(Collection<String> dirNames) {
this.paths = dirNames;
}
@Override
public boolean accept(Path path) {
return paths.contains(path.getName());
}
}
/**
* Get the log directory for a specific snapshot
* @param snapshotDir directory where the specific snapshot will be store
* @param serverName name of the parent regionserver for the log files
* @return path to the log home directory for the archive files.
*/
public static Path getSnapshotHLogsDir(Path snapshotDir, String serverName) {
return new Path(snapshotDir, HLogUtil.getHLogDirectoryName(serverName));
}
}

View File

@ -0,0 +1,42 @@
/**
* 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.snapshot;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Exception thrown when we get a request for a snapshot we don't recognize.
*/
@SuppressWarnings("serial")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class UnknownSnapshotException extends HBaseSnapshotException {
/**
* @param msg full information about the failure
*/
public UnknownSnapshotException(String msg) {
super(msg);
}
public UnknownSnapshotException(String msg, Exception e) {
super(msg, e);
}
}

View File

@ -607,7 +607,25 @@ public class FSTableDescriptors implements TableDescriptors {
public static boolean createTableDescriptor(FileSystem fs, Path rootdir,
HTableDescriptor htableDescriptor, boolean forceCreation)
throws IOException {
FileStatus status = getTableInfoPath(fs, rootdir, htableDescriptor.getNameAsString());
Path tabledir = FSUtils.getTablePath(rootdir, htableDescriptor.getNameAsString());
return createTableDescriptorForTableDirectory(fs, tabledir, htableDescriptor, forceCreation);
}
/**
* Create a new HTableDescriptor in HDFS in the specified table directory. Happens when we create
* a new table or snapshot a table.
* @param fs filesystem where the descriptor should be written
* @param tabledir directory under which we should write the file
* @param htableDescriptor description of the table to write
* @param forceCreation if <tt>true</tt>,then even if previous table descriptor is present it will
* be overwritten
* @return <tt>true</tt> if the we successfully created the file, <tt>false</tt> if the file
* already exists and we weren't forcing the descriptor creation.
* @throws IOException if a filesystem error occurs
*/
public static boolean createTableDescriptorForTableDirectory(FileSystem fs, Path tabledir,
HTableDescriptor htableDescriptor, boolean forceCreation) throws IOException {
FileStatus status = getTableInfoPath(fs, tabledir);
if (status != null) {
LOG.info("Current tableInfoPath = " + status.getPath());
if (!forceCreation) {
@ -617,8 +635,7 @@ public class FSTableDescriptors implements TableDescriptors {
}
}
}
Path p = writeTableDescriptor(fs, htableDescriptor,
FSUtils.getTablePath(rootdir, htableDescriptor.getNameAsString()), status);
Path p = writeTableDescriptor(fs, htableDescriptor, tabledir, status);
return p != null;
}
}

View File

@ -946,6 +946,27 @@ public abstract class FSUtils {
return true;
}
/**
* A {@link PathFilter} that returns only regular files.
*/
static class FileFilter implements PathFilter {
private final FileSystem fs;
public FileFilter(final FileSystem fs) {
this.fs = fs;
}
@Override
public boolean accept(Path p) {
try {
return fs.isFile(p);
} catch (IOException e) {
LOG.debug("unable to verify if path=" + p + " is a regular file", e);
return false;
}
}
}
/**
* A {@link PathFilter} that returns directories.
*/
@ -956,13 +977,14 @@ public abstract class FSUtils {
this.fs = fs;
}
@Override
public boolean accept(Path p) {
boolean isValid = false;
try {
if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(p.toString())) {
isValid = false;
} else {
isValid = this.fs.getFileStatus(p).isDir();
isValid = fs.getFileStatus(p).isDir();
}
} catch (IOException e) {
LOG.warn("An error occurred while verifying if [" + p.toString() +
@ -1310,19 +1332,6 @@ public abstract class FSUtils {
return fs.exists(path);
}
/**
* Log the current state of the filesystem from a certain root directory
* @param fs filesystem to investigate
* @param root root file/directory to start logging from
* @param LOG log to output information
* @throws IOException if an unexpected exception occurs
*/
public static void logFileSystemState(final FileSystem fs, final Path root, Log LOG)
throws IOException {
LOG.debug("Current file system:");
logFSTree(LOG, fs, root, "|-");
}
/**
* Throw an exception if an action is not permitted by a user on a file.
*
@ -1359,6 +1368,19 @@ public abstract class FSUtils {
return false;
}
/**
* Log the current state of the filesystem from a certain root directory
* @param fs filesystem to investigate
* @param root root file/directory to start logging from
* @param LOG log to output information
* @throws IOException if an unexpected exception occurs
*/
public static void logFileSystemState(final FileSystem fs, final Path root, Log LOG)
throws IOException {
LOG.debug("Current file system:");
logFSTree(LOG, fs, root, "|-");
}
/**
* Recursive helper to log the state of the FS
*

View File

@ -0,0 +1,194 @@
/**
*
* 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.util;
import java.io.IOException;
import java.util.NavigableSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* Utility methods for interacting with the hbase.root file system.
*/
@InterfaceAudience.Private
public final class FSVisitor {
private static final Log LOG = LogFactory.getLog(FSVisitor.class);
public interface StoreFileVisitor {
void storeFile(final String region, final String family, final String hfileName)
throws IOException;
}
public interface RecoveredEditsVisitor {
void recoveredEdits (final String region, final String logfile)
throws IOException;
}
public interface LogFileVisitor {
void logFile (final String server, final String logfile)
throws IOException;
}
private FSVisitor() {
// private constructor for utility class
}
/**
* Iterate over the table store files
*
* @param fs {@link FileSystem}
* @param tableDir {@link Path} to the table directory
* @param visitor callback object to get the store files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitTableStoreFiles(final FileSystem fs, final Path tableDir,
final StoreFileVisitor visitor) throws IOException {
FileStatus[] regions = FSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs));
if (regions == null) {
LOG.info("No regions under directory:" + tableDir);
return;
}
for (FileStatus region: regions) {
visitRegionStoreFiles(fs, region.getPath(), visitor);
}
}
/**
* Iterate over the region store files
*
* @param fs {@link FileSystem}
* @param regionDir {@link Path} to the region directory
* @param visitor callback object to get the store files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitRegionStoreFiles(final FileSystem fs, final Path regionDir,
final StoreFileVisitor visitor) throws IOException {
FileStatus[] families = FSUtils.listStatus(fs, regionDir, new FSUtils.FamilyDirFilter(fs));
if (families == null) {
LOG.info("No families under region directory:" + regionDir);
return;
}
PathFilter fileFilter = new FSUtils.FileFilter(fs);
for (FileStatus family: families) {
Path familyDir = family.getPath();
String familyName = familyDir.getName();
// get all the storeFiles in the family
FileStatus[] storeFiles = FSUtils.listStatus(fs, familyDir, fileFilter);
if (storeFiles == null) {
LOG.debug("No hfiles found for family: " + familyDir + ", skipping.");
continue;
}
for (FileStatus hfile: storeFiles) {
Path hfilePath = hfile.getPath();
visitor.storeFile(regionDir.getName(), familyName, hfilePath.getName());
}
}
}
/**
* Iterate over each region in the table and inform about recovered.edits
*
* @param fs {@link FileSystem}
* @param tableDir {@link Path} to the table directory
* @param visitor callback object to get the recovered.edits files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitTableRecoveredEdits(final FileSystem fs, final Path tableDir,
final FSVisitor.RecoveredEditsVisitor visitor) throws IOException {
FileStatus[] regions = FSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs));
if (regions == null) {
LOG.info("No regions under directory:" + tableDir);
return;
}
for (FileStatus region: regions) {
visitRegionRecoveredEdits(fs, region.getPath(), visitor);
}
}
/**
* Iterate over recovered.edits of the specified region
*
* @param fs {@link FileSystem}
* @param regionDir {@link Path} to the Region directory
* @param visitor callback object to get the recovered.edits files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitRegionRecoveredEdits(final FileSystem fs, final Path regionDir,
final FSVisitor.RecoveredEditsVisitor visitor) throws IOException {
NavigableSet<Path> files = HLogUtil.getSplitEditFilesSorted(fs, regionDir);
if (files == null || files.size() == 0) return;
for (Path source: files) {
// check to see if the file is zero length, in which case we can skip it
FileStatus stat = fs.getFileStatus(source);
if (stat.getLen() <= 0) continue;
visitor.recoveredEdits(regionDir.getName(), source.getName());
}
}
/**
* Iterate over hbase log files
*
* @param fs {@link FileSystem}
* @param rootDir {@link Path} to the HBase root folder
* @param visitor callback object to get the log files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitLogFiles(final FileSystem fs, final Path rootDir,
final LogFileVisitor visitor) throws IOException {
Path logsDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
FileStatus[] logServerDirs = FSUtils.listStatus(fs, logsDir);
if (logServerDirs == null) {
LOG.info("No logs under directory:" + logsDir);
return;
}
for (FileStatus serverLogs: logServerDirs) {
String serverName = serverLogs.getPath().getName();
FileStatus[] hlogs = FSUtils.listStatus(fs, serverLogs.getPath());
if (hlogs == null) {
LOG.debug("No hfiles found for server: " + serverName + ", skipping.");
continue;
}
for (FileStatus hlogRef: hlogs) {
visitor.logFile(serverName, hlogRef.getPath().getName());
}
}
}
}

View File

@ -98,6 +98,24 @@ public class HFileArchiveUtil {
return HRegion.getRegionDir(archiveDir, encodedRegionName);
}
/**
* Get the archive directory for a given region under the specified table
* @param rootdir {@link Path} to the root directory where hbase files are stored (for building
* the archive path)
* @param tabledir the original table directory. Cannot be null.
* @param regiondir the path to the region directory. Cannot be null.
* @return {@link Path} to the directory to archive the given region, or <tt>null</tt> if it
* should not be archived
*/
public static Path getRegionArchiveDir(Path rootdir, Path tabledir, Path regiondir) {
// get the archive directory for a table
Path archiveDir = getTableArchivePath(rootdir, tabledir.getName());
// then add on the region path under the archive
String encodedRegionName = regiondir.getName();
return HRegion.getRegionDir(archiveDir, encodedRegionName);
}
/**
* Get the path to the table archive directory based on the configured archive directory.
* <p>
@ -109,7 +127,22 @@ public class HFileArchiveUtil {
*/
public static Path getTableArchivePath(Path tabledir) {
Path root = tabledir.getParent();
return new Path(new Path(root,HConstants.HFILE_ARCHIVE_DIRECTORY), tabledir.getName());
return getTableArchivePath(root, tabledir.getName());
}
/**
* Get the path to the table archive directory based on the configured archive directory.
* <p>
* Get the path to the table's archive directory.
* <p>
* Generally of the form: /hbase/.archive/[tablename]
* @param rootdir {@link Path} to the root directory where hbase files are stored (for building
* the archive path)
* @param tableName Name of the table to be archived. Cannot be null.
* @return {@link Path} to the archive directory for the table
*/
public static Path getTableArchivePath(final Path rootdir, final String tableName) {
return new Path(getArchivePath(rootdir), tableName);
}
/**
@ -133,6 +166,16 @@ public class HFileArchiveUtil {
* @throws IOException if an unexpected error occurs
*/
public static Path getArchivePath(Configuration conf) throws IOException {
return new Path(FSUtils.getRootDir(conf), HConstants.HFILE_ARCHIVE_DIRECTORY);
return getArchivePath(FSUtils.getRootDir(conf));
}
/**
* Get the full path to the archive directory on the configured {@link FileSystem}
* @param rootdir {@link Path} to the root directory where hbase files are stored (for building
* the archive path)
* @return the full {@link Path} to the archive directory, as defined by the configuration
*/
private static Path getArchivePath(final Path rootdir) {
return new Path(rootdir, HConstants.HFILE_ARCHIVE_DIRECTORY);
}
}

View File

@ -235,22 +235,25 @@ public class JVMClusterUtil {
}
}
// Do active after.
if (activeMaster != null) activeMaster.master.shutdown();
if (activeMaster != null)
activeMaster.master.shutdown();
}
boolean noWait = false;
boolean wasInterrupted = false;
final long maxTime = System.currentTimeMillis() + 120 * 1000;
if (regionservers != null) {
// first try nicely.
for (RegionServerThread t : regionservers) {
t.getRegionServer().stop("Shutdown requested");
}
for (RegionServerThread t : regionservers) {
if (t.isAlive() && !noWait && System.currentTimeMillis() < maxTime) {
if (t.isAlive() && !wasInterrupted && System.currentTimeMillis() < maxTime) {
try {
t.join(maxTime);
} catch (InterruptedException e) {
LOG.info("Got InterruptedException on shutdown - " +
"not waiting anymore on region server ends", e);
noWait = true; // someone wants us to speed up.
wasInterrupted = true; // someone wants us to speed up.
}
}
}
@ -260,14 +263,16 @@ public class JVMClusterUtil {
for (RegionServerThread t : regionservers) {
if (t.isAlive()) {
try {
LOG.warn("RegionServerThreads remaining, give one more chance before interrupting");
t.join(10);
} catch (InterruptedException e) {
noWait = true;
wasInterrupted = true;
}
}
}
for (RegionServerThread t : regionservers) {
if (t.isAlive()) {
LOG.warn("RegionServerThreads taking too long to stop, interrupting");
t.interrupt();
}
}
@ -278,7 +283,7 @@ public class JVMClusterUtil {
if (masters != null) {
for (JVMClusterUtil.MasterThread t : masters) {
while (t.master.isAlive() && !noWait) {
while (t.master.isAlive() && !wasInterrupted) {
try {
// The below has been replaced to debug sometime hangs on end of
// tests.
@ -287,7 +292,7 @@ public class JVMClusterUtil {
} catch(InterruptedException e) {
LOG.info("Got InterruptedException on shutdown - " +
"not waiting anymore on master ends", e);
noWait = true;
wasInterrupted = true;
}
}
}
@ -295,9 +300,9 @@ public class JVMClusterUtil {
LOG.info("Shutdown of " +
((masters != null) ? masters.size() : "0") + " master(s) and " +
((regionservers != null) ? regionservers.size() : "0") +
" regionserver(s) " + (noWait ? "interrupted" : "complete"));
" regionserver(s) " + (wasInterrupted ? "interrupted" : "complete"));
if (!noWait){
if (wasInterrupted){
Thread.currentThread().interrupt();
}
}

View File

@ -0,0 +1,201 @@
/**
*
* 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.util;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.Future;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.regionserver.HRegion;
/**
* Utility methods for interacting with the regions.
*/
@InterfaceAudience.Private
public abstract class ModifyRegionUtils {
private static final Log LOG = LogFactory.getLog(ModifyRegionUtils.class);
private ModifyRegionUtils() {
}
public interface RegionFillTask {
public void fillRegion(final HRegion region) throws IOException;
}
/**
* Create new set of regions on the specified file-system.
* NOTE: that you should add the regions to .META. after this operation.
*
* @param conf {@link Configuration}
* @param rootDir Root directory for HBase instance
* @param hTableDescriptor description of the table
* @param newRegions {@link HRegionInfo} that describes the regions to create
* @throws IOException
*/
public static List<HRegionInfo> createRegions(final Configuration conf, final Path rootDir,
final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions) throws IOException {
return createRegions(conf, rootDir, hTableDescriptor, newRegions, null);
}
/**
* Create new set of regions on the specified file-system.
* NOTE: that you should add the regions to .META. after this operation.
*
* @param conf {@link Configuration}
* @param rootDir Root directory for HBase instance
* @param hTableDescriptor description of the table
* @param newRegions {@link HRegionInfo} that describes the regions to create
* @param task {@link RegionFillTask} custom code to populate region after creation
* @throws IOException
*/
public static List<HRegionInfo> createRegions(final Configuration conf, final Path rootDir,
final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions,
final RegionFillTask task) throws IOException {
if (newRegions == null) return null;
int regionNumber = newRegions.length;
ThreadPoolExecutor regionOpenAndInitThreadPool = getRegionOpenAndInitThreadPool(conf,
"RegionOpenAndInitThread-" + hTableDescriptor.getNameAsString(), regionNumber);
CompletionService<HRegionInfo> completionService = new ExecutorCompletionService<HRegionInfo>(
regionOpenAndInitThreadPool);
List<HRegionInfo> regionInfos = new ArrayList<HRegionInfo>();
for (final HRegionInfo newRegion : newRegions) {
completionService.submit(new Callable<HRegionInfo>() {
public HRegionInfo call() throws IOException {
// 1. Create HRegion
HRegion region = HRegion.createHRegion(newRegion,
rootDir, conf, hTableDescriptor, null,
false, true);
HRegion.writeRegioninfoOnFilesystem(region.getRegionInfo(), region.getRegionDir(),
region.getFilesystem(), conf);
try {
// 2. Custom user code to interact with the created region
if (task != null) {
task.fillRegion(region);
}
} finally {
// 3. Close the new region to flush to disk. Close log file too.
region.close();
}
return region.getRegionInfo();
}
});
}
try {
// 4. wait for all regions to finish creation
for (int i = 0; i < regionNumber; i++) {
Future<HRegionInfo> future = completionService.take();
HRegionInfo regionInfo = future.get();
regionInfos.add(regionInfo);
}
} catch (InterruptedException e) {
LOG.error("Caught " + e + " during region creation");
throw new InterruptedIOException(e.getMessage());
} catch (ExecutionException e) {
throw new IOException(e);
} finally {
regionOpenAndInitThreadPool.shutdownNow();
}
return regionInfos;
}
/*
* used by createRegions() to get the thread pool executor based on the
* "hbase.hregion.open.and.init.threads.max" property.
*/
static ThreadPoolExecutor getRegionOpenAndInitThreadPool(final Configuration conf,
final String threadNamePrefix, int regionNumber) {
int maxThreads = Math.min(regionNumber, conf.getInt(
"hbase.hregion.open.and.init.threads.max", 10));
ThreadPoolExecutor regionOpenAndInitThreadPool = Threads
.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
new ThreadFactory() {
private int count = 1;
public Thread newThread(Runnable r) {
Thread t = new Thread(r, threadNamePrefix + "-" + count++);
return t;
}
});
return regionOpenAndInitThreadPool;
}
/**
* Trigger immediate assignment of the regions in round-robin fashion
*
* @param assignmentManager
* @param regions
*/
public static void assignRegions(final AssignmentManager assignmentManager,
final List<HRegionInfo> regions) throws IOException {
try {
assignmentManager.getRegionStates().createRegionStates(regions);
assignmentManager.assign(regions);
} catch (InterruptedException e) {
LOG.error("Caught " + e + " during round-robin assignment");
throw new InterruptedIOException(e.getMessage());
}
}
/**
* Remove specified regions by removing them from file-system and .META.
* (The regions must be offline).
*
* @param fs {@link FileSystem} on which to delete the region directory
* @param catalogTracker the catalog tracker
* @param regions list of {@link HRegionInfo} to delete.
*/
public static void deleteRegions(final Configuration conf, final FileSystem fs,
final CatalogTracker catalogTracker, final List<HRegionInfo> regions) throws IOException {
if (regions != null && regions.size() > 0) {
List<Delete> deletes = new ArrayList<Delete>(regions.size());
for (HRegionInfo hri: regions) {
deletes.add(new Delete(hri.getRegionName()));
// "Delete" region from FS
HFileArchiver.archiveRegion(conf, fs, hri);
}
MetaEditor.deleteFromMetaTable(catalogTracker, deletes);
}
}
}

View File

@ -26,18 +26,15 @@ import java.net.InetSocketAddress;
import java.net.Socket;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Properties;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
import javax.security.auth.login.AppConfigurationEntry;
import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.authentication.util.KerberosUtil;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -52,17 +49,19 @@ import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.DeleteNodeFailSilent;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.SetData;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.authentication.util.KerberosUtil;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.client.ZooKeeperSaslClient;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.client.ZooKeeperSaslClient;
import org.apache.zookeeper.proto.CreateRequest;
import org.apache.zookeeper.proto.DeleteRequest;
import org.apache.zookeeper.proto.SetDataRequest;
@ -862,6 +861,10 @@ public class ZKUtil {
/**
* Set data into node creating node if it doesn't yet exist.
* Does not set watch.
*
* WARNING: this is not atomic -- it is possible to get a 0-byte data value in the znode before
* data is written
*
* @param zkw zk reference
* @param znode path of node
* @param data data to set for node
@ -1080,7 +1083,7 @@ public class ZKUtil {
}
/**
* Creates the specified node, if the node does not exist. Does not set a
* Creates the specified node, iff the node does not exist. Does not set a
* watch and fails silently if the node already exists.
*
* The node created is persistent and open access.
@ -1091,10 +1094,27 @@ public class ZKUtil {
*/
public static void createAndFailSilent(ZooKeeperWatcher zkw,
String znode) throws KeeperException {
createAndFailSilent(zkw,
(CreateAndFailSilent)ZKUtilOp.createAndFailSilent(znode, new byte[0]));
createAndFailSilent(zkw, znode, new byte[0]);
}
/**
* Creates the specified node containing specified data, iff the node does not exist. Does
* not set a watch and fails silently if the node already exists.
*
* The node created is persistent and open access.
*
* @param zkw zk reference
* @param znode path of node
* @param data a byte array data to store in the znode
* @throws KeeperException if unexpected zookeeper exception
*/
public static void createAndFailSilent(ZooKeeperWatcher zkw,
String znode, byte[] data)
throws KeeperException {
createAndFailSilent(zkw,
(CreateAndFailSilent)ZKUtilOp.createAndFailSilent(znode, data));
}
private static void createAndFailSilent(ZooKeeperWatcher zkw, CreateAndFailSilent cafs)
throws KeeperException {
CreateRequest create = (CreateRequest)toZooKeeperOp(zkw, cafs).toRequestRecord();
@ -1132,12 +1152,30 @@ public class ZKUtil {
* @throws KeeperException if unexpected zookeeper exception
*/
public static void createWithParents(ZooKeeperWatcher zkw, String znode)
throws KeeperException {
createWithParents(zkw, znode, new byte[0]);
}
/**
* Creates the specified node and all parent nodes required for it to exist. The creation of
* parent znodes is not atomic with the leafe znode creation but the data is written atomically
* when the leaf node is created.
*
* No watches are set and no errors are thrown if the node already exists.
*
* The nodes created are persistent and open access.
*
* @param zkw zk reference
* @param znode path of node
* @throws KeeperException if unexpected zookeeper exception
*/
public static void createWithParents(ZooKeeperWatcher zkw, String znode, byte[] data)
throws KeeperException {
try {
if(znode == null) {
return;
}
zkw.getRecoverableZooKeeper().create(znode, new byte[0], createACL(zkw, znode),
zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
CreateMode.PERSISTENT);
} catch(KeeperException.NodeExistsException nee) {
return;
@ -1694,4 +1732,37 @@ public class ZKUtil {
ke.initCause(e);
return ke;
}
/**
* Recursively print the current state of ZK (non-transactional)
* @param root name of the root directory in zk to print
* @throws KeeperException
*/
public static void logZKTree(ZooKeeperWatcher zkw, String root) {
if (!LOG.isDebugEnabled()) return;
LOG.debug("Current zk system:");
String prefix = "|-";
LOG.debug(prefix + root);
try {
logZKTree(zkw, root, prefix);
} catch (KeeperException e) {
throw new RuntimeException(e);
}
}
/**
* Helper method to print the current state of the ZK tree.
* @see #logZKTree(ZooKeeperWatcher, String)
* @throws KeeperException if an unexpected exception occurs
*/
protected static void logZKTree(ZooKeeperWatcher zkw, String root, String prefix) throws KeeperException {
List<String> children = ZKUtil.listChildrenNoWatch(zkw, root);
if (children == null) return;
for (String child : children) {
LOG.debug(prefix + child);
String node = ZKUtil.joinZNode(root.equals("/") ? "" : root, child);
logZKTree(zkw, node, prefix + "---");
}
}
}

View File

@ -951,6 +951,13 @@
properties from a zoo.cfg file has been deprecated.
</description>
</property>
<property>
<name>hbase.snapshot.enabled</name>
<value>true</value>
<description>
Set to true to allow snapshots to be taken / restored / cloned.
</description>
</property>
<property>
<name>hbase.rest.threads.max</name>
<value>100</value>

View File

@ -0,0 +1,221 @@
<%--
/**
* 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.
*/
--%>
<%@ page contentType="text/html;charset=UTF-8"
import="java.util.Date"
import="java.util.HashMap"
import="org.apache.hadoop.conf.Configuration"
import="org.apache.hadoop.hbase.client.HBaseAdmin"
import="org.apache.hadoop.hbase.client.HConnectionManager"
import="org.apache.hadoop.hbase.HRegionInfo"
import="org.apache.hadoop.hbase.ServerName"
import="org.apache.hadoop.hbase.ServerLoad"
import="org.apache.hadoop.hbase.RegionLoad"
import="org.apache.hadoop.hbase.master.HMaster"
import="org.apache.hadoop.hbase.util.Bytes"
import="org.apache.hadoop.hbase.util.FSUtils"
import="org.apache.hadoop.hbase.protobuf.ProtobufUtil"
import="org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription"
import="java.util.List"
import="java.util.Map"
import="org.apache.hadoop.hbase.HConstants"%><%
HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
Configuration conf = master.getConfiguration();
HBaseAdmin hbadmin = new HBaseAdmin(conf);
boolean readOnly = conf.getBoolean("hbase.master.ui.readonly", false);
String snapshotName = request.getParameter("name");
SnapshotDescription snapshot = null;
for (SnapshotDescription snapshotDesc: hbadmin.getCompletedSnapshots()) {
if (snapshotName.equals(snapshotDesc.getName())) {
snapshot = snapshotDesc;
break;
}
}
String action = request.getParameter("action");
String cloneName = request.getParameter("cloneName");
boolean isActionResultPage = (!readOnly && action != null);
%>
<?xml version="1.0" encoding="UTF-8" ?>
<!-- Commenting out DOCTYPE so our blue outline shows on hadoop 0.20.205.0, etc.
See tail of HBASE-2110 for explaination.
<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN"
"http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
-->
<html xmlns="http://www.w3.org/1999/xhtml">
<head>
<meta charset="utf-8">
<% if (isActionResultPage) { %>
<title>HBase Master: <%= master.getServerName() %></title>
<% } else { %>
<title>Snapshot: <%= snapshotName %></title>
<% } %>
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<meta name="description" content="">
<meta name="author" content="">
<link href="/static/css/bootstrap.css" rel="stylesheet">
<link href="/static/css/hbase.css" rel="stylesheet">
<link href="/static/css/bootstrap-responsive.css" rel="stylesheet">
<!--[if lt IE 9]>
<script src="/static/js/html5shiv.js"></script>
<![endif]-->
<% if (isActionResultPage) { %>
<meta http-equiv="refresh" content="5,javascript:history.back()" />
<% } %>
</head>
<body>
<div class="navbar navbar-fixed-top">
<div class="navbar-inner">
<div class="container">
<a class="btn btn-navbar" data-toggle="collapse" data-target=".nav-collapse">
<span class="icon-bar"></span>
<span class="icon-bar"></span>
<span class="icon-bar"></span>
</a>
<a class="brand" href="/master-status">HBase Master</a>
<div class="nav-collapse">
<ul class="nav">
<li><a href="/">Home</a></li>
<li><a href="/tablesDetailed.jsp">Table Details</a></li>
<li><a href="/logs/">Local logs</a></li>
<li><a href="/logLevel">Log Level</a></li>
<li><a href="/dump">Debug dump</a></li>
<li><a href="/jmx">Metrics Dump</a></li>
</ul>
</div><!--/.nav-collapse -->
</div>
</div>
</div>
<% if (isActionResultPage) { %>
<div class="container">
<div class="row inner_header">
<div class="span8">
<h1>Snapshot action request...</h1>
</div>
<div class="span4 logo">
<img src="/static/hbase_logo.png" height="66" width="266" alt="HBase logo"/>
</div>
</div>
<p><hr><p>
<%
if (action.equals("restore")) {
hbadmin.restoreSnapshot(snapshotName);
%> Restore Snapshot request accepted. <%
} else if (action.equals("clone")) {
if (cloneName != null && cloneName.length() > 0) {
hbadmin.cloneSnapshot(snapshotName, cloneName);
%> Clone from Snapshot request accepted. <%
} else {
%> Clone from Snapshot request failed, No table name specified. <%
}
}
%>
<p>Go <a href="javascript:history.back()">Back</a>, or wait for the redirect.
</div>
<% } else if (snapshot == null) { %>
<div class="container">
<div class="row inner_header">
<div class="span8">
<h1>Snapshot "<%= snapshotName %>" does not exists</h1>
</div>
<div class="span4 logo">
<img src="/static/hbase_logo.png" height="66" width="266" alt="HBase logo"/>
</div>
</div>
<p>Go <a href="javascript:history.back()">Back</a>, or wait for the redirect.
<% } else { %>
<div class="container">
<div class="row">
<div class="span8">
<h1>Snapshot: <%= snapshotName %></h1>
</div>
<div class="span4">
<img src="/static/hbase_logo.png" height="66" width="266" alt="HBase logo"/>
</div>
</div>
<h2>Snapshot Attributes</h2>
<table class="table" width="90%" >
<tr>
<th>Table</th>
<th>Creation Time</th>
<th>Type</th>
<th>Format Version</th>
</tr>
<tr>
<td><a href="table.jsp?name=<%= snapshot.getTable() %>"><%= snapshot.getTable() %></a></td>
<td><%= new Date(snapshot.getCreationTime()) %></td>
<td><%= snapshot.getType() %></td>
<td><%= snapshot.getVersion() %></td>
</tr>
</table>
<%
} // end else
HConnectionManager.deleteConnection(hbadmin.getConfiguration());
%>
<% if (!readOnly && action == null && snapshot != null) { %>
<p><hr><p>
Actions:
<p>
<center>
<table class="table" width="90%" >
<tr>
<form method="get">
<input type="hidden" name="action" value="clone">
<input type="hidden" name="name" value="<%= snapshotName %>">
<td style="border-style: none; text-align: center">
<input style="font-size: 12pt; width: 10em" type="submit" value="Clone" class="btn"></td>
<td style="border-style: none" width="5%">&nbsp;</td>
<td style="border-style: none">New Table Name (clone):<input type="text" name="cloneName" size="40"></td>
<td style="border-style: none">
This action will create a new table by cloning the snapshot content.
There are no copies of data involved.
And writing on the newly created table will not influence the snapshot data.
</td>
</form>
</tr>
<tr><td style="border-style: none" colspan="4">&nbsp;</td></tr>
<tr>
<form method="get">
<input type="hidden" name="action" value="restore">
<input type="hidden" name="name" value="<%= snapshotName %>">
<td style="border-style: none; text-align: center">
<input style="font-size: 12pt; width: 10em" type="submit" value="Restore" class="btn"></td>
<td style="border-style: none" width="5%">&nbsp;</td>
<td style="border-style: none">&nbsp;</td>
<td style="border-style: none">Restore a specified snapshot.
The restore will replace the content of the original table,
bringing back the content to the snapshot state.
The table must be disabled.</td>
</form>
</tr>
</table>
</center>
<p>
</div>
<% } %>
<script src="/static/js/jquery.min.js" type="text/javascript"></script>
<script src="/static/js/bootstrap.min.js" type="text/javascript"></script>
</body>
</html>

View File

@ -655,6 +655,42 @@ module Hbase
end
end
#----------------------------------------------------------------------------------------------
# Take a snapshot of specified table
def snapshot(table, snapshot_name)
@admin.snapshot(snapshot_name.to_java_bytes, table.to_java_bytes)
end
#----------------------------------------------------------------------------------------------
# Restore specified snapshot
def restore_snapshot(snapshot_name)
@admin.restoreSnapshot(snapshot_name.to_java_bytes)
end
#----------------------------------------------------------------------------------------------
# Create a new table by cloning the snapshot content
def clone_snapshot(snapshot_name, table)
@admin.cloneSnapshot(snapshot_name.to_java_bytes, table.to_java_bytes)
end
#----------------------------------------------------------------------------------------------
# Rename specified snapshot
def rename_snapshot(old_snapshot_name, new_snapshot_name)
@admin.renameSnapshot(old_snapshot_name.to_java_bytes, new_snapshot_name.to_java_bytes)
end
#----------------------------------------------------------------------------------------------
# Delete specified snapshot
def delete_snapshot(snapshot_name)
@admin.deleteSnapshot(snapshot_name.to_java_bytes)
end
#----------------------------------------------------------------------------------------------
# Returns a list of snapshots
def list_snapshot
@admin.getCompletedSnapshots
end
# Apply config specific to a table/column to its descriptor
def set_descriptor_config(descriptor, config)
raise(ArgumentError, "#{CONFIGURATION} must be a Hash type") unless config.kind_of?(Hash)

View File

@ -409,14 +409,14 @@ EOF
# Generally used for admin functions which just have one name and take the table name
def self.add_admin_utils(*args)
args.each do |method|
define_method method do
@shell.command(method, @name)
define_method method do |*method_args|
@shell.command(method, @name, *method_args)
end
end
end
#Add the following admin utilities to the table
add_admin_utils :enable, :disable, :flush, :drop, :describe
add_admin_utils :enable, :disable, :flush, :drop, :describe, :snapshot
#----------------------------
#give the general help for the table

View File

@ -307,6 +307,19 @@ Shell.load_command_group(
]
)
Shell.load_command_group(
'snapshot',
:full_name => 'CLUSTER SNAPSHOT TOOLS',
:commands => %w[
snapshot
clone_snapshot
restore_snapshot
rename_snapshot
delete_snapshot
list_snapshots
]
)
Shell.load_command_group(
'security',
:full_name => 'SECURITY TOOLS',

View File

@ -0,0 +1,40 @@
#
# 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.
#
module Shell
module Commands
class CloneSnapshot < Command
def help
return <<-EOF
Create a new table by cloning the snapshot content.
There're no copies of data involved.
And writing on the newly created table will not influence the snapshot data.
Examples:
hbase> clone_snapshot 'snapshotName', 'tableName'
EOF
end
def command(snapshot_name, table)
format_simple_command do
admin.clone_snapshot(snapshot_name, table)
end
end
end
end
end

View File

@ -0,0 +1,37 @@
#
# 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.
#
module Shell
module Commands
class DeleteSnapshot < Command
def help
return <<-EOF
Delete a specified snapshot. Examples:
hbase> delete_snapshot 'snapshotName',
EOF
end
def command(snapshot_name)
format_simple_command do
admin.delete_snapshot(snapshot_name)
end
end
end
end
end

Some files were not shown because too many files have changed in this diff Show More