HBASE-6836 Offline snapshots (Jesse Yates)
git-svn-id: https://svn.apache.org/repos/asf/hbase/branches/hbase-7290@1445782 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
ef53214f02
commit
cb6f286858
|
@ -741,10 +741,17 @@ 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 snapshots 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";
|
||||
|
||||
public static final List<String> HBASE_NON_USER_TABLE_DIRS = new ArrayList<String>(
|
||||
Arrays.asList(new String[] { HREGION_LOGDIR_NAME, HREGION_OLDLOGDIR_NAME, CORRUPT_DIR_NAME,
|
||||
toString(META_TABLE_NAME), toString(ROOT_TABLE_NAME), SPLIT_LOGDIR_NAME,
|
||||
HBCK_SIDELINEDIR_NAME, HFILE_ARCHIVE_DIRECTORY }));
|
||||
HBCK_SIDELINEDIR_NAME, HFILE_ARCHIVE_DIRECTORY, SNAPSHOT_DIR_NAME }));
|
||||
|
||||
private HConstants() {
|
||||
// Can't be instantiated with this ctor.
|
||||
|
|
|
@ -11321,6 +11321,10 @@ public final class HBaseProtos {
|
|||
// optional .SnapshotDescription.Type type = 4 [default = TIMESTAMP];
|
||||
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
|
||||
|
@ -11352,20 +11356,23 @@ public final class HBaseProtos {
|
|||
|
||||
public enum Type
|
||||
implements com.google.protobuf.ProtocolMessageEnum {
|
||||
TIMESTAMP(0, 0),
|
||||
GLOBAL(1, 1),
|
||||
DISABLED(0, 0),
|
||||
TIMESTAMP(1, 1),
|
||||
GLOBAL(2, 2),
|
||||
;
|
||||
|
||||
public static final int TIMESTAMP_VALUE = 0;
|
||||
public static final int GLOBAL_VALUE = 1;
|
||||
public static final int DISABLED_VALUE = 0;
|
||||
public static final int TIMESTAMP_VALUE = 1;
|
||||
public static final int GLOBAL_VALUE = 2;
|
||||
|
||||
|
||||
public final int getNumber() { return value; }
|
||||
|
||||
public static Type valueOf(int value) {
|
||||
switch (value) {
|
||||
case 0: return TIMESTAMP;
|
||||
case 1: return GLOBAL;
|
||||
case 0: return DISABLED;
|
||||
case 1: return TIMESTAMP;
|
||||
case 2: return GLOBAL;
|
||||
default: return null;
|
||||
}
|
||||
}
|
||||
|
@ -11396,7 +11403,7 @@ public final class HBaseProtos {
|
|||
}
|
||||
|
||||
private static final Type[] VALUES = {
|
||||
TIMESTAMP, GLOBAL,
|
||||
DISABLED, TIMESTAMP, GLOBAL,
|
||||
};
|
||||
|
||||
public static Type valueOf(
|
||||
|
@ -11504,11 +11511,22 @@ public final class HBaseProtos {
|
|||
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.TIMESTAMP;
|
||||
version_ = 0;
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
|
@ -11538,6 +11556,9 @@ public final class HBaseProtos {
|
|||
if (((bitField0_ & 0x00000008) == 0x00000008)) {
|
||||
output.writeEnum(4, type_.getNumber());
|
||||
}
|
||||
if (((bitField0_ & 0x00000010) == 0x00000010)) {
|
||||
output.writeInt32(5, version_);
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
||||
|
@ -11563,6 +11584,10 @@ public final class HBaseProtos {
|
|||
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;
|
||||
|
@ -11606,6 +11631,11 @@ public final class HBaseProtos {
|
|||
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;
|
||||
|
@ -11631,6 +11661,10 @@ public final class HBaseProtos {
|
|||
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;
|
||||
}
|
||||
|
@ -11755,6 +11789,8 @@ public final class HBaseProtos {
|
|||
bitField0_ = (bitField0_ & ~0x00000004);
|
||||
type_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type.TIMESTAMP;
|
||||
bitField0_ = (bitField0_ & ~0x00000008);
|
||||
version_ = 0;
|
||||
bitField0_ = (bitField0_ & ~0x00000010);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -11809,6 +11845,10 @@ public final class HBaseProtos {
|
|||
to_bitField0_ |= 0x00000008;
|
||||
}
|
||||
result.type_ = type_;
|
||||
if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
|
||||
to_bitField0_ |= 0x00000010;
|
||||
}
|
||||
result.version_ = version_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
|
@ -11837,6 +11877,9 @@ public final class HBaseProtos {
|
|||
if (other.hasType()) {
|
||||
setType(other.getType());
|
||||
}
|
||||
if (other.hasVersion()) {
|
||||
setVersion(other.getVersion());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
@ -11898,6 +11941,11 @@ public final class HBaseProtos {
|
|||
}
|
||||
break;
|
||||
}
|
||||
case 40: {
|
||||
bitField0_ |= 0x00000010;
|
||||
version_ = input.readInt32();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -12021,6 +12069,27 @@ public final class HBaseProtos {
|
|||
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)
|
||||
}
|
||||
|
||||
|
@ -12174,18 +12243,19 @@ public final class HBaseProtos {
|
|||
"value\030\002 \002(\t\",\n\rNameBytesPair\022\014\n\004name\030\001 \002" +
|
||||
"(\t\022\r\n\005value\030\002 \001(\014\"/\n\016BytesBytesPair\022\r\n\005f" +
|
||||
"irst\030\001 \002(\014\022\016\n\006second\030\002 \002(\014\",\n\rNameInt64P" +
|
||||
"air\022\014\n\004name\030\001 \001(\t\022\r\n\005value\030\002 \001(\003\"\242\001\n\023Sna" +
|
||||
"air\022\014\n\004name\030\001 \001(\t\022\r\n\005value\030\002 \001(\003\"\301\001\n\023Sna" +
|
||||
"pshotDescription\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\0222\n\004type\030\004" +
|
||||
" \001(\0162\031.SnapshotDescription.Type:\tTIMESTA" +
|
||||
"MP\"!\n\004Type\022\r\n\tTIMESTAMP\020\000\022\n\n\006GLOBAL\020\001*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_O",
|
||||
"R_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*_\n\007Key" +
|
||||
"Type\022\013\n\007MINIMUM\020\000\022\007\n\003PUT\020\004\022\n\n\006DELETE\020\010\022\021" +
|
||||
"\n\rDELETE_COLUMN\020\014\022\021\n\rDELETE_FAMILY\020\016\022\014\n\007" +
|
||||
"MAXIMUM\020\377\001B>\n*org.apache.hadoop.hbase.pr" +
|
||||
"otobuf.generatedB\013HBaseProtosH\001\240\001\001"
|
||||
"MP\022\017\n\007version\030\005 \001(\005\"/\n\004Type\022\014\n\010DISABLED\020" +
|
||||
"\000\022\r\n\tTIMESTAMP\020\001\022\n\n\006GLOBAL\020\002*r\n\013CompareT" +
|
||||
"ype\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUA",
|
||||
"L\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_OP\020\006*_\n\007KeyType\022\013\n\007M" +
|
||||
"INIMUM\020\000\022\007\n\003PUT\020\004\022\n\n\006DELETE\020\010\022\021\n\rDELETE_" +
|
||||
"COLUMN\020\014\022\021\n\rDELETE_FAMILY\020\016\022\014\n\007MAXIMUM\020\377" +
|
||||
"\001B>\n*org.apache.hadoop.hbase.protobuf.ge" +
|
||||
"neratedB\013HBaseProtosH\001\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -12333,7 +12403,7 @@ public final class HBaseProtos {
|
|||
internal_static_SnapshotDescription_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_SnapshotDescription_descriptor,
|
||||
new java.lang.String[] { "Name", "Table", "CreationTime", "Type", },
|
||||
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);
|
||||
return null;
|
||||
|
|
|
@ -14840,9 +14840,9 @@ public final class MasterAdminProtos {
|
|||
public interface TakeSnapshotResponseOrBuilder
|
||||
extends com.google.protobuf.MessageOrBuilder {
|
||||
|
||||
// required int64 expectedTime = 1;
|
||||
boolean hasExpectedTime();
|
||||
long getExpectedTime();
|
||||
// required int64 expectedTimeout = 1;
|
||||
boolean hasExpectedTimeout();
|
||||
long getExpectedTimeout();
|
||||
}
|
||||
public static final class TakeSnapshotResponse extends
|
||||
com.google.protobuf.GeneratedMessage
|
||||
|
@ -14873,25 +14873,25 @@ public final class MasterAdminProtos {
|
|||
}
|
||||
|
||||
private int bitField0_;
|
||||
// required int64 expectedTime = 1;
|
||||
public static final int EXPECTEDTIME_FIELD_NUMBER = 1;
|
||||
private long expectedTime_;
|
||||
public boolean hasExpectedTime() {
|
||||
// required int64 expectedTimeout = 1;
|
||||
public static final int EXPECTEDTIMEOUT_FIELD_NUMBER = 1;
|
||||
private long expectedTimeout_;
|
||||
public boolean hasExpectedTimeout() {
|
||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
public long getExpectedTime() {
|
||||
return expectedTime_;
|
||||
public long getExpectedTimeout() {
|
||||
return expectedTimeout_;
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
expectedTime_ = 0L;
|
||||
expectedTimeout_ = 0L;
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
byte isInitialized = memoizedIsInitialized;
|
||||
if (isInitialized != -1) return isInitialized == 1;
|
||||
|
||||
if (!hasExpectedTime()) {
|
||||
if (!hasExpectedTimeout()) {
|
||||
memoizedIsInitialized = 0;
|
||||
return false;
|
||||
}
|
||||
|
@ -14903,7 +14903,7 @@ public final class MasterAdminProtos {
|
|||
throws java.io.IOException {
|
||||
getSerializedSize();
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
output.writeInt64(1, expectedTime_);
|
||||
output.writeInt64(1, expectedTimeout_);
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
@ -14916,7 +14916,7 @@ public final class MasterAdminProtos {
|
|||
size = 0;
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeInt64Size(1, expectedTime_);
|
||||
.computeInt64Size(1, expectedTimeout_);
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
|
@ -14941,10 +14941,10 @@ public final class MasterAdminProtos {
|
|||
org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotResponse) obj;
|
||||
|
||||
boolean result = true;
|
||||
result = result && (hasExpectedTime() == other.hasExpectedTime());
|
||||
if (hasExpectedTime()) {
|
||||
result = result && (getExpectedTime()
|
||||
== other.getExpectedTime());
|
||||
result = result && (hasExpectedTimeout() == other.hasExpectedTimeout());
|
||||
if (hasExpectedTimeout()) {
|
||||
result = result && (getExpectedTimeout()
|
||||
== other.getExpectedTimeout());
|
||||
}
|
||||
result = result &&
|
||||
getUnknownFields().equals(other.getUnknownFields());
|
||||
|
@ -14955,9 +14955,9 @@ public final class MasterAdminProtos {
|
|||
public int hashCode() {
|
||||
int hash = 41;
|
||||
hash = (19 * hash) + getDescriptorForType().hashCode();
|
||||
if (hasExpectedTime()) {
|
||||
hash = (37 * hash) + EXPECTEDTIME_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashLong(getExpectedTime());
|
||||
if (hasExpectedTimeout()) {
|
||||
hash = (37 * hash) + EXPECTEDTIMEOUT_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashLong(getExpectedTimeout());
|
||||
}
|
||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||
return hash;
|
||||
|
@ -15075,7 +15075,7 @@ public final class MasterAdminProtos {
|
|||
|
||||
public Builder clear() {
|
||||
super.clear();
|
||||
expectedTime_ = 0L;
|
||||
expectedTimeout_ = 0L;
|
||||
bitField0_ = (bitField0_ & ~0x00000001);
|
||||
return this;
|
||||
}
|
||||
|
@ -15118,7 +15118,7 @@ public final class MasterAdminProtos {
|
|||
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
to_bitField0_ |= 0x00000001;
|
||||
}
|
||||
result.expectedTime_ = expectedTime_;
|
||||
result.expectedTimeout_ = expectedTimeout_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
|
@ -15135,15 +15135,15 @@ public final class MasterAdminProtos {
|
|||
|
||||
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotResponse other) {
|
||||
if (other == org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotResponse.getDefaultInstance()) return this;
|
||||
if (other.hasExpectedTime()) {
|
||||
setExpectedTime(other.getExpectedTime());
|
||||
if (other.hasExpectedTimeout()) {
|
||||
setExpectedTimeout(other.getExpectedTimeout());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
||||
public final boolean isInitialized() {
|
||||
if (!hasExpectedTime()) {
|
||||
if (!hasExpectedTimeout()) {
|
||||
|
||||
return false;
|
||||
}
|
||||
|
@ -15175,7 +15175,7 @@ public final class MasterAdminProtos {
|
|||
}
|
||||
case 8: {
|
||||
bitField0_ |= 0x00000001;
|
||||
expectedTime_ = input.readInt64();
|
||||
expectedTimeout_ = input.readInt64();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -15184,23 +15184,23 @@ public final class MasterAdminProtos {
|
|||
|
||||
private int bitField0_;
|
||||
|
||||
// required int64 expectedTime = 1;
|
||||
private long expectedTime_ ;
|
||||
public boolean hasExpectedTime() {
|
||||
// required int64 expectedTimeout = 1;
|
||||
private long expectedTimeout_ ;
|
||||
public boolean hasExpectedTimeout() {
|
||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
public long getExpectedTime() {
|
||||
return expectedTime_;
|
||||
public long getExpectedTimeout() {
|
||||
return expectedTimeout_;
|
||||
}
|
||||
public Builder setExpectedTime(long value) {
|
||||
public Builder setExpectedTimeout(long value) {
|
||||
bitField0_ |= 0x00000001;
|
||||
expectedTime_ = value;
|
||||
expectedTimeout_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
public Builder clearExpectedTime() {
|
||||
public Builder clearExpectedTimeout() {
|
||||
bitField0_ = (bitField0_ & ~0x00000001);
|
||||
expectedTime_ = 0L;
|
||||
expectedTimeout_ = 0L;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
@ -19883,56 +19883,57 @@ public final class MasterAdminProtos {
|
|||
"anitorEnabledRequest\"0\n\037IsCatalogJanitor",
|
||||
"EnabledResponse\022\r\n\005value\030\001 \002(\010\"=\n\023TakeSn" +
|
||||
"apshotRequest\022&\n\010snapshot\030\001 \002(\0132\024.Snapsh" +
|
||||
"otDescription\",\n\024TakeSnapshotResponse\022\024\n" +
|
||||
"\014expectedTime\030\001 \002(\003\"\025\n\023ListSnapshotReque" +
|
||||
"st\"?\n\024ListSnapshotResponse\022\'\n\tsnapshots\030" +
|
||||
"\001 \003(\0132\024.SnapshotDescription\"?\n\025DeleteSna" +
|
||||
"pshotRequest\022&\n\010snapshot\030\001 \002(\0132\024.Snapsho" +
|
||||
"tDescription\"\030\n\026DeleteSnapshotResponse\"?" +
|
||||
"\n\025IsSnapshotDoneRequest\022&\n\010snapshot\030\001 \001(" +
|
||||
"\0132\024.SnapshotDescription\"U\n\026IsSnapshotDon",
|
||||
"eResponse\022\023\n\004done\030\001 \001(\010:\005false\022&\n\010snapsh" +
|
||||
"ot\030\002 \001(\0132\024.SnapshotDescription2\376\013\n\022Maste" +
|
||||
"rAdminService\0222\n\taddColumn\022\021.AddColumnRe" +
|
||||
"quest\032\022.AddColumnResponse\022;\n\014deleteColum" +
|
||||
"n\022\024.DeleteColumnRequest\032\025.DeleteColumnRe" +
|
||||
"sponse\022;\n\014modifyColumn\022\024.ModifyColumnReq" +
|
||||
"uest\032\025.ModifyColumnResponse\0225\n\nmoveRegio" +
|
||||
"n\022\022.MoveRegionRequest\032\023.MoveRegionRespon" +
|
||||
"se\022;\n\014assignRegion\022\024.AssignRegionRequest" +
|
||||
"\032\025.AssignRegionResponse\022A\n\016unassignRegio",
|
||||
"n\022\026.UnassignRegionRequest\032\027.UnassignRegi" +
|
||||
"onResponse\022>\n\rofflineRegion\022\025.OfflineReg" +
|
||||
"ionRequest\032\026.OfflineRegionResponse\0228\n\013de" +
|
||||
"leteTable\022\023.DeleteTableRequest\032\024.DeleteT" +
|
||||
"ableResponse\0228\n\013enableTable\022\023.EnableTabl" +
|
||||
"eRequest\032\024.EnableTableResponse\022;\n\014disabl" +
|
||||
"eTable\022\024.DisableTableRequest\032\025.DisableTa" +
|
||||
"bleResponse\0228\n\013modifyTable\022\023.ModifyTable" +
|
||||
"Request\032\024.ModifyTableResponse\0228\n\013createT" +
|
||||
"able\022\023.CreateTableRequest\032\024.CreateTableR",
|
||||
"esponse\022/\n\010shutdown\022\020.ShutdownRequest\032\021." +
|
||||
"ShutdownResponse\0225\n\nstopMaster\022\022.StopMas" +
|
||||
"terRequest\032\023.StopMasterResponse\022,\n\007balan" +
|
||||
"ce\022\017.BalanceRequest\032\020.BalanceResponse\022M\n" +
|
||||
"\022setBalancerRunning\022\032.SetBalancerRunning" +
|
||||
"Request\032\033.SetBalancerRunningResponse\022;\n\016" +
|
||||
"runCatalogScan\022\023.CatalogScanRequest\032\024.Ca" +
|
||||
"talogScanResponse\022S\n\024enableCatalogJanito" +
|
||||
"r\022\034.EnableCatalogJanitorRequest\032\035.Enable" +
|
||||
"CatalogJanitorResponse\022\\\n\027isCatalogJanit",
|
||||
"orEnabled\022\037.IsCatalogJanitorEnabledReque" +
|
||||
"st\032 .IsCatalogJanitorEnabledResponse\022L\n\021" +
|
||||
"execMasterService\022\032.CoprocessorServiceRe" +
|
||||
"quest\032\033.CoprocessorServiceResponse\0227\n\010sn" +
|
||||
"apshot\022\024.TakeSnapshotRequest\032\025.TakeSnaps" +
|
||||
"hotResponse\022<\n\rlistSnapshots\022\024.ListSnaps" +
|
||||
"hotRequest\032\025.ListSnapshotResponse\022A\n\016del" +
|
||||
"eteSnapshot\022\026.DeleteSnapshotRequest\032\027.De" +
|
||||
"leteSnapshotResponse\022A\n\016isSnapshotDone\022\026" +
|
||||
".IsSnapshotDoneRequest\032\027.IsSnapshotDoneR",
|
||||
"esponseBG\n*org.apache.hadoop.hbase.proto" +
|
||||
"buf.generatedB\021MasterAdminProtosH\001\210\001\001\240\001\001"
|
||||
"otDescription\"/\n\024TakeSnapshotResponse\022\027\n" +
|
||||
"\017expectedTimeout\030\001 \002(\003\"\025\n\023ListSnapshotRe" +
|
||||
"quest\"?\n\024ListSnapshotResponse\022\'\n\tsnapsho" +
|
||||
"ts\030\001 \003(\0132\024.SnapshotDescription\"?\n\025Delete" +
|
||||
"SnapshotRequest\022&\n\010snapshot\030\001 \002(\0132\024.Snap" +
|
||||
"shotDescription\"\030\n\026DeleteSnapshotRespons" +
|
||||
"e\"?\n\025IsSnapshotDoneRequest\022&\n\010snapshot\030\001" +
|
||||
" \001(\0132\024.SnapshotDescription\"U\n\026IsSnapshot",
|
||||
"DoneResponse\022\023\n\004done\030\001 \001(\010:\005false\022&\n\010sna" +
|
||||
"pshot\030\002 \001(\0132\024.SnapshotDescription2\376\013\n\022Ma" +
|
||||
"sterAdminService\0222\n\taddColumn\022\021.AddColum" +
|
||||
"nRequest\032\022.AddColumnResponse\022;\n\014deleteCo" +
|
||||
"lumn\022\024.DeleteColumnRequest\032\025.DeleteColum" +
|
||||
"nResponse\022;\n\014modifyColumn\022\024.ModifyColumn" +
|
||||
"Request\032\025.ModifyColumnResponse\0225\n\nmoveRe" +
|
||||
"gion\022\022.MoveRegionRequest\032\023.MoveRegionRes" +
|
||||
"ponse\022;\n\014assignRegion\022\024.AssignRegionRequ" +
|
||||
"est\032\025.AssignRegionResponse\022A\n\016unassignRe",
|
||||
"gion\022\026.UnassignRegionRequest\032\027.UnassignR" +
|
||||
"egionResponse\022>\n\rofflineRegion\022\025.Offline" +
|
||||
"RegionRequest\032\026.OfflineRegionResponse\0228\n" +
|
||||
"\013deleteTable\022\023.DeleteTableRequest\032\024.Dele" +
|
||||
"teTableResponse\0228\n\013enableTable\022\023.EnableT" +
|
||||
"ableRequest\032\024.EnableTableResponse\022;\n\014dis" +
|
||||
"ableTable\022\024.DisableTableRequest\032\025.Disabl" +
|
||||
"eTableResponse\0228\n\013modifyTable\022\023.ModifyTa" +
|
||||
"bleRequest\032\024.ModifyTableResponse\0228\n\013crea" +
|
||||
"teTable\022\023.CreateTableRequest\032\024.CreateTab",
|
||||
"leResponse\022/\n\010shutdown\022\020.ShutdownRequest" +
|
||||
"\032\021.ShutdownResponse\0225\n\nstopMaster\022\022.Stop" +
|
||||
"MasterRequest\032\023.StopMasterResponse\022,\n\007ba" +
|
||||
"lance\022\017.BalanceRequest\032\020.BalanceResponse" +
|
||||
"\022M\n\022setBalancerRunning\022\032.SetBalancerRunn" +
|
||||
"ingRequest\032\033.SetBalancerRunningResponse\022" +
|
||||
";\n\016runCatalogScan\022\023.CatalogScanRequest\032\024" +
|
||||
".CatalogScanResponse\022S\n\024enableCatalogJan" +
|
||||
"itor\022\034.EnableCatalogJanitorRequest\032\035.Ena" +
|
||||
"bleCatalogJanitorResponse\022\\\n\027isCatalogJa",
|
||||
"nitorEnabled\022\037.IsCatalogJanitorEnabledRe" +
|
||||
"quest\032 .IsCatalogJanitorEnabledResponse\022" +
|
||||
"L\n\021execMasterService\022\032.CoprocessorServic" +
|
||||
"eRequest\032\033.CoprocessorServiceResponse\0227\n" +
|
||||
"\010snapshot\022\024.TakeSnapshotRequest\032\025.TakeSn" +
|
||||
"apshotResponse\022<\n\rlistSnapshots\022\024.ListSn" +
|
||||
"apshotRequest\032\025.ListSnapshotResponse\022A\n\016" +
|
||||
"deleteSnapshot\022\026.DeleteSnapshotRequest\032\027" +
|
||||
".DeleteSnapshotResponse\022A\n\016isSnapshotDon" +
|
||||
"e\022\026.IsSnapshotDoneRequest\032\027.IsSnapshotDo",
|
||||
"neResponseBG\n*org.apache.hadoop.hbase.pr" +
|
||||
"otobuf.generatedB\021MasterAdminProtosH\001\210\001\001" +
|
||||
"\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -20256,7 +20257,7 @@ public final class MasterAdminProtos {
|
|||
internal_static_TakeSnapshotResponse_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_TakeSnapshotResponse_descriptor,
|
||||
new java.lang.String[] { "ExpectedTime", },
|
||||
new java.lang.String[] { "ExpectedTimeout", },
|
||||
org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotResponse.class,
|
||||
org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotResponse.Builder.class);
|
||||
internal_static_ListSnapshotRequest_descriptor =
|
||||
|
|
|
@ -182,7 +182,7 @@ message TakeSnapshotRequest{
|
|||
}
|
||||
|
||||
message TakeSnapshotResponse{
|
||||
required int64 expectedTime = 1;
|
||||
required int64 expectedTimeout = 1;
|
||||
}
|
||||
|
||||
message ListSnapshotRequest{
|
||||
|
|
|
@ -277,8 +277,10 @@ message SnapshotDescription {
|
|||
optional string table = 2; // not needed for delete, but checked for in taking snapshot
|
||||
optional int64 creationTime = 3 [default = 0];
|
||||
enum Type {
|
||||
TIMESTAMP = 0;
|
||||
GLOBAL = 1;
|
||||
DISABLED = 0;
|
||||
TIMESTAMP = 1;
|
||||
GLOBAL = 2;
|
||||
}
|
||||
optional Type type = 4 [default = TIMESTAMP];
|
||||
optional int32 version = 5;
|
||||
}
|
||||
|
|
|
@ -74,7 +74,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRe
|
|||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse;
|
||||
import org.apache.hadoop.hbase.security.KerberosInfo;
|
||||
import org.apache.hadoop.hbase.security.TokenInfo;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.SnapshotCreationException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.UnknownSnapshotException;
|
||||
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
|
|
@ -103,10 +103,10 @@ 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.SnapshotCreationException;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.SnapshotCreationException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.UnknownSnapshotException;
|
||||
import org.apache.hadoop.hbase.util.Addressing;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
@ -2217,14 +2217,15 @@ public class HBaseAdmin implements Abortable, Closeable {
|
|||
TakeSnapshotResponse response = takeSnapshotAsync(snapshot);
|
||||
final IsSnapshotDoneRequest request = IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot)
|
||||
.build();
|
||||
IsSnapshotDoneResponse done = IsSnapshotDoneResponse.newBuilder().buildPartial();
|
||||
IsSnapshotDoneResponse done = null;
|
||||
long start = EnvironmentEdgeManager.currentTimeMillis();
|
||||
long max = response.getExpectedTime();
|
||||
long max = response.getExpectedTimeout();
|
||||
long maxPauseTime = max / this.numRetries;
|
||||
int tries = 0;
|
||||
LOG.debug("Waiting a max of " + max + " ms for snapshot to complete. (max " + maxPauseTime
|
||||
+ " ms per retry)");
|
||||
while ((EnvironmentEdgeManager.currentTimeMillis() - start) < max && !done.getDone()) {
|
||||
while (tries == 0
|
||||
|| ((EnvironmentEdgeManager.currentTimeMillis() - start) < max && !done.getDone())) {
|
||||
try {
|
||||
// sleep a backoff <= pauseTime amount
|
||||
long sleep = getPauseTime(tries++);
|
||||
|
@ -2245,9 +2246,10 @@ public class HBaseAdmin implements Abortable, Closeable {
|
|||
}
|
||||
});
|
||||
}
|
||||
;
|
||||
if (!done.getDone()) {
|
||||
throw new SnapshotCreationException("Snapshot '" + snapshot.getName()
|
||||
+ "' wasn't completed in expectedTime:" + max + " ms");
|
||||
+ "' wasn't completed in expectedTime:" + max + " ms", snapshot);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -135,6 +135,7 @@ 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
|
||||
|
||||
// Updates from master to ZK. This is done by the master and there is
|
||||
// nothing to process by either Master or RS
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
|
@ -49,6 +50,9 @@ 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.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.Chore;
|
||||
|
@ -103,6 +107,7 @@ 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.manage.SnapshotManager;
|
||||
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||
|
@ -113,6 +118,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;
|
||||
|
@ -178,10 +184,18 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Repor
|
|||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
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.snapshot.exception.HBaseSnapshotException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.SnapshotCreationException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.SnapshotDoesNotExistException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.SnapshotExistsException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.TablePartiallyOpenException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.UnknownSnapshotException;
|
||||
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.CompressionTest;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
|
||||
import org.apache.hadoop.hbase.util.HasThread;
|
||||
import org.apache.hadoop.hbase.util.InfoServer;
|
||||
|
@ -201,8 +215,6 @@ import org.apache.hadoop.metrics.util.MBeanUtil;
|
|||
import org.apache.hadoop.net.DNS;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
@ -327,8 +339,12 @@ Server {
|
|||
|
||||
private SpanReceiverHost spanReceiverHost;
|
||||
|
||||
|
||||
private Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
|
||||
|
||||
// monitor for snapshot of hbase tables
|
||||
private SnapshotManager snapshotManager;
|
||||
|
||||
/**
|
||||
* Initializes the HMaster. The steps are as follows:
|
||||
* <p>
|
||||
|
@ -503,6 +519,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");
|
||||
|
@ -567,6 +584,10 @@ Server {
|
|||
", sessionid=0x" +
|
||||
Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()) +
|
||||
", cluster-up flag was=" + wasUp);
|
||||
|
||||
// create the snapshot monitor
|
||||
// TODO should this be config based?
|
||||
this.snapshotManager = new SnapshotManager(this, zooKeeper, this.executorService);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2409,31 +2430,226 @@ Server {
|
|||
return this.hfileCleaner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TakeSnapshotResponse snapshot(RpcController controller, TakeSnapshotRequest request)
|
||||
throws ServiceException {
|
||||
throw new ServiceException(new UnsupportedOperationException(
|
||||
"Snapshots are not implemented yet."));
|
||||
/**
|
||||
* Exposed for TESTING!
|
||||
* @return the underlying snapshot manager
|
||||
*/
|
||||
SnapshotManager getSnapshotManagerForTesting() {
|
||||
return this.snapshotManager;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TakeSnapshotResponse snapshot(RpcController controller, TakeSnapshotRequest request)
|
||||
throws ServiceException {
|
||||
LOG.debug("Starting snapshot for:" + request);
|
||||
// get the snapshot information
|
||||
SnapshotDescription snapshot = SnapshotDescriptionUtils.validate(request.getSnapshot(),
|
||||
this.conf);
|
||||
|
||||
// check to see if we already completed the snapshot
|
||||
if (isSnapshotCompleted(snapshot)) {
|
||||
throw new ServiceException(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 = this.tableDescriptors.get(snapshot.getTable());
|
||||
} catch (FileNotFoundException e) {
|
||||
String msg = "Table:" + snapshot.getTable() + " info doesn't exist!";
|
||||
LOG.error(msg);
|
||||
throw new ServiceException(new SnapshotCreationException(msg, e, snapshot));
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(new SnapshotCreationException(
|
||||
"Error while geting table description for table " + snapshot.getTable(), e, snapshot));
|
||||
}
|
||||
if (desc == null) {
|
||||
throw new ServiceException(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();
|
||||
|
||||
// if the table is enabled, then have the RS run actually the snapshot work
|
||||
if (this.assignmentManager.getZKTable().isEnabledTable(snapshot.getTable())) {
|
||||
LOG.debug("Table enabled, starting distributed snapshot.");
|
||||
throw new ServiceException(new UnsupportedOperationException(
|
||||
"Enabled table snapshots are not yet supported"));
|
||||
}
|
||||
// For disabled table, snapshot is created by the master
|
||||
else if (this.assignmentManager.getZKTable().isDisabledTable(snapshot.getTable())) {
|
||||
LOG.debug("Table is disabled, running snapshot entirely on master.");
|
||||
try {
|
||||
snapshotManager.snapshotDisabledTable(snapshot);
|
||||
} catch (HBaseSnapshotException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
|
||||
LOG.debug("Started snapshot: " + snapshot);
|
||||
} else {
|
||||
LOG.error("Can't snapshot table '" + snapshot.getTable()
|
||||
+ "', isn't open or closed, we don't know what to do!");
|
||||
throw new ServiceException(new SnapshotCreationException(
|
||||
"Table is not entirely open or closed", new TablePartiallyOpenException(
|
||||
snapshot.getTable() + " isn't fully open."), snapshot));
|
||||
}
|
||||
// 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 listSnapshots(RpcController controller, ListSnapshotRequest request)
|
||||
throws ServiceException {
|
||||
throw new ServiceException(new UnsupportedOperationException(
|
||||
"Snapshots are not implemented yet."));
|
||||
try {
|
||||
ListSnapshotResponse.Builder builder = ListSnapshotResponse.newBuilder();
|
||||
|
||||
// first create the snapshot description and check to see if it exists
|
||||
Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(this.getMasterFileSystem()
|
||||
.getRootDir());
|
||||
|
||||
// if there are no snapshots, return an empty list
|
||||
if (!this.getMasterFileSystem().getFileSystem().exists(snapshotDir)) {
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
FileSystem fs = this.getMasterFileSystem().getFileSystem();
|
||||
|
||||
// ignore all the snapshots in progress
|
||||
FileStatus[] snapshots = fs.listStatus(snapshotDir,
|
||||
new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs));
|
||||
// look 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);
|
||||
builder.addSnapshots(desc);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Found a corrupted snapshot " + snapshot.getPath(), e);
|
||||
} finally {
|
||||
if (in != null) {
|
||||
in.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
return builder.build();
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DeleteSnapshotResponse deleteSnapshot(RpcController controller,
|
||||
DeleteSnapshotRequest request) throws ServiceException {
|
||||
throw new ServiceException(new UnsupportedOperationException(
|
||||
"Snapshots are not implemented yet."));
|
||||
try {
|
||||
// check to see if it is completed
|
||||
if (!isSnapshotCompleted(request.getSnapshot())) {
|
||||
throw new SnapshotDoesNotExistException(request.getSnapshot());
|
||||
}
|
||||
|
||||
String snapshotName = request.getSnapshot().getName();
|
||||
LOG.debug("Deleting snapshot: " + snapshotName);
|
||||
// first create the snapshot description and check to see if it exists
|
||||
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, this
|
||||
.getMasterFileSystem().getRootDir());
|
||||
|
||||
// delete the existing snapshot
|
||||
if (!this.getMasterFileSystem().getFileSystem().delete(snapshotDir, true)) {
|
||||
throw new ServiceException("Failed to delete snapshot directory: " + snapshotDir);
|
||||
}
|
||||
return DeleteSnapshotResponse.newBuilder().build();
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public IsSnapshotDoneResponse isSnapshotDone(RpcController controller,
|
||||
IsSnapshotDoneRequest request) throws ServiceException {
|
||||
throw new ServiceException(new UnsupportedOperationException(
|
||||
"Snapshots are not implemented yet."));
|
||||
LOG.debug("Checking to see if snapshot from request:" + request + " is done");
|
||||
try {
|
||||
// check the request to make sure it has a snapshot
|
||||
if (!request.hasSnapshot()) {
|
||||
throw new UnknownSnapshotException(
|
||||
"No snapshot name passed in request, can't figure out which snapshot you want to check.");
|
||||
}
|
||||
|
||||
SnapshotDescription expected = request.getSnapshot();
|
||||
IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder();
|
||||
|
||||
// check to see if the sentinel exists
|
||||
SnapshotSentinel sentinel = this.snapshotManager.getCurrentSnapshotSentinel();
|
||||
if (sentinel != null) {
|
||||
|
||||
// pass on any failure we find in the sentinel
|
||||
HBaseSnapshotException e = sentinel.getExceptionIfFailed();
|
||||
if (e != null) throw e;
|
||||
|
||||
// get the current snapshot and compare it against the requested
|
||||
SnapshotDescription snapshot = sentinel.getSnapshot();
|
||||
LOG.debug("Have a snapshot to compare:" + snapshot);
|
||||
if (expected.getName().equals(snapshot.getName())) {
|
||||
LOG.trace("Running snapshot (" + snapshot.getName() + ") does match request:"
|
||||
+ expected.getName());
|
||||
|
||||
// check to see if we are done
|
||||
if (sentinel.isFinished()) {
|
||||
builder.setDone(true);
|
||||
LOG.debug("Snapshot " + snapshot + " has completed, notifying client.");
|
||||
} else if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Sentinel isn't finished with snapshot!");
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// check to see if the snapshot is already on the fs
|
||||
if (!isSnapshotCompleted(expected)) {
|
||||
throw new UnknownSnapshotException("Snapshot:" + expected.getName()
|
||||
+ " is not currently running or one of the known completed snapshots.");
|
||||
}
|
||||
|
||||
builder.setDone(true);
|
||||
return builder.build();
|
||||
} catch (HBaseSnapshotException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
*/
|
||||
private boolean isSnapshotCompleted(SnapshotDescription snapshot) throws ServiceException {
|
||||
final Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, this
|
||||
.getMasterFileSystem().getRootDir());
|
||||
FileSystem fs = this.getMasterFileSystem().getFileSystem();
|
||||
|
||||
// check to see if the snapshot already exists
|
||||
try {
|
||||
return fs.exists(snapshotDir);
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,52 @@
|
|||
/**
|
||||
* 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.Stoppable;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
|
||||
|
||||
/**
|
||||
* Watch the current snapshot under process
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
public interface SnapshotSentinel extends Stoppable {
|
||||
|
||||
/**
|
||||
* 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();
|
||||
|
||||
/**
|
||||
* @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 <tt>null</tt> if the snapshot succeeded, or the {@link HBaseSnapshotException} that
|
||||
* caused the snapshot to fail.
|
||||
*/
|
||||
public HBaseSnapshotException getExceptionIfFailed();
|
||||
|
||||
}
|
|
@ -0,0 +1,236 @@
|
|||
/**
|
||||
* 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.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.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.catalog.MetaReader;
|
||||
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.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.server.errorhandling.OperationAttemptTimer;
|
||||
import org.apache.hadoop.hbase.server.snapshot.TakeSnapshotUtils;
|
||||
import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
|
||||
import org.apache.hadoop.hbase.server.snapshot.task.CopyRecoveredEditsTask;
|
||||
import org.apache.hadoop.hbase.server.snapshot.task.ReferenceRegionHFilesTask;
|
||||
import org.apache.hadoop.hbase.server.snapshot.task.TableInfoCopyTask;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
/**
|
||||
* 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 EventHandler implements SnapshotSentinel {
|
||||
private static final Log LOG = LogFactory.getLog(DisabledTableSnapshotHandler.class);
|
||||
|
||||
private volatile boolean stopped = false;
|
||||
|
||||
protected final Configuration conf;
|
||||
protected final FileSystem fs;
|
||||
protected final Path rootDir;
|
||||
|
||||
private final MasterServices masterServices;
|
||||
|
||||
private final SnapshotDescription snapshot;
|
||||
|
||||
private final Path workingDir;
|
||||
|
||||
private final String tableName;
|
||||
|
||||
private final OperationAttemptTimer timer;
|
||||
private final SnapshotExceptionSnare monitor;
|
||||
|
||||
private final MasterSnapshotVerifier verify;
|
||||
|
||||
/**
|
||||
* @param snapshot descriptor of the snapshot to take
|
||||
* @param server parent server
|
||||
* @param masterServices master services provider
|
||||
* @throws IOException on unexpected error
|
||||
*/
|
||||
public DisabledTableSnapshotHandler(SnapshotDescription snapshot, Server server,
|
||||
final MasterServices masterServices)
|
||||
throws IOException {
|
||||
super(server, EventType.C_M_SNAPSHOT_TABLE);
|
||||
this.masterServices = masterServices;
|
||||
this.tableName = snapshot.getTable();
|
||||
|
||||
this.snapshot = snapshot;
|
||||
this.monitor = new SnapshotExceptionSnare(snapshot);
|
||||
|
||||
this.conf = this.masterServices.getConfiguration();
|
||||
this.fs = this.masterServices.getMasterFileSystem().getFileSystem();
|
||||
|
||||
this.rootDir = FSUtils.getRootDir(this.conf);
|
||||
this.workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
|
||||
|
||||
// prepare the verify
|
||||
this.verify = new MasterSnapshotVerifier(masterServices, snapshot, rootDir);
|
||||
|
||||
// setup the timer
|
||||
timer = 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 process() {
|
||||
LOG.info("Running table snapshot operation " + eventType + " on table " + tableName);
|
||||
try {
|
||||
timer.start();
|
||||
// write down the snapshot info in the working directory
|
||||
SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, workingDir, this.fs);
|
||||
|
||||
// 1. get all the regions hosting this table.
|
||||
List<Pair<HRegionInfo, ServerName>> regionsAndLocations = null;
|
||||
while (regionsAndLocations == null) {
|
||||
try {
|
||||
regionsAndLocations = MetaReader.getTableRegionsAndLocations(
|
||||
this.server.getCatalogTracker(), Bytes.toBytes(tableName), true);
|
||||
} catch (InterruptedException e) {
|
||||
// check to see if we failed, in which case return
|
||||
if (this.monitor.checkForError()) return;
|
||||
// otherwise, just reset the interrupt and keep on going
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
// 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:"
|
||||
+ 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.failOnError();
|
||||
|
||||
// 2.2 for each region, copy over its recovered.edits directory
|
||||
Path regionDir = HRegion.getRegionDir(rootDir, regionInfo);
|
||||
new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir, snapshotRegionDir).run();
|
||||
monitor.failOnError();
|
||||
|
||||
// 2.3 reference all the files in the region
|
||||
new ReferenceRegionHFilesTask(snapshot, monitor, regionDir, fs, snapshotRegionDir).run();
|
||||
monitor.failOnError();
|
||||
}
|
||||
|
||||
// 3. write the table info to disk
|
||||
LOG.info("Starting to copy tableinfo for offline snapshot:\n" + snapshot);
|
||||
TableInfoCopyTask tableInfo = new TableInfoCopyTask(this.monitor, snapshot, fs,
|
||||
FSUtils.getRootDir(conf));
|
||||
tableInfo.run();
|
||||
monitor.failOnError();
|
||||
|
||||
// 4. verify the snapshot is valid
|
||||
verify.verifySnapshot(this.workingDir, serverNames);
|
||||
|
||||
// 5. complete the snapshot
|
||||
SnapshotDescriptionUtils.completeSnapshot(this.snapshot, this.rootDir, this.workingDir,
|
||||
this.fs);
|
||||
|
||||
} catch (Exception e) {
|
||||
// make sure we capture the exception to propagate back to the client later
|
||||
monitor.snapshotFailure("Failed due to exception:" + e.getMessage(), snapshot, e);
|
||||
} finally {
|
||||
LOG.debug("Marking snapshot" + this.snapshot + " as finished.");
|
||||
this.stopped = true;
|
||||
|
||||
// 6. mark the timer as finished - even if we got an exception, we don't need to time the
|
||||
// operation any further
|
||||
timer.complete();
|
||||
|
||||
LOG.debug("Launching cleanup of working dir:" + workingDir);
|
||||
try {
|
||||
// don't mark the snapshot as a failure if we can't cleanup - the snapshot worked.
|
||||
if (!this.fs.delete(this.workingDir, true)) {
|
||||
LOG.error("Couldn't delete snapshot working directory:" + workingDir);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Couldn't delete snapshot working directory:" + workingDir);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isFinished() {
|
||||
return this.stopped;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SnapshotDescription getSnapshot() {
|
||||
return snapshot;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop(String why) {
|
||||
if (this.stopped) return;
|
||||
this.stopped = true;
|
||||
LOG.info("Stopping disabled snapshot because: " + why);
|
||||
// pass along the stop as a failure. This keeps all the 'should I stop running?' logic in a
|
||||
// single place, though it is technically a little bit of an overload of how the error handler
|
||||
// should be used.
|
||||
this.monitor.snapshotFailure("Failing snapshot because server is stopping.", snapshot);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isStopped() {
|
||||
return this.stopped;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HBaseSnapshotException getExceptionIfFailed() {
|
||||
try {
|
||||
this.monitor.failOnError();
|
||||
} catch (HBaseSnapshotException e) {
|
||||
return e;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,252 @@
|
|||
/**
|
||||
* 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.server.snapshot.TakeSnapshotUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.CorruptedSnapshotException;
|
||||
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>
|
||||
* 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>
|
||||
* <li>HLogs for each server running the snapshot have been referenced
|
||||
* <ul>
|
||||
* <li>Only checked for {@link Type#GLOBAL} snapshots</li>
|
||||
* </ul>
|
||||
* </li>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
public final class MasterSnapshotVerifier {
|
||||
|
||||
private SnapshotDescription snapshot;
|
||||
private FileSystem fs;
|
||||
private Path rootDir;
|
||||
private String tableName;
|
||||
private MasterServices services;
|
||||
|
||||
/**
|
||||
* Build a util for the given snapshot
|
||||
* @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 hlogs, if they exist, are valid
|
||||
if (shouldCheckLogs(snapshot.getType())) {
|
||||
verifyLogs(snapshotDir, snapshotServers);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check to see if the snapshot should verify the logs directory based on the type of the logs.
|
||||
* @param type type of snapshot being taken
|
||||
* @return <tt>true</tt> if the logs directory should be verified, <tt>false</tt> otherwise
|
||||
*/
|
||||
private boolean shouldCheckLogs(Type type) {
|
||||
// This is better handled in the Type enum via type, but since its PB based, this is the
|
||||
// simplest way to handle it
|
||||
return type.equals(Type.GLOBAL);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 the snapshot are valid
|
||||
* @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) {
|
||||
verifyRegion(fs, snapshotDir, region);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify that the region (regioninfo, hfiles) are valid
|
||||
* @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)) {
|
||||
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 = HRegionInfo.parseFrom(in);
|
||||
if (!region.equals(found)) {
|
||||
throw new CorruptedSnapshotException("Found region info (" + found
|
||||
+ ") doesn't match expected region:" + region, snapshot);
|
||||
}
|
||||
|
||||
// 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) && !fs.equals(archived)) {
|
||||
throw new CorruptedSnapshotException("Can't find hfile: " + hfile.getPath()
|
||||
+ " in the real (" + archivedCfDir + ") or archive (" + archivedCfDir
|
||||
+ ") directory for the primary table.", snapshot);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that the logs stored in the log directory for the snapshot are valid - it contains all
|
||||
* the expected logs for all servers involved in the snapshot.
|
||||
* @param snapshotDir snapshot directory to check
|
||||
* @param snapshotServers list of the names of servers involved in the snapshot.
|
||||
* @throws CorruptedSnapshotException if the hlogs in the snapshot are not correct
|
||||
* @throws IOException if we can't reach the filesystem
|
||||
*/
|
||||
private void verifyLogs(Path snapshotDir, Set<String> snapshotServers)
|
||||
throws CorruptedSnapshotException, IOException {
|
||||
Path snapshotLogDir = new Path(snapshotDir, HConstants.HREGION_LOGDIR_NAME);
|
||||
Path logsDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
|
||||
TakeSnapshotUtils.verifyAllLogsGotReferenced(fs, logsDir, snapshotServers, snapshot,
|
||||
snapshotLogDir);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,182 @@
|
|||
/**
|
||||
* 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.manage;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
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.Stoppable;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.SnapshotSentinel;
|
||||
import org.apache.hadoop.hbase.master.snapshot.DisabledTableSnapshotHandler;
|
||||
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.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.SnapshotCreationException;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* This class monitors the whole process of snapshots via ZooKeeper. There is only one
|
||||
* SnapshotMonitor for the master.
|
||||
* <p>
|
||||
* Start monitoring a snapshot by calling method monitor() before the snapshot is started across the
|
||||
* cluster via ZooKeeper. SnapshotMonitor would stop monitoring this snapshot only if it is finished
|
||||
* or aborted.
|
||||
* <p>
|
||||
* Note: There could be only one snapshot being processed and monitored at a time over the cluster.
|
||||
* Start monitoring a snapshot only when the previous one reaches an end status.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
public class SnapshotManager implements Stoppable {
|
||||
private static final Log LOG = LogFactory.getLog(SnapshotManager.class);
|
||||
|
||||
// TODO - enable having multiple snapshots with multiple monitors
|
||||
|
||||
private final MasterServices master;
|
||||
private SnapshotSentinel handler;
|
||||
private ExecutorService pool;
|
||||
private final Path rootDir;
|
||||
|
||||
private boolean stopped;
|
||||
|
||||
public SnapshotManager(final MasterServices master, final ZooKeeperWatcher watcher,
|
||||
final ExecutorService executorService) throws KeeperException {
|
||||
this.master = master;
|
||||
this.pool = executorService;
|
||||
this.rootDir = master.getMasterFileSystem().getRootDir();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return <tt>true</tt> if there is a snapshot currently being taken, <tt>false</tt> otherwise
|
||||
*/
|
||||
public boolean isTakingSnapshot() {
|
||||
return handler != null && !handler.isFinished();
|
||||
}
|
||||
|
||||
/**
|
||||
* 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("Already running another snapshot:"
|
||||
+ this.handler.getSnapshot(), snapshot);
|
||||
}
|
||||
|
||||
try {
|
||||
// delete the working directory, since we aren't running the snapshot
|
||||
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 a disabled table.
|
||||
* <p>
|
||||
* 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
|
||||
*/
|
||||
public synchronized void snapshotDisabledTable(SnapshotDescription snapshot)
|
||||
throws HBaseSnapshotException {
|
||||
// setup the snapshot
|
||||
prepareToTakeSnapshot(snapshot);
|
||||
|
||||
// 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.master);
|
||||
this.handler = handler;
|
||||
this.pool.submit(handler);
|
||||
} catch (IOException e) {
|
||||
// cleanup the working directory
|
||||
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:"
|
||||
+ snapshot);
|
||||
}
|
||||
} catch (IOException e1) {
|
||||
LOG.error("Couldn't delete working directory (" + workingDir + " for snapshot:" + snapshot);
|
||||
}
|
||||
// fail the snapshot
|
||||
throw new SnapshotCreationException("Could not build snapshot handler", e, snapshot);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the current handler for the snapshot
|
||||
*/
|
||||
public SnapshotSentinel getCurrentSnapshotSentinel() {
|
||||
return this.handler;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop(String why) {
|
||||
// short circuit
|
||||
if (this.stopped) return;
|
||||
// make sure we get stop
|
||||
this.stopped = true;
|
||||
// pass the stop onto all the listeners
|
||||
if (this.handler != null) this.handler.stop(why);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isStopped() {
|
||||
return this.stopped;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the handler for the current snapshot
|
||||
* <p>
|
||||
* Exposed for TESTING
|
||||
* @param handler handler the master should use
|
||||
*/
|
||||
public void setSnapshotHandlerForTesting(SnapshotSentinel handler) {
|
||||
this.handler = handler;
|
||||
}
|
||||
}
|
|
@ -781,27 +781,49 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
return this.memstoreSize.getAndAdd(memStoreSize);
|
||||
}
|
||||
|
||||
/*
|
||||
* Write out an info file under the region directory. Useful recovering
|
||||
* mangled regions.
|
||||
/**
|
||||
* Write out an info file under the stored region directory. Useful recovering mangled regions.
|
||||
* @throws IOException
|
||||
*/
|
||||
private void checkRegioninfoOnFilesystem() throws IOException {
|
||||
Path regioninfoPath = new Path(this.regiondir, REGIONINFO_FILE);
|
||||
// Compose the content of the file so we can compare to length in filesystem. If not same,
|
||||
// rewrite it (it may have been written in the old format using Writables instead of pb). The
|
||||
checkRegioninfoOnFilesystem(this.regiondir);
|
||||
}
|
||||
|
||||
/**
|
||||
* Write out an info file under the region directory. Useful recovering mangled regions.
|
||||
* @param regiondir directory under which to write out the region info
|
||||
* @throws IOException
|
||||
*/
|
||||
private void checkRegioninfoOnFilesystem(Path regiondir) throws IOException {
|
||||
writeRegioninfoOnFilesystem(regionInfo, regiondir, getFilesystem(), conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Write out an info file under the region directory. Useful recovering mangled regions. If the
|
||||
* regioninfo already exists on disk and there is information in the file, then we fast exit.
|
||||
* @param regionInfo information about the region
|
||||
* @param regiondir directory under which to write out the region info
|
||||
* @param fs {@link FileSystem} on which to write the region info
|
||||
* @param conf {@link Configuration} from which to extract specific file locations
|
||||
* @throws IOException on unexpected error.
|
||||
*/
|
||||
public static void writeRegioninfoOnFilesystem(HRegionInfo regionInfo, Path regiondir,
|
||||
FileSystem fs, Configuration conf) throws IOException {
|
||||
Path regioninfoPath = new Path(regiondir, REGIONINFO_FILE);
|
||||
// Compose the content of the file so we can compare to length in filesystem. If not same,
|
||||
// rewrite it (it may have been written in the old format using Writables instead of pb). The
|
||||
// pb version is much shorter -- we write now w/o the toString version -- so checking length
|
||||
// only should be sufficient. I don't want to read the file every time to check if it pb
|
||||
// only should be sufficient. I don't want to read the file every time to check if it pb
|
||||
// serialized.
|
||||
byte [] content = getDotRegionInfoFileContent(this.getRegionInfo());
|
||||
boolean exists = this.fs.exists(regioninfoPath);
|
||||
FileStatus status = exists? this.fs.getFileStatus(regioninfoPath): null;
|
||||
byte[] content = getDotRegionInfoFileContent(regionInfo);
|
||||
boolean exists = fs.exists(regioninfoPath);
|
||||
FileStatus status = exists ? fs.getFileStatus(regioninfoPath) : null;
|
||||
if (status != null && status.getLen() == content.length) {
|
||||
// Then assume the content good and move on.
|
||||
return;
|
||||
}
|
||||
// Create in tmpdir and then move into place in case we crash after
|
||||
// create but before close. If we don't successfully close the file,
|
||||
// create but before close. If we don't successfully close the file,
|
||||
// subsequent region reopens will fail the below because create is
|
||||
// registered in NN.
|
||||
|
||||
|
@ -809,7 +831,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
|
||||
|
||||
// And then create the file
|
||||
Path tmpPath = new Path(getTmpDir(), REGIONINFO_FILE);
|
||||
Path tmpPath = new Path(getTmpDir(regiondir), REGIONINFO_FILE);
|
||||
|
||||
// If datanode crashes or if the RS goes down just before the close is called while trying to
|
||||
// close the created regioninfo file in the .tmp directory then on next
|
||||
|
@ -1251,7 +1273,11 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
* will have its contents removed when the region is reopened.
|
||||
*/
|
||||
Path getTmpDir() {
|
||||
return new Path(getRegionDir(), REGION_TEMP_SUBDIR);
|
||||
return getTmpDir(getRegionDir());
|
||||
}
|
||||
|
||||
static Path getTmpDir(Path regionDir) {
|
||||
return new Path(regionDir, REGION_TEMP_SUBDIR);
|
||||
}
|
||||
|
||||
void triggerMajorCompaction() {
|
||||
|
|
|
@ -59,7 +59,7 @@ public class ExceptionOrchestrator<E extends Exception> implements ExceptionList
|
|||
.create();
|
||||
|
||||
/** Error visitor for framework listeners */
|
||||
final ForwardingErrorVisitor genericVisitor = new ForwardingErrorVisitor();
|
||||
public final ForwardingErrorVisitor genericVisitor = new ForwardingErrorVisitor();
|
||||
|
||||
public ExceptionOrchestrator() {
|
||||
this("generic-error-dispatcher");
|
||||
|
|
|
@ -0,0 +1,325 @@
|
|||
/**
|
||||
* 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.server.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.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.server.errorhandling.ExceptionListener;
|
||||
import org.apache.hadoop.hbase.server.errorhandling.OperationAttemptTimer;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.CorruptedSnapshotException;
|
||||
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(SnapshotDescription, Path, String)}
|
||||
* @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
|
||||
*/
|
||||
@SuppressWarnings("rawtypes")
|
||||
public static OperationAttemptTimer getMasterTimerAndBindToMonitor(SnapshotDescription snapshot,
|
||||
Configuration conf, ExceptionListener monitor) {
|
||||
long maxTime = SnapshotDescriptionUtils.getMaxMasterTimeout(conf, snapshot.getType(),
|
||||
SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
|
||||
return new OperationAttemptTimer(monitor, maxTime, snapshot);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 if (realLogs == null) {
|
||||
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));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,71 @@
|
|||
/**
|
||||
* 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.server.snapshot.error;
|
||||
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.server.errorhandling.impl.ExceptionSnare;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.UnexpectedSnapshotException;
|
||||
|
||||
/**
|
||||
* {@link ExceptionSnare} for snapshot exceptions, ensuring that only the first exception is
|
||||
* retained and always returned via {@link #failOnError()}.
|
||||
* <p>
|
||||
* Ensures that any generic exceptions received via
|
||||
* {@link #receiveError(String, HBaseSnapshotException, Object...)} are in fact propagated as
|
||||
* {@link HBaseSnapshotException}.
|
||||
*/
|
||||
public class SnapshotExceptionSnare extends ExceptionSnare<HBaseSnapshotException> implements
|
||||
SnapshotFailureListener {
|
||||
|
||||
private SnapshotDescription snapshot;
|
||||
|
||||
/**
|
||||
* Create a snare that expects errors for the passed snapshot. Any untyped exceptions passed to
|
||||
* {@link #receiveError(String, HBaseSnapshotException, Object...)} are wrapped as an
|
||||
* {@link UnexpectedSnapshotException} with the passed {@link SnapshotDescription}.
|
||||
* @param snapshot
|
||||
*/
|
||||
public SnapshotExceptionSnare(SnapshotDescription snapshot) {
|
||||
this.snapshot = snapshot;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void snapshotFailure(String reason, SnapshotDescription snapshot) {
|
||||
this.receiveError(reason, null, snapshot);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void snapshotFailure(String reason, SnapshotDescription snapshot, Exception t) {
|
||||
this.receiveError(reason,
|
||||
t instanceof HBaseSnapshotException ? (HBaseSnapshotException) t
|
||||
: new UnexpectedSnapshotException(reason, t, snapshot), snapshot);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failOnError() throws HBaseSnapshotException {
|
||||
try {
|
||||
super.failOnError();
|
||||
} catch (Exception e) {
|
||||
if (e instanceof HBaseSnapshotException) {
|
||||
throw (HBaseSnapshotException) e;
|
||||
}
|
||||
throw new UnexpectedSnapshotException(e.getMessage(), e, snapshot);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/**
|
||||
* 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.server.snapshot.error;
|
||||
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
|
||||
/**
|
||||
* Generic running snapshot failure listener
|
||||
*/
|
||||
public interface SnapshotFailureListener {
|
||||
|
||||
/**
|
||||
* Notification that a given snapshot failed because of an error on the local server
|
||||
* @param snapshot snapshot that failed
|
||||
* @param reason explanation of why the snapshot failed
|
||||
*/
|
||||
public void snapshotFailure(String reason, SnapshotDescription snapshot);
|
||||
|
||||
/**
|
||||
* Notification that a given snapshot failed because of an error on the local server
|
||||
* @param reason reason the snapshot failed
|
||||
* @param snapshot the snapshot that failed
|
||||
* @param t the exception that caused the failure
|
||||
*/
|
||||
public void snapshotFailure(String reason, SnapshotDescription snapshot, Exception t);
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
/**
|
||||
* 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.server.snapshot.task;
|
||||
|
||||
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.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
|
||||
|
||||
/**
|
||||
* 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, SnapshotExceptionSnare monitor,
|
||||
FileSystem fs, Path regionDir, Path snapshotRegionDir) {
|
||||
super(snapshot, monitor, "Copy recovered.edits for region:" + regionDir.getName());
|
||||
this.fs = fs;
|
||||
this.regiondir = regionDir;
|
||||
this.outputDir = HLogUtil.getRegionDirRecoveredEditsDir(snapshotRegionDir);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process() throws IOException {
|
||||
NavigableSet<Path> files = HLogUtil.getSplitEditFilesSorted(this.fs, regiondir);
|
||||
if (files == null || files.size() == 0) return;
|
||||
|
||||
// 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.failOnError();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,126 @@
|
|||
/**
|
||||
* 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.server.snapshot.task;
|
||||
|
||||
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.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.server.snapshot.TakeSnapshotUtils;
|
||||
import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
|
||||
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,
|
||||
SnapshotExceptionSnare monitor, Path regionDir, final FileSystem fs, Path regionSnapshotDir) {
|
||||
super(snapshot, monitor, "Reference hfiles for region:" + regionDir.getName());
|
||||
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 process() 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;
|
||||
}
|
||||
|
||||
// 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) {
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,103 @@
|
|||
/**
|
||||
* 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.server.snapshot.task;
|
||||
|
||||
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.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.server.snapshot.TakeSnapshotUtils;
|
||||
import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
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);
|
||||
// XXX does this need to be HasThread?
|
||||
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 fileystem information
|
||||
* @param fs filesystem where the log files are stored and should be referenced
|
||||
* @throws IOException
|
||||
*/
|
||||
public ReferenceServerWALsTask(SnapshotDescription snapshot,
|
||||
SnapshotExceptionSnare failureListener, final Path logDir, final Configuration conf,
|
||||
final FileSystem fs) throws IOException {
|
||||
super(snapshot, failureListener, "Reference WALs for server:" + logDir.getName());
|
||||
this.fs = fs;
|
||||
this.conf = conf;
|
||||
this.serverName = logDir.getName();
|
||||
this.logDir = logDir;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process() throws IOException {
|
||||
// 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.info("No logs for server directory:" + logDir
|
||||
+ ", done referencing files.");
|
||||
|
||||
if (LOG.isDebugEnabled()) LOG.debug("Adding references for WAL files:"
|
||||
+ Arrays.toString(serverLogs));
|
||||
|
||||
for (FileStatus file : serverLogs) {
|
||||
this.failOnError();
|
||||
|
||||
// TODO - switch to using MonitoredTask
|
||||
// add the reference to the file
|
||||
// 0. Build a reference path based on the file name
|
||||
// get the current snapshot directory
|
||||
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");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,81 @@
|
|||
/**
|
||||
* 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.server.snapshot.task;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.server.errorhandling.ExceptionCheckable;
|
||||
import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
|
||||
|
||||
/**
|
||||
* General snapshot operation taken on a regionserver
|
||||
*/
|
||||
public abstract class SnapshotTask implements ExceptionCheckable<HBaseSnapshotException>, Runnable {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(SnapshotTask.class);
|
||||
|
||||
private final SnapshotExceptionSnare errorMonitor;
|
||||
private final String desc;
|
||||
|
||||
protected final SnapshotDescription snapshot;
|
||||
|
||||
/**
|
||||
* @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
|
||||
* @param description description of the task being run, for logging
|
||||
*/
|
||||
public SnapshotTask(SnapshotDescription snapshot, SnapshotExceptionSnare monitor,
|
||||
String description) {
|
||||
this.snapshot = snapshot;
|
||||
this.errorMonitor = monitor;
|
||||
this.desc = description;
|
||||
}
|
||||
|
||||
protected final void snapshotFailure(String message, Exception e) {
|
||||
this.errorMonitor.snapshotFailure(message, this.snapshot, e);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failOnError() throws HBaseSnapshotException {
|
||||
this.errorMonitor.failOnError();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean checkForError() {
|
||||
return this.errorMonitor.checkForError();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
LOG.debug("Running: " + desc);
|
||||
this.process();
|
||||
} catch (Exception e) {
|
||||
this.snapshotFailure("Failed to run " + this.desc, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Run the task for the snapshot.
|
||||
* @throws Exception if the task fails. Will be propagated to any other tasks watching the same
|
||||
* {@link SnapshotErrorListener}.
|
||||
*/
|
||||
protected abstract void process() throws Exception;
|
||||
}
|
|
@ -0,0 +1,75 @@
|
|||
/**
|
||||
* 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.server.snapshot.task;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
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.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
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 failureListener 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(SnapshotExceptionSnare failureListener, SnapshotDescription snapshot,
|
||||
FileSystem fs, Path rootDir) {
|
||||
super(snapshot, failureListener, "Copy table info for table: " + snapshot.getTable());
|
||||
this.rootDir = rootDir;
|
||||
this.fs = fs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process() throws IOException {
|
||||
LOG.debug("Running table info copy.");
|
||||
this.failOnError();
|
||||
LOG.debug("Attempting to copy table info for snapshot:" + this.snapshot);
|
||||
// get the HTable descriptor
|
||||
HTableDescriptor orig = FSTableDescriptors.getTableDescriptor(fs, rootDir,
|
||||
Bytes.toBytes(this.snapshot.getTable()));
|
||||
|
||||
this.failOnError();
|
||||
// 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.");
|
||||
}
|
||||
}
|
|
@ -17,19 +17,128 @@
|
|||
*/
|
||||
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.snapshot.exception.CorruptedSnapshotException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.SnapshotCreationException;
|
||||
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(SnapshotDescription, Path, Path, FileSystem)}, 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";
|
||||
|
||||
private 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_WAIT_TIME_GLOBAL_SNAPSHOT = "hbase.snapshot.global.master.timeout";
|
||||
public static final String REGION_WAIT_TIME_GLOBAL_SNAPSHOT = "hbase.snapshot.global.region.timeout";
|
||||
public static final String MASTER_WAIT_TIME_TIMESTAMP_SNAPSHOT = "hbase.snapshot.timestamp.master.timeout";
|
||||
public static final String REGION_WAIT_TIME_TIMESTAMP_SNAPSHOT = "hbase.snapshot.timestamp.region.timeout";
|
||||
public static final String MASTER_WAIT_TIME_DISABLED_SNAPSHOT = "hbase.snapshot.disabled.master.timeout";
|
||||
|
||||
/** Default timeout of 60 sec for a snapshot timeout on a region */
|
||||
public static final long DEFAULT_REGION_SNAPSHOT_TIMEOUT = 60000;
|
||||
|
||||
/** By default, wait 60 seconds for a snapshot to complete */
|
||||
public static final long DEFAULT_MAX_WAIT_TIME = 60000;
|
||||
|
||||
/**
|
||||
* Conf key for amount of time the in the future a timestamp snapshot should be taken (ms).
|
||||
* Defaults to {@value SnapshotDescriptionUtils#DEFAULT_TIMESTAMP_SNAPSHOT_SPLIT_IN_FUTURE}
|
||||
*/
|
||||
public static final String TIMESTAMP_SNAPSHOT_SPLIT_POINT_ADDITION = "hbase.snapshot.timestamp.master.splittime";
|
||||
/** Start 2 seconds in the future, if no start time given */
|
||||
public static final long DEFAULT_TIMESTAMP_SNAPSHOT_SPLIT_IN_FUTURE = 2000;
|
||||
|
||||
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
|
||||
|
@ -44,4 +153,219 @@ public class SnapshotDescriptionUtils {
|
|||
// 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 GLOBAL:
|
||||
confKey = MASTER_WAIT_TIME_GLOBAL_SNAPSHOT;
|
||||
break;
|
||||
case TIMESTAMP:
|
||||
|
||||
confKey = MASTER_WAIT_TIME_TIMESTAMP_SNAPSHOT;
|
||||
case DISABLED:
|
||||
default:
|
||||
confKey = MASTER_WAIT_TIME_DISABLED_SNAPSHOT;
|
||||
}
|
||||
return conf.getLong(confKey, defaultMaxWaitTime);
|
||||
}
|
||||
|
||||
/**
|
||||
* @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 region should wait for a snapshot to complete
|
||||
*/
|
||||
public static long getMaxRegionTimeout(Configuration conf, SnapshotDescription.Type type,
|
||||
long defaultMaxWaitTime) {
|
||||
String confKey;
|
||||
switch (type) {
|
||||
case GLOBAL:
|
||||
confKey = REGION_WAIT_TIME_GLOBAL_SNAPSHOT;
|
||||
break;
|
||||
case TIMESTAMP:
|
||||
default:
|
||||
confKey = REGION_WAIT_TIME_TIMESTAMP_SNAPSHOT;
|
||||
}
|
||||
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 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(new Path(getSnapshotsDir(rootDir), SNAPSHOT_TMP_DIR_NAME),
|
||||
snapshot.getName());
|
||||
}
|
||||
|
||||
/**
|
||||
* 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();
|
||||
if (snapshot.getType().equals(SnapshotDescription.Type.TIMESTAMP)) {
|
||||
long increment = conf.getLong(
|
||||
SnapshotDescriptionUtils.TIMESTAMP_SNAPSHOT_SPLIT_POINT_ADDITION,
|
||||
SnapshotDescriptionUtils.DEFAULT_TIMESTAMP_SNAPSHOT_SPLIT_IN_FUTURE);
|
||||
LOG.debug("Setting timestamp snasphot in future by " + increment + " ms.");
|
||||
time += increment;
|
||||
}
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
/**
|
||||
* 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.exception;
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
|
@ -15,46 +15,62 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.snapshot;
|
||||
package org.apache.hadoop.hbase.snapshot.exception;
|
||||
|
||||
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 when a snapshot fails.
|
||||
* General exception base class for when a snapshot fails
|
||||
*/
|
||||
@SuppressWarnings("serial")
|
||||
public class HBaseSnapshotException extends HBaseIOException {
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public abstract 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);
|
||||
}
|
||||
|
||||
public HBaseSnapshotException(String msg, Throwable cause) {
|
||||
super(msg, cause);
|
||||
}
|
||||
|
||||
public HBaseSnapshotException(Throwable cause) {
|
||||
super(cause);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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;
|
||||
}
|
||||
|
||||
public HBaseSnapshotException(Throwable cause, SnapshotDescription desc) {
|
||||
super(cause);
|
||||
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;
|
||||
}
|
|
@ -15,7 +15,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.snapshot;
|
||||
package org.apache.hadoop.hbase.snapshot.exception;
|
||||
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
|
||||
|
@ -25,27 +25,30 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescriptio
|
|||
@SuppressWarnings("serial")
|
||||
public class SnapshotCreationException extends HBaseSnapshotException {
|
||||
|
||||
public SnapshotCreationException(String msg, SnapshotDescription desc) {
|
||||
super(msg, desc);
|
||||
}
|
||||
|
||||
public SnapshotCreationException(String msg, Throwable cause, SnapshotDescription desc) {
|
||||
super(msg, cause, desc);
|
||||
}
|
||||
|
||||
public SnapshotCreationException(String msg, Throwable cause) {
|
||||
super(msg, cause);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
|
||||
public SnapshotCreationException(Throwable cause, SnapshotDescription desc) {
|
||||
super(cause, desc);
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
|
||||
public SnapshotCreationException(Throwable cause) {
|
||||
super(cause);
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
}
|
|
@ -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.exception;
|
||||
|
||||
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 desc expected snapshot to find
|
||||
*/
|
||||
public SnapshotDoesNotExistException(SnapshotDescription desc) {
|
||||
super("Snapshot doesn't exist on the filesystem", desc);
|
||||
}
|
||||
|
||||
}
|
|
@ -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.exception;
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
|
@ -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.exception;
|
||||
|
||||
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 partial 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));
|
||||
}
|
||||
}
|
|
@ -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.exception;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
|
||||
/**
|
||||
* General exception when an unexpected error occurs while running a snapshot.
|
||||
*/
|
||||
@SuppressWarnings("serial")
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public class UnexpectedSnapshotException extends HBaseSnapshotException {
|
||||
|
||||
/**
|
||||
* General exception for some cause
|
||||
* @param msg reason why the snapshot couldn't be completed
|
||||
* @param cause root cause of the failure
|
||||
* @param snapshot description of the snapshot attempted
|
||||
*/
|
||||
public UnexpectedSnapshotException(String msg, Exception cause, SnapshotDescription snapshot) {
|
||||
super(msg, cause, snapshot);
|
||||
}
|
||||
|
||||
}
|
|
@ -15,14 +15,23 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.snapshot;
|
||||
package org.apache.hadoop.hbase.snapshot.exception;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* Exception thrown when we get a snapshot error about a snapshot we don't know or recognize.
|
||||
* Exception thrown when we get a request for a snapshot we don't recognize.
|
||||
*/
|
||||
@SuppressWarnings("serial")
|
||||
public class UnknownSnapshotException extends SnapshotCreationException {
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public class UnknownSnapshotException extends HBaseSnapshotException {
|
||||
|
||||
|
||||
/**
|
||||
* @param msg full infomration about the failure
|
||||
*/
|
||||
public UnknownSnapshotException(String msg) {
|
||||
super(msg);
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -43,9 +43,9 @@ import com.google.protobuf.RpcController;
|
|||
* Test snapshot logic from the client
|
||||
*/
|
||||
@Category(SmallTests.class)
|
||||
public class TestSnapshotsFromAdmin {
|
||||
public class TestSnapshotFromAdmin {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestSnapshotsFromAdmin.class);
|
||||
private static final Log LOG = LogFactory.getLog(TestSnapshotFromAdmin.class);
|
||||
|
||||
/**
|
||||
* Test that the logic for doing 'correct' back-off based on exponential increase and the max-time
|
||||
|
@ -80,7 +80,8 @@ public class TestSnapshotsFromAdmin {
|
|||
Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
|
||||
Mockito.when(mockConnection.getKeepAliveMasterAdmin()).thenReturn(mockMaster);
|
||||
// set the max wait time for the snapshot to complete
|
||||
TakeSnapshotResponse response = TakeSnapshotResponse.newBuilder().setExpectedTime(maxWaitTime)
|
||||
TakeSnapshotResponse response = TakeSnapshotResponse.newBuilder()
|
||||
.setExpectedTimeout(maxWaitTime)
|
||||
.build();
|
||||
Mockito
|
||||
.when(
|
||||
|
@ -97,7 +98,7 @@ public class TestSnapshotsFromAdmin {
|
|||
|
||||
// setup the admin and run the test
|
||||
HBaseAdmin admin = new HBaseAdmin(mockConnection);
|
||||
String snapshot = "snasphot";
|
||||
String snapshot = "snapshot";
|
||||
String table = "table";
|
||||
// get start time
|
||||
long start = System.currentTimeMillis();
|
||||
|
@ -105,15 +106,16 @@ public class TestSnapshotsFromAdmin {
|
|||
long finish = System.currentTimeMillis();
|
||||
long elapsed = (finish - start);
|
||||
assertTrue("Elapsed time:" + elapsed + " is more than expected max:" + time, elapsed <= time);
|
||||
admin.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Make sure that we validate the snapshot name and the table name before we pass anything across
|
||||
* the wire
|
||||
* @throws IOException on failure
|
||||
* @throws Exception on failure
|
||||
*/
|
||||
@Test
|
||||
public void testValidateSnapshotName() throws IOException {
|
||||
public void testValidateSnapshotName() throws Exception {
|
||||
HConnectionManager.HConnectionImplementation mockConnection = Mockito
|
||||
.mock(HConnectionManager.HConnectionImplementation.class);
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
|
@ -130,6 +132,21 @@ public class TestSnapshotsFromAdmin {
|
|||
failSnapshotStart(admin, builder.setName("snapshot").setTable("-table").build());
|
||||
failSnapshotStart(admin, builder.setName("snapshot").setTable("table fails").build());
|
||||
failSnapshotStart(admin, builder.setName("snapshot").setTable("tab%le").build());
|
||||
|
||||
// mock the master connection
|
||||
MasterAdminKeepAliveConnection master = Mockito.mock(MasterAdminKeepAliveConnection.class);
|
||||
Mockito.when(mockConnection.getKeepAliveMasterAdmin()).thenReturn(master);
|
||||
TakeSnapshotResponse response = TakeSnapshotResponse.newBuilder().setExpectedTimeout(0).build();
|
||||
Mockito.when(
|
||||
master.snapshot((RpcController) Mockito.isNull(), Mockito.any(TakeSnapshotRequest.class)))
|
||||
.thenReturn(response);
|
||||
IsSnapshotDoneResponse doneResponse = IsSnapshotDoneResponse.newBuilder().setDone(true).build();
|
||||
Mockito.when(
|
||||
master.isSnapshotDone((RpcController) Mockito.isNull(),
|
||||
Mockito.any(IsSnapshotDoneRequest.class))).thenReturn(doneResponse);
|
||||
|
||||
// make sure that we can use valid names
|
||||
admin.snapshot(builder.setName("snapshot").setTable("table").build());
|
||||
}
|
||||
|
||||
private void failSnapshotStart(HBaseAdmin admin, SnapshotDescription snapshot) throws IOException {
|
|
@ -0,0 +1,199 @@
|
|||
/**
|
||||
* 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.client;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
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.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.LargeTests;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.SnapshotCreationException;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Test create/using/deleting snapshots from the client
|
||||
* <p>
|
||||
* This is an end-to-end test for the snapshot utility
|
||||
*/
|
||||
@Category(LargeTests.class)
|
||||
public class TestSnapshotFromClient {
|
||||
private static final Log LOG = LogFactory.getLog(TestSnapshotFromClient.class);
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
private static final int NUM_RS = 2;
|
||||
private static final String STRING_TABLE_NAME = "test";
|
||||
private static final byte[] TEST_FAM = Bytes.toBytes("fam");
|
||||
private static final byte[] TABLE_NAME = Bytes.toBytes(STRING_TABLE_NAME);
|
||||
|
||||
/**
|
||||
* Setup the config for the cluster
|
||||
* @throws Exception on failure
|
||||
*/
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws Exception {
|
||||
setupConf(UTIL.getConfiguration());
|
||||
UTIL.startMiniCluster(NUM_RS);
|
||||
}
|
||||
|
||||
private static void setupConf(Configuration conf) {
|
||||
// disable the ui
|
||||
conf.setInt("hbase.regionsever.info.port", -1);
|
||||
// change the flush size to a small amount, regulating number of store files
|
||||
conf.setInt("hbase.hregion.memstore.flush.size", 25000);
|
||||
// so make sure we get a compaction when doing a load, but keep around some
|
||||
// files in the store
|
||||
conf.setInt("hbase.hstore.compaction.min", 10);
|
||||
conf.setInt("hbase.hstore.compactionThreshold", 10);
|
||||
// block writes if we get to 12 store files
|
||||
conf.setInt("hbase.hstore.blockingStoreFiles", 12);
|
||||
// drop the number of attempts for the hbase admin
|
||||
conf.setInt("hbase.client.retries.number", 1);
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
UTIL.createTable(TABLE_NAME, TEST_FAM);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
UTIL.deleteTable(TABLE_NAME);
|
||||
// and cleanup the archive directory
|
||||
try {
|
||||
UTIL.getTestFileSystem().delete(new Path(UTIL.getDefaultRootDirPath(), ".archive"), true);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failure to delete archive directory", e);
|
||||
}
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void cleanupTest() throws Exception {
|
||||
try {
|
||||
UTIL.shutdownMiniCluster();
|
||||
} catch (Exception e) {
|
||||
LOG.warn("failure shutting down cluster", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test snapshotting a table that is offline
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testOfflineTableSnapshot() throws Exception {
|
||||
HBaseAdmin admin = UTIL.getHBaseAdmin();
|
||||
// make sure we don't fail on listing snapshots
|
||||
SnapshotTestingUtils.assertNoSnapshots(admin);
|
||||
|
||||
// put some stuff in the table
|
||||
HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME);
|
||||
UTIL.loadTable(table, TEST_FAM);
|
||||
|
||||
// get the name of all the regionservers hosting the snapshotted table
|
||||
Set<String> snapshotServers = new HashSet<String>();
|
||||
List<RegionServerThread> servers = UTIL.getMiniHBaseCluster().getLiveRegionServerThreads();
|
||||
for (RegionServerThread server : servers) {
|
||||
if (server.getRegionServer().getOnlineRegions(TABLE_NAME).size() > 0) {
|
||||
snapshotServers.add(server.getRegionServer().getServerName().toString());
|
||||
}
|
||||
}
|
||||
|
||||
LOG.debug("FS state before disable:");
|
||||
FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
|
||||
FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
|
||||
// XXX if this is flakey, might want to consider using the async version and looping as
|
||||
// disableTable can succeed and still timeout.
|
||||
admin.disableTable(TABLE_NAME);
|
||||
|
||||
LOG.debug("FS state before snapshot:");
|
||||
FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
|
||||
FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
|
||||
|
||||
// take a snapshot of the disabled table
|
||||
byte[] snapshot = Bytes.toBytes("offlineTableSnapshot");
|
||||
admin.snapshot(snapshot, TABLE_NAME);
|
||||
LOG.debug("Snapshot completed.");
|
||||
|
||||
// make sure we have the snapshot
|
||||
List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin,
|
||||
snapshot, TABLE_NAME);
|
||||
|
||||
// make sure its a valid snapshot
|
||||
FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
|
||||
Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
|
||||
LOG.debug("FS state after snapshot:");
|
||||
FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
|
||||
FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
|
||||
|
||||
SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), TABLE_NAME, TEST_FAM, rootDir,
|
||||
admin, fs, false, new Path(rootDir, HConstants.HREGION_LOGDIR_NAME), snapshotServers);
|
||||
|
||||
admin.deleteSnapshot(snapshot);
|
||||
snapshots = admin.listSnapshots();
|
||||
SnapshotTestingUtils.assertNoSnapshots(admin);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSnapshotFailsOnNonExistantTable() throws Exception {
|
||||
HBaseAdmin admin = UTIL.getHBaseAdmin();
|
||||
// make sure we don't fail on listing snapshots
|
||||
SnapshotTestingUtils.assertNoSnapshots(admin);
|
||||
String tableName = "_not_a_table";
|
||||
|
||||
// make sure the table doesn't exist
|
||||
boolean fail = false;
|
||||
do {
|
||||
try {
|
||||
admin.getTableDescriptor(Bytes.toBytes(tableName));
|
||||
fail = true;
|
||||
LOG.error("Table:" + tableName + " already exists, checking a new name");
|
||||
tableName = tableName+"!";
|
||||
}catch(TableNotFoundException e) {
|
||||
fail = false;
|
||||
}
|
||||
} while (fail);
|
||||
|
||||
// snapshot the non-existant table
|
||||
try {
|
||||
admin.snapshot("fail", tableName);
|
||||
fail("Snapshot succeeded even though there is not table.");
|
||||
} catch (SnapshotCreationException e) {
|
||||
LOG.info("Correctly failed to snapshot a non-existant table:" + e.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,248 @@
|
|||
/**
|
||||
* 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 static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.master.snapshot.DisabledTableSnapshotHandler;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteSnapshotRequest;
|
||||
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.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.SnapshotCreationException;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.UnknownSnapshotException;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
/**
|
||||
* Test the master-related aspects of a snapshot
|
||||
*/
|
||||
@Category(MediumTests.class)
|
||||
public class TestSnapshotFromMaster {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestSnapshotFromMaster.class);
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
private static final int NUM_RS = 2;
|
||||
private static Path rootDir;
|
||||
private static Path snapshots;
|
||||
private static FileSystem fs;
|
||||
private static HMaster master;
|
||||
|
||||
/**
|
||||
* Setup the config for the cluster
|
||||
*/
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws Exception {
|
||||
setupConf(UTIL.getConfiguration());
|
||||
UTIL.startMiniCluster(NUM_RS);
|
||||
fs = UTIL.getDFSCluster().getFileSystem();
|
||||
rootDir = FSUtils.getRootDir(UTIL.getConfiguration());
|
||||
snapshots = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
|
||||
master = UTIL.getMiniHBaseCluster().getMaster();
|
||||
}
|
||||
|
||||
private static void setupConf(Configuration conf) {
|
||||
// disable the ui
|
||||
conf.setInt("hbase.regionsever.info.port", -1);
|
||||
// change the flush size to a small amount, regulating number of store files
|
||||
conf.setInt("hbase.hregion.memstore.flush.size", 25000);
|
||||
// so make sure we get a compaction when doing a load, but keep around some
|
||||
// files in the store
|
||||
conf.setInt("hbase.hstore.compaction.min", 5);
|
||||
conf.setInt("hbase.hstore.compactionThreshold", 5);
|
||||
// block writes if we get to 12 store files
|
||||
conf.setInt("hbase.hstore.blockingStoreFiles", 12);
|
||||
// drop the number of attempts for the hbase admin
|
||||
conf.setInt("hbase.client.retries.number", 1);
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
master.getSnapshotManagerForTesting().setSnapshotHandlerForTesting(null);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
if (!fs.delete(snapshots, true)) {
|
||||
throw new IOException("Couldn't delete snapshots directory (" + snapshots
|
||||
+ " for an unknown reason");
|
||||
}
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void cleanupTest() throws Exception {
|
||||
try {
|
||||
UTIL.shutdownMiniCluster();
|
||||
} catch (Exception e) {
|
||||
// NOOP;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that the contract from the master for checking on a snapshot are valid.
|
||||
* <p>
|
||||
* <ol>
|
||||
* <li>If a snapshot fails with an error, we expect to get the source error.</li>
|
||||
* <li>If there is no snapshot name supplied, we should get an error.</li>
|
||||
* <li>If asking about a snapshot has hasn't occurred, you should get an error.</li>
|
||||
* </ol>
|
||||
*/
|
||||
@Test(timeout = 15000)
|
||||
public void testIsDoneContract() throws Exception {
|
||||
|
||||
IsSnapshotDoneRequest.Builder builder = IsSnapshotDoneRequest.newBuilder();
|
||||
String snapshotName = "asyncExpectedFailureTest";
|
||||
|
||||
// check that we get an exception when looking up snapshot where one hasn't happened
|
||||
SnapshotTestingUtils.expectSnapshotDoneException(master, builder.build(),
|
||||
UnknownSnapshotException.class);
|
||||
|
||||
// and that we get the same issue, even if we specify a name
|
||||
SnapshotDescription desc = SnapshotDescription.newBuilder().setName(snapshotName).build();
|
||||
builder.setSnapshot(desc);
|
||||
SnapshotTestingUtils.expectSnapshotDoneException(master, builder.build(),
|
||||
UnknownSnapshotException.class);
|
||||
|
||||
// set a mock handler to simulate a snapshot
|
||||
DisabledTableSnapshotHandler mockHandler = Mockito.mock(DisabledTableSnapshotHandler.class);
|
||||
Mockito.when(mockHandler.getExceptionIfFailed()).thenReturn(null);
|
||||
Mockito.when(mockHandler.getSnapshot()).thenReturn(desc);
|
||||
Mockito.when(mockHandler.isFinished()).thenReturn(new Boolean(true));
|
||||
|
||||
master.getSnapshotManagerForTesting().setSnapshotHandlerForTesting(mockHandler);
|
||||
|
||||
// if we do a lookup without a snapshot name, we should fail - you should always know your name
|
||||
builder = IsSnapshotDoneRequest.newBuilder();
|
||||
SnapshotTestingUtils.expectSnapshotDoneException(master, builder.build(),
|
||||
UnknownSnapshotException.class);
|
||||
|
||||
// then do the lookup for the snapshot that it is done
|
||||
builder.setSnapshot(desc);
|
||||
IsSnapshotDoneResponse response = master.isSnapshotDone(null, builder.build());
|
||||
assertTrue("Snapshot didn't complete when it should have.", response.getDone());
|
||||
|
||||
// now try the case where we are looking for a snapshot we didn't take
|
||||
builder.setSnapshot(SnapshotDescription.newBuilder().setName("Not A Snapshot").build());
|
||||
SnapshotTestingUtils.expectSnapshotDoneException(master, builder.build(),
|
||||
UnknownSnapshotException.class);
|
||||
|
||||
// then create a snapshot to the fs and make sure that we can find it when checking done
|
||||
snapshotName = "completed";
|
||||
FileSystem fs = master.getMasterFileSystem().getFileSystem();
|
||||
Path root = master.getMasterFileSystem().getRootDir();
|
||||
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, root);
|
||||
desc = desc.toBuilder().setName(snapshotName).build();
|
||||
SnapshotDescriptionUtils.writeSnapshotInfo(desc, snapshotDir, fs);
|
||||
|
||||
builder.setSnapshot(desc);
|
||||
response = master.isSnapshotDone(null, builder.build());
|
||||
assertTrue("Completed, on-disk snapshot not found", response.getDone());
|
||||
|
||||
HBaseSnapshotException testException = new SnapshotCreationException("test fail", desc);
|
||||
Mockito.when(mockHandler.getExceptionIfFailed()).thenReturn(testException);
|
||||
try {
|
||||
master.isSnapshotDone(null, builder.build());
|
||||
fail("Master should have passed along snapshot error, but didn't");
|
||||
}catch(ServiceException e) {
|
||||
LOG.debug("Correctly got exception back from the master on failure: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testListSnapshots() throws Exception {
|
||||
// first check when there are no snapshots
|
||||
ListSnapshotRequest request = ListSnapshotRequest.newBuilder().build();
|
||||
ListSnapshotResponse response = master.listSnapshots(null, request);
|
||||
assertEquals("Found unexpected number of snapshots", 0, response.getSnapshotsCount());
|
||||
|
||||
// write one snapshot to the fs
|
||||
String snapshotName = "completed";
|
||||
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
|
||||
SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName(snapshotName).build();
|
||||
SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, snapshotDir, fs);
|
||||
|
||||
// check that we get one snapshot
|
||||
response = master.listSnapshots(null, request);
|
||||
assertEquals("Found unexpected number of snapshots", 1, response.getSnapshotsCount());
|
||||
List<SnapshotDescription> snapshots = response.getSnapshotsList();
|
||||
List<SnapshotDescription> expected = Lists.newArrayList(snapshot);
|
||||
assertEquals("Returned snapshots don't match created snapshots", expected, snapshots);
|
||||
|
||||
// write a second snapshot
|
||||
snapshotName = "completed_two";
|
||||
snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
|
||||
snapshot = SnapshotDescription.newBuilder().setName(snapshotName).build();
|
||||
SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, snapshotDir, fs);
|
||||
expected.add(snapshot);
|
||||
|
||||
// check that we get one snapshot
|
||||
response = master.listSnapshots(null, request);
|
||||
assertEquals("Found unexpected number of snapshots", 2, response.getSnapshotsCount());
|
||||
snapshots = response.getSnapshotsList();
|
||||
assertEquals("Returned snapshots don't match created snapshots", expected, snapshots);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeleteSnapshot() throws Exception {
|
||||
|
||||
String snapshotName = "completed";
|
||||
SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName(snapshotName).build();
|
||||
|
||||
DeleteSnapshotRequest request = DeleteSnapshotRequest.newBuilder().setSnapshot(snapshot)
|
||||
.build();
|
||||
try {
|
||||
master.deleteSnapshot(null, request);
|
||||
fail("Master didn't throw exception when attempting to delete snapshot that doesn't exist");
|
||||
} catch (ServiceException e) {
|
||||
LOG.debug("Correctly failed delete of non-existant snapshot:" + e.getMessage());
|
||||
}
|
||||
|
||||
// write one snapshot to the fs
|
||||
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
|
||||
SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, snapshotDir, fs);
|
||||
|
||||
// then delete the existing snapshot,which shouldn't cause an exception to be thrown
|
||||
master.deleteSnapshot(null, request);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,132 @@
|
|||
/**
|
||||
* 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.manage;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
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.snapshot.DisabledTableSnapshotHandler;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.SnapshotCreationException;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
/**
|
||||
* Test basic snapshot manager functionality
|
||||
*/
|
||||
@Category(SmallTests.class)
|
||||
public class TestSnapshotManager {
|
||||
private static final Log LOG = LogFactory.getLog(TestSnapshotManager.class);
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
|
||||
MasterServices services = Mockito.mock(MasterServices.class);
|
||||
ZooKeeperWatcher watcher = Mockito.mock(ZooKeeperWatcher.class);
|
||||
ExecutorService pool = Mockito.mock(ExecutorService.class);
|
||||
MasterFileSystem mfs = Mockito.mock(MasterFileSystem.class);
|
||||
FileSystem fs;
|
||||
{
|
||||
try {
|
||||
fs = UTIL.getTestFileSystem();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("Couldn't get test filesystem", e);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private SnapshotManager getNewManager() throws KeeperException {
|
||||
Mockito.reset(services, watcher, pool);
|
||||
Mockito.when(services.getMasterFileSystem()).thenReturn(mfs);
|
||||
Mockito.when(mfs.getFileSystem()).thenReturn(fs);
|
||||
Mockito.when(mfs.getRootDir()).thenReturn(UTIL.getDataTestDir());
|
||||
return new SnapshotManager(services, watcher, pool);
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Test
|
||||
public void testInProcess() throws KeeperException, SnapshotCreationException {
|
||||
SnapshotManager manager = getNewManager();
|
||||
SnapshotSentinel handler = Mockito.mock(SnapshotSentinel.class);
|
||||
assertFalse("Manager is in process when there is no current handler", manager.isTakingSnapshot());
|
||||
manager.setSnapshotHandlerForTesting(handler);
|
||||
Mockito.when(handler.isFinished()).thenReturn(false);
|
||||
assertTrue("Manager isn't in process when handler is running", manager.isTakingSnapshot());
|
||||
Mockito.when(handler.isFinished()).thenReturn(true);
|
||||
assertFalse("Manager is process when handler isn't running", manager.isTakingSnapshot());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that we stop the running disabled table snapshot by passing along an error to the error
|
||||
* handler.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testStopPropagation() throws Exception {
|
||||
// create a new orchestrator and hook up a listener
|
||||
SnapshotManager manager = getNewManager();
|
||||
FSUtils.setRootDir(UTIL.getConfiguration(), UTIL.getDataTestDir());
|
||||
|
||||
// setup a mock snapshot to run
|
||||
String tableName = "some table";
|
||||
SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("testAbort")
|
||||
.setTable(tableName).build();
|
||||
// mock out all the expected call to the master services
|
||||
// this allows us to run must faster and without using a minicluster
|
||||
|
||||
// ensure the table exists when we ask for it
|
||||
TableDescriptors tables = Mockito.mock(TableDescriptors.class);
|
||||
Mockito.when(services.getTableDescriptors()).thenReturn(tables);
|
||||
HTableDescriptor descriptor = Mockito.mock(HTableDescriptor.class);
|
||||
Mockito.when(tables.get(Mockito.anyString())).thenReturn(descriptor);
|
||||
|
||||
// return the local file system as the backing to the MasterFileSystem
|
||||
MasterFileSystem mfs = Mockito.mock(MasterFileSystem.class);
|
||||
Mockito.when(mfs.getFileSystem()).thenReturn(UTIL.getTestFileSystem());
|
||||
Mockito.when(services.getMasterFileSystem()).thenReturn(mfs);
|
||||
Mockito.when(services.getConfiguration()).thenReturn(UTIL.getConfiguration());
|
||||
|
||||
// create a new handler that we will check for errors
|
||||
manager.snapshotDisabledTable(snapshot);
|
||||
// make sure we submitted the handler, but because its mocked, it doesn't run it.
|
||||
Mockito.verify(pool, Mockito.times(1)).submit(Mockito.any(DisabledTableSnapshotHandler.class));
|
||||
|
||||
// pass along the stop notification
|
||||
manager.stop("stopping for test");
|
||||
SnapshotSentinel handler = manager.getCurrentSnapshotSentinel();
|
||||
assertNotNull("Snare didn't receive error notification from snapshot manager.",
|
||||
handler.getExceptionIfFailed());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,74 @@
|
|||
/**
|
||||
* 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.server.snapshot.error;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.server.errorhandling.ExceptionListener;
|
||||
import org.apache.hadoop.hbase.server.errorhandling.OperationAttemptTimer;
|
||||
import org.apache.hadoop.hbase.server.snapshot.TakeSnapshotUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Test the exception snare propagates errors as expected
|
||||
*/
|
||||
public class TestSnapshotExceptionSnare {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestSnapshotExceptionSnare.class);
|
||||
|
||||
/**
|
||||
* This test ensures that we only propagate snapshot exceptions, even if we don't get a snapshot
|
||||
* exception
|
||||
*/
|
||||
@Test
|
||||
@SuppressWarnings({ "rawtypes", "unchecked" })
|
||||
public void testPropagatesOnlySnapshotException() {
|
||||
SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("snapshot").build();
|
||||
ExceptionListener snare = new SnapshotExceptionSnare(snapshot);
|
||||
snare.receiveError("Some message", new Exception());
|
||||
try {
|
||||
((SnapshotExceptionSnare) snare).failOnError();
|
||||
fail("Snare didn't throw an exception");
|
||||
} catch (HBaseSnapshotException e) {
|
||||
LOG.error("Correctly got a snapshot exception" + e);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPropatesTimerError() {
|
||||
SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("snapshot").build();
|
||||
SnapshotExceptionSnare snare = new SnapshotExceptionSnare(snapshot);
|
||||
Configuration conf = new Configuration();
|
||||
// don't let the timer count down before we fire it off
|
||||
conf.setLong(SnapshotDescriptionUtils.MASTER_WAIT_TIME_DISABLED_SNAPSHOT, Long.MAX_VALUE);
|
||||
OperationAttemptTimer timer = TakeSnapshotUtils.getMasterTimerAndBindToMonitor(snapshot, conf,
|
||||
snare);
|
||||
timer.trigger();
|
||||
try {
|
||||
snare.failOnError();
|
||||
} catch (HBaseSnapshotException e) {
|
||||
LOG.info("Correctly failed from timer:" + e.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,128 @@
|
|||
/**
|
||||
* 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.server.snapshot.task;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
/**
|
||||
* Test that we correctly copy the recovered edits from a directory
|
||||
*/
|
||||
@Category(SmallTests.class)
|
||||
public class TestCopyRecoveredEditsTask {
|
||||
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
|
||||
@Test
|
||||
public void testCopyFiles() throws Exception {
|
||||
|
||||
SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("snapshot").build();
|
||||
SnapshotExceptionSnare monitor = Mockito.mock(SnapshotExceptionSnare.class);
|
||||
FileSystem fs = UTIL.getTestFileSystem();
|
||||
Path root = UTIL.getDataTestDir();
|
||||
String regionName = "regionA";
|
||||
Path regionDir = new Path(root, regionName);
|
||||
Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, root);
|
||||
|
||||
try {
|
||||
// doesn't really matter where the region's snapshot directory is, but this is pretty close
|
||||
Path snapshotRegionDir = new Path(workingDir, regionName);
|
||||
fs.mkdirs(snapshotRegionDir);
|
||||
|
||||
// put some stuff in the recovered.edits directory
|
||||
Path edits = HLogUtil.getRegionDirRecoveredEditsDir(regionDir);
|
||||
fs.mkdirs(edits);
|
||||
// make a file with some data
|
||||
Path file1 = new Path(edits, "0000000000000002352");
|
||||
FSDataOutputStream out = fs.create(file1);
|
||||
byte[] data = new byte[] { 1, 2, 3, 4 };
|
||||
out.write(data);
|
||||
out.close();
|
||||
// make an empty file
|
||||
Path empty = new Path(edits, "empty");
|
||||
fs.createNewFile(empty);
|
||||
|
||||
CopyRecoveredEditsTask task = new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir,
|
||||
snapshotRegionDir);
|
||||
task.run();
|
||||
|
||||
Path snapshotEdits = HLogUtil.getRegionDirRecoveredEditsDir(snapshotRegionDir);
|
||||
FileStatus[] snapshotEditFiles = FSUtils.listStatus(fs, snapshotEdits);
|
||||
assertEquals("Got wrong number of files in the snapshot edits", 1, snapshotEditFiles.length);
|
||||
FileStatus file = snapshotEditFiles[0];
|
||||
assertEquals("Didn't copy expected file", file1.getName(), file.getPath().getName());
|
||||
|
||||
Mockito.verify(monitor, Mockito.never()).receiveError(Mockito.anyString(),
|
||||
Mockito.any(HBaseSnapshotException.class));
|
||||
Mockito.verify(monitor, Mockito.never()).snapshotFailure(Mockito.anyString(),
|
||||
Mockito.any(SnapshotDescription.class));
|
||||
Mockito.verify(monitor, Mockito.never()).snapshotFailure(Mockito.anyString(),
|
||||
Mockito.any(SnapshotDescription.class), Mockito.any(Exception.class));
|
||||
} finally {
|
||||
// cleanup the working directory
|
||||
FSUtils.delete(fs, regionDir, true);
|
||||
FSUtils.delete(fs, workingDir, true);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that we don't get an exception if there is no recovered edits directory to copy
|
||||
* @throws Exception on failure
|
||||
*/
|
||||
@Test
|
||||
public void testNoEditsDir() throws Exception {
|
||||
SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("snapshot").build();
|
||||
SnapshotExceptionSnare monitor = Mockito.mock(SnapshotExceptionSnare.class);
|
||||
FileSystem fs = UTIL.getTestFileSystem();
|
||||
Path root = UTIL.getDataTestDir();
|
||||
String regionName = "regionA";
|
||||
Path regionDir = new Path(root, regionName);
|
||||
Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, root);
|
||||
try {
|
||||
// doesn't really matter where the region's snapshot directory is, but this is pretty close
|
||||
Path snapshotRegionDir = new Path(workingDir, regionName);
|
||||
fs.mkdirs(snapshotRegionDir);
|
||||
Path regionEdits = HLogUtil.getRegionDirRecoveredEditsDir(regionDir);
|
||||
assertFalse("Edits dir exists already - it shouldn't", fs.exists(regionEdits));
|
||||
|
||||
CopyRecoveredEditsTask task = new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir,
|
||||
snapshotRegionDir);
|
||||
task.run();
|
||||
} finally {
|
||||
// cleanup the working directory
|
||||
FSUtils.delete(fs, regionDir, true);
|
||||
FSUtils.delete(fs, workingDir, true);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,92 @@
|
|||
/**
|
||||
* 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.server.snapshot.task;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestReferenceRegionHFilesTask {
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
|
||||
@Test
|
||||
public void testRun() throws IOException {
|
||||
FileSystem fs = UTIL.getTestFileSystem();
|
||||
// setup the region internals
|
||||
Path testdir = UTIL.getDataTestDir();
|
||||
Path regionDir = new Path(testdir, "region");
|
||||
Path family1 = new Path(regionDir, "fam1");
|
||||
// make an empty family
|
||||
Path family2 = new Path(regionDir, "fam2");
|
||||
fs.mkdirs(family2);
|
||||
|
||||
// add some files to family 1
|
||||
Path file1 = new Path(family1, "05f99689ae254693836613d1884c6b63");
|
||||
fs.createNewFile(file1);
|
||||
Path file2 = new Path(family1, "7ac9898bf41d445aa0003e3d699d5d26");
|
||||
fs.createNewFile(file2);
|
||||
|
||||
// create the snapshot directory
|
||||
Path snapshotRegionDir = new Path(testdir, HConstants.SNAPSHOT_DIR_NAME);
|
||||
fs.mkdirs(snapshotRegionDir);
|
||||
|
||||
SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("name")
|
||||
.setTable("table").build();
|
||||
SnapshotExceptionSnare monitor = Mockito.mock(SnapshotExceptionSnare.class);
|
||||
ReferenceRegionHFilesTask task = new ReferenceRegionHFilesTask(snapshot, monitor, regionDir,
|
||||
fs, snapshotRegionDir);
|
||||
task.run();
|
||||
|
||||
// make sure we never get an error
|
||||
Mockito.verify(monitor, Mockito.never()).snapshotFailure(Mockito.anyString(),
|
||||
Mockito.eq(snapshot));
|
||||
Mockito.verify(monitor, Mockito.never()).snapshotFailure(Mockito.anyString(),
|
||||
Mockito.eq(snapshot), Mockito.any(Exception.class));
|
||||
|
||||
// verify that all the hfiles get referenced
|
||||
List<String> hfiles = new ArrayList<String>(2);
|
||||
FileStatus[] regions = FSUtils.listStatus(fs, snapshotRegionDir);
|
||||
for (FileStatus region : regions) {
|
||||
FileStatus[] fams = FSUtils.listStatus(fs, region.getPath());
|
||||
for (FileStatus fam : fams) {
|
||||
FileStatus[] files = FSUtils.listStatus(fs, fam.getPath());
|
||||
for (FileStatus file : files) {
|
||||
hfiles.add(file.getPath().getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
assertTrue("Didn't reference :" + file1, hfiles.contains(file1.getName()));
|
||||
assertTrue("Didn't reference :" + file1, hfiles.contains(file2.getName()));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,52 @@
|
|||
/**
|
||||
* 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.server.snapshot.task;
|
||||
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestSnapshotTask {
|
||||
|
||||
/**
|
||||
* Check that errors from running the task get propagated back to the error listener.
|
||||
*/
|
||||
@Test
|
||||
public void testErrorPropagationg() {
|
||||
SnapshotExceptionSnare error = Mockito.mock(SnapshotExceptionSnare.class);
|
||||
SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("snapshot")
|
||||
.setTable("table").build();
|
||||
final Exception thrown = new Exception("Failed!");
|
||||
SnapshotTask fail = new SnapshotTask(snapshot, error, "always fails") {
|
||||
|
||||
@Override
|
||||
protected void process() throws Exception {
|
||||
throw thrown;
|
||||
}
|
||||
};
|
||||
fail.run();
|
||||
|
||||
Mockito.verify(error, Mockito.times(1)).snapshotFailure(Mockito.anyString(),
|
||||
Mockito.eq(snapshot), Mockito.eq(thrown));
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,99 @@
|
|||
/**
|
||||
* 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.server.snapshot.task;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
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.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.server.snapshot.TakeSnapshotUtils;
|
||||
import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
/**
|
||||
* Test that the WAL reference task works as expected
|
||||
*/
|
||||
public class TestWALReferenceTask {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestWALReferenceTask.class);
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
|
||||
@Test
|
||||
public void testRun() throws IOException {
|
||||
Configuration conf = UTIL.getConfiguration();
|
||||
FileSystem fs = UTIL.getTestFileSystem();
|
||||
// setup the log dir
|
||||
Path testDir = UTIL.getDataTestDir();
|
||||
Set<String> servers = new HashSet<String>();
|
||||
Path logDir = new Path(testDir, ".logs");
|
||||
Path server1Dir = new Path(logDir, "Server1");
|
||||
servers.add(server1Dir.getName());
|
||||
Path server2Dir = new Path(logDir, "me.hbase.com,56073,1348618509968");
|
||||
servers.add(server2Dir.getName());
|
||||
// logs under server 1
|
||||
Path log1_1 = new Path(server1Dir, "me.hbase.com%2C56073%2C1348618509968.1348618520536");
|
||||
Path log1_2 = new Path(server1Dir, "me.hbase.com%2C56073%2C1348618509968.1234567890123");
|
||||
// logs under server 2
|
||||
Path log2_1 = new Path(server2Dir, "me.hbase.com%2C56074%2C1348618509998.1348618515589");
|
||||
Path log2_2 = new Path(server2Dir, "me.hbase.com%2C56073%2C1348618509968.1234567890123");
|
||||
|
||||
// create all the log files
|
||||
fs.createNewFile(log1_1);
|
||||
fs.createNewFile(log1_2);
|
||||
fs.createNewFile(log2_1);
|
||||
fs.createNewFile(log2_2);
|
||||
|
||||
FSUtils.logFileSystemState(fs, testDir, LOG);
|
||||
FSUtils.setRootDir(conf, testDir);
|
||||
SnapshotDescription snapshot = SnapshotDescription.newBuilder()
|
||||
.setName("testWALReferenceSnapshot").build();
|
||||
SnapshotExceptionSnare listener = Mockito.mock(SnapshotExceptionSnare.class);
|
||||
|
||||
// reference all the files in the first server directory
|
||||
ReferenceServerWALsTask task = new ReferenceServerWALsTask(snapshot, listener, server1Dir,
|
||||
conf, fs);
|
||||
task.run();
|
||||
|
||||
// reference all the files in the first server directory
|
||||
task = new ReferenceServerWALsTask(snapshot, listener, server2Dir, conf, fs);
|
||||
task.run();
|
||||
|
||||
// verify that we got everything
|
||||
FSUtils.logFileSystemState(fs, testDir, LOG);
|
||||
Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, testDir);
|
||||
Path snapshotLogDir = new Path(workingDir, HConstants.HREGION_LOGDIR_NAME);
|
||||
|
||||
// make sure we reference the all the wal files
|
||||
TakeSnapshotUtils.verifyAllLogsGotReferenced(fs, logDir, servers, snapshot, snapshotLogDir);
|
||||
|
||||
// make sure we never got an error
|
||||
Mockito.verify(listener, Mockito.atLeastOnce()).failOnError();
|
||||
Mockito.verifyNoMoreInteractions(listener);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,188 @@
|
|||
/**
|
||||
* 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 static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
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.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneResponse;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.server.snapshot.TakeSnapshotUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
import org.junit.Assert;
|
||||
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
/**
|
||||
* Utilities class for snapshots
|
||||
*/
|
||||
public class SnapshotTestingUtils {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(SnapshotTestingUtils.class);
|
||||
|
||||
/**
|
||||
* Assert that we don't have any snapshots lists
|
||||
* @throws IOException if the admin operation fails
|
||||
*/
|
||||
public static void assertNoSnapshots(HBaseAdmin admin) throws IOException {
|
||||
assertEquals("Have some previous snapshots", 0, admin.listSnapshots().size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Make sure that there is only one snapshot returned from the master and its name and table match
|
||||
* the passed in parameters.
|
||||
*/
|
||||
public static void assertOneSnapshotThatMatches(HBaseAdmin admin, SnapshotDescription snapshot)
|
||||
throws IOException {
|
||||
assertOneSnapshotThatMatches(admin, snapshot.getName(), snapshot.getTable());
|
||||
}
|
||||
|
||||
/**
|
||||
* Make sure that there is only one snapshot returned from the master and its name and table match
|
||||
* the passed in parameters.
|
||||
*/
|
||||
public static List<SnapshotDescription> assertOneSnapshotThatMatches(HBaseAdmin admin,
|
||||
String snapshotName, String tableName) throws IOException {
|
||||
// list the snapshot
|
||||
List<SnapshotDescription> snapshots = admin.listSnapshots();
|
||||
|
||||
assertEquals("Should only have 1 snapshot", 1, snapshots.size());
|
||||
assertEquals(snapshotName, snapshots.get(0).getName());
|
||||
assertEquals(tableName, snapshots.get(0).getTable());
|
||||
|
||||
return snapshots;
|
||||
}
|
||||
|
||||
/**
|
||||
* Make sure that there is only one snapshot returned from the master and its name and table match
|
||||
* the passed in parameters.
|
||||
*/
|
||||
public static List<SnapshotDescription> assertOneSnapshotThatMatches(HBaseAdmin admin,
|
||||
byte[] snapshot, byte[] tableName) throws IOException {
|
||||
return assertOneSnapshotThatMatches(admin, Bytes.toString(snapshot), Bytes.toString(tableName));
|
||||
}
|
||||
|
||||
/**
|
||||
* Confirm that the snapshot contains references to all the files that should be in the snapshot
|
||||
*/
|
||||
public static void confirmSnapshotValid(SnapshotDescription snapshotDescriptor, byte[] tableName,
|
||||
byte[] testFamily, Path rootDir, HBaseAdmin admin, FileSystem fs, boolean requireLogs,
|
||||
Path logsDir, Set<String> snapshotServers) throws IOException {
|
||||
Path snapshotDir = SnapshotDescriptionUtils
|
||||
.getCompletedSnapshotDir(snapshotDescriptor, rootDir);
|
||||
assertTrue(fs.exists(snapshotDir));
|
||||
Path snapshotinfo = new Path(snapshotDir, SnapshotDescriptionUtils.SNAPSHOTINFO_FILE);
|
||||
assertTrue(fs.exists(snapshotinfo));
|
||||
// check the logs dir
|
||||
if (requireLogs) {
|
||||
TakeSnapshotUtils.verifyAllLogsGotReferenced(fs, logsDir, snapshotServers,
|
||||
snapshotDescriptor, new Path(snapshotDir, HConstants.HREGION_LOGDIR_NAME));
|
||||
}
|
||||
// check the table info
|
||||
HTableDescriptor desc = FSTableDescriptors.getTableDescriptor(fs, rootDir, tableName);
|
||||
HTableDescriptor snapshotDesc = FSTableDescriptors.getTableDescriptor(fs, snapshotDir);
|
||||
assertEquals(desc, snapshotDesc);
|
||||
|
||||
// check the region snapshot for all the regions
|
||||
List<HRegionInfo> regions = admin.getTableRegions(tableName);
|
||||
for (HRegionInfo info : regions) {
|
||||
String regionName = info.getEncodedName();
|
||||
Path regionDir = new Path(snapshotDir, regionName);
|
||||
HRegionInfo snapshotRegionInfo = HRegion.loadDotRegionInfoFileContent(fs, regionDir);
|
||||
assertEquals(info, snapshotRegionInfo);
|
||||
// check to make sure we have the family
|
||||
Path familyDir = new Path(regionDir, Bytes.toString(testFamily));
|
||||
assertTrue("Expected to find: " + familyDir + ", but it doesn't exist", fs.exists(familyDir));
|
||||
// make sure we have some files references
|
||||
assertTrue(fs.listStatus(familyDir).length > 0);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method for testing async snapshot operations. Just waits for the given snapshot to
|
||||
* complete on the server by repeatedly checking the master.
|
||||
* @param master running the snapshot
|
||||
* @param snapshot to check
|
||||
* @param sleep amount to sleep between checks to see if the snapshot is done
|
||||
* @throws ServiceException if the snapshot fails
|
||||
*/
|
||||
public static void waitForSnapshotToComplete(HMaster master, SnapshotDescription snapshot,
|
||||
long sleep) throws ServiceException {
|
||||
final IsSnapshotDoneRequest request = IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot)
|
||||
.build();
|
||||
IsSnapshotDoneResponse done = IsSnapshotDoneResponse.newBuilder().buildPartial();
|
||||
while (!done.getDone()) {
|
||||
done = master.isSnapshotDone(null, request);
|
||||
try {
|
||||
Thread.sleep(sleep);
|
||||
} catch (InterruptedException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void cleanupSnapshot(HBaseAdmin admin, byte[] tableName) throws IOException {
|
||||
SnapshotTestingUtils.cleanupSnapshot(admin, Bytes.toString(tableName));
|
||||
}
|
||||
|
||||
public static void cleanupSnapshot(HBaseAdmin admin, String snapshotName) throws IOException {
|
||||
// delete the taken snapshot
|
||||
admin.deleteSnapshot(snapshotName);
|
||||
assertNoSnapshots(admin);
|
||||
}
|
||||
|
||||
/**
|
||||
* Expect the snapshot to throw an error when checking if the snapshot is complete
|
||||
* @param master master to check
|
||||
* @param snapshot the {@link SnapshotDescription} request to pass to the master
|
||||
* @param clazz expected exception from the master
|
||||
*/
|
||||
public static void expectSnapshotDoneException(HMaster master, IsSnapshotDoneRequest snapshot,
|
||||
Class<? extends HBaseSnapshotException> clazz) {
|
||||
try {
|
||||
master.isSnapshotDone(null, snapshot);
|
||||
Assert.fail("didn't fail to lookup a snapshot");
|
||||
} catch (ServiceException se) {
|
||||
try {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
} catch (HBaseSnapshotException e) {
|
||||
assertEquals("Threw wrong snapshot exception!", clazz, e.getClass());
|
||||
} catch (Throwable t) {
|
||||
Assert.fail("Threw an unexpected exception:" + t);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,138 @@
|
|||
/**
|
||||
* 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 static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
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.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdge;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
|
||||
import org.junit.After;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Test that the {@link SnapshotDescription} helper is helping correctly.
|
||||
*/
|
||||
@Category(SmallTests.class)
|
||||
public class TestSnapshotDescriptionUtils {
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
private static FileSystem fs;
|
||||
private static Path root;
|
||||
|
||||
@BeforeClass
|
||||
public static void setupFS() throws Exception {
|
||||
fs = UTIL.getTestFileSystem();
|
||||
root = new Path(UTIL.getDataTestDir(), "hbase");
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanupFS() throws Exception {
|
||||
if (fs.exists(root)) {
|
||||
if (!fs.delete(root, true)) {
|
||||
throw new IOException("Failed to delete root test dir: " + root);
|
||||
}
|
||||
if (!fs.mkdirs(root)) {
|
||||
throw new IOException("Failed to create root test dir: " + root);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestSnapshotDescriptionUtils.class);
|
||||
|
||||
@Test
|
||||
public void testValidateDescriptor() {
|
||||
EnvironmentEdge edge = new EnvironmentEdge() {
|
||||
@Override
|
||||
public long currentTimeMillis() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
EnvironmentEdgeManagerTestHelper.injectEdge(edge);
|
||||
|
||||
// check a basic snapshot description
|
||||
SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
|
||||
builder.setName("snapshot");
|
||||
builder.setTable("table");
|
||||
|
||||
// check that time is to an amount in the future
|
||||
Configuration conf = new Configuration(false);
|
||||
conf.setLong(SnapshotDescriptionUtils.TIMESTAMP_SNAPSHOT_SPLIT_POINT_ADDITION, 1);
|
||||
SnapshotDescription desc = SnapshotDescriptionUtils.validate(builder.build(), conf);
|
||||
assertEquals("Description creation time wasn't set correctly", 1, desc.getCreationTime());
|
||||
|
||||
// test a global snapshot
|
||||
edge = new EnvironmentEdge() {
|
||||
@Override
|
||||
public long currentTimeMillis() {
|
||||
return 2;
|
||||
}
|
||||
};
|
||||
EnvironmentEdgeManagerTestHelper.injectEdge(edge);
|
||||
builder.setType(Type.GLOBAL);
|
||||
desc = SnapshotDescriptionUtils.validate(builder.build(), conf);
|
||||
assertEquals("Description creation time wasn't set correctly", 2, desc.getCreationTime());
|
||||
|
||||
// test that we don't override a given value
|
||||
builder.setCreationTime(10);
|
||||
desc = SnapshotDescriptionUtils.validate(builder.build(), conf);
|
||||
assertEquals("Description creation time wasn't set correctly", 10, desc.getCreationTime());
|
||||
|
||||
try {
|
||||
SnapshotDescriptionUtils.validate(SnapshotDescription.newBuilder().setName("fail").build(),
|
||||
conf);
|
||||
fail("Snapshot was considered valid without a table name");
|
||||
} catch (IllegalArgumentException e) {
|
||||
LOG.debug("Correctly failed when snapshot doesn't have a tablename");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that we throw an exception if there is no working snapshot directory when we attempt to
|
||||
* 'complete' the snapshot
|
||||
* @throws Exception on failure
|
||||
*/
|
||||
@Test
|
||||
public void testCompleteSnapshotWithNoSnapshotDirectoryFailure() throws Exception {
|
||||
Path snapshotDir = new Path(root, ".snapshot");
|
||||
Path tmpDir = new Path(snapshotDir, ".tmp");
|
||||
Path workingDir = new Path(tmpDir, "not_a_snapshot");
|
||||
assertFalse("Already have working snapshot dir: " + workingDir
|
||||
+ " but shouldn't. Test file leak?", fs.exists(workingDir));
|
||||
SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("snapshot").build();
|
||||
try {
|
||||
SnapshotDescriptionUtils.completeSnapshot(snapshot, root, workingDir, fs);
|
||||
fail("Shouldn't successfully complete move of a non-existent directory.");
|
||||
} catch (IOException e) {
|
||||
LOG.info("Correctly failed to move non-existant directory: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue