diff --git hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java index 13ce610..eb162e0 100644 --- hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java +++ hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java @@ -4564,6 +4564,26 @@ public final class BackupProtos { * required bool compacted = 11; */ boolean getCompacted(); + + // repeated string rel_wal_ref = 12; + /** + * repeated string rel_wal_ref = 12; + */ + java.util.List + getRelWalRefList(); + /** + * repeated string rel_wal_ref = 12; + */ + int getRelWalRefCount(); + /** + * repeated string rel_wal_ref = 12; + */ + java.lang.String getRelWalRef(int index); + /** + * repeated string rel_wal_ref = 12; + */ + com.google.protobuf.ByteString + getRelWalRefBytes(int index); } /** * Protobuf type {@code hbase.pb.BackupManifest} @@ -4686,6 +4706,14 @@ public final class BackupProtos { compacted_ = input.readBool(); break; } + case 98: { + if (!((mutable_bitField0_ & 0x00000800) == 0x00000800)) { + relWalRef_ = new com.google.protobuf.LazyStringArrayList(); + mutable_bitField0_ |= 0x00000800; + } + relWalRef_.add(input.readBytes()); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -4703,6 +4731,9 @@ public final class BackupProtos { if (((mutable_bitField0_ & 0x00000200) == 0x00000200)) { dependentBackupImage_ = java.util.Collections.unmodifiableList(dependentBackupImage_); } + if (((mutable_bitField0_ & 0x00000800) == 0x00000800)) { + relWalRef_ = new com.google.protobuf.UnmodifiableLazyStringList(relWalRef_); + } this.unknownFields = unknownFields.build(); makeExtensionsImmutable(); } @@ -5025,6 +5056,36 @@ public final class BackupProtos { return compacted_; } + // repeated string rel_wal_ref = 12; + public static final int REL_WAL_REF_FIELD_NUMBER = 12; + private com.google.protobuf.LazyStringList relWalRef_; + /** + * repeated string rel_wal_ref = 12; + */ + public java.util.List + getRelWalRefList() { + return relWalRef_; + } + /** + * repeated string rel_wal_ref = 12; + */ + public int getRelWalRefCount() { + return relWalRef_.size(); + } + /** + * repeated string rel_wal_ref = 12; + */ + public java.lang.String getRelWalRef(int index) { + return relWalRef_.get(index); + } + /** + * repeated string rel_wal_ref = 12; + */ + public com.google.protobuf.ByteString + getRelWalRefBytes(int index) { + return relWalRef_.getByteString(index); + } + private void initFields() { version_ = ""; backupId_ = ""; @@ -5037,6 +5098,7 @@ public final class BackupProtos { tstMap_ = java.util.Collections.emptyList(); dependentBackupImage_ = java.util.Collections.emptyList(); compacted_ = false; + relWalRef_ = com.google.protobuf.LazyStringArrayList.EMPTY; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -5129,6 +5191,9 @@ public final class BackupProtos { if (((bitField0_ & 0x00000080) == 0x00000080)) { output.writeBool(11, compacted_); } + for (int i = 0; i < relWalRef_.size(); i++) { + output.writeBytes(12, relWalRef_.getByteString(i)); + } getUnknownFields().writeTo(output); } @@ -5182,6 +5247,15 @@ public final class BackupProtos { size += com.google.protobuf.CodedOutputStream .computeBoolSize(11, compacted_); } + { + int dataSize = 0; + for (int i = 0; i < relWalRef_.size(); i++) { + dataSize += com.google.protobuf.CodedOutputStream + .computeBytesSizeNoTag(relWalRef_.getByteString(i)); + } + size += dataSize; + size += 1 * getRelWalRefList().size(); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -5251,6 +5325,8 @@ public final class BackupProtos { result = result && (getCompacted() == other.getCompacted()); } + result = result && getRelWalRefList() + .equals(other.getRelWalRefList()); result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -5308,6 +5384,10 @@ public final class BackupProtos { hash = (37 * hash) + COMPACTED_FIELD_NUMBER; hash = (53 * hash) + hashBoolean(getCompacted()); } + if (getRelWalRefCount() > 0) { + hash = (37 * hash) + REL_WAL_REF_FIELD_NUMBER; + hash = (53 * hash) + getRelWalRefList().hashCode(); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -5454,6 +5534,8 @@ public final class BackupProtos { } compacted_ = false; bitField0_ = (bitField0_ & ~0x00000400); + relWalRef_ = com.google.protobuf.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000800); return this; } @@ -5541,6 +5623,12 @@ public final class BackupProtos { to_bitField0_ |= 0x00000080; } result.compacted_ = compacted_; + if (((bitField0_ & 0x00000800) == 0x00000800)) { + relWalRef_ = new com.google.protobuf.UnmodifiableLazyStringList( + relWalRef_); + bitField0_ = (bitField0_ & ~0x00000800); + } + result.relWalRef_ = relWalRef_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -5663,6 +5751,16 @@ public final class BackupProtos { if (other.hasCompacted()) { setCompacted(other.getCompacted()); } + if (!other.relWalRef_.isEmpty()) { + if (relWalRef_.isEmpty()) { + relWalRef_ = other.relWalRef_; + bitField0_ = (bitField0_ & ~0x00000800); + } else { + ensureRelWalRefIsMutable(); + relWalRef_.addAll(other.relWalRef_); + } + onChanged(); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -6805,6 +6903,99 @@ public final class BackupProtos { return this; } + // repeated string rel_wal_ref = 12; + private com.google.protobuf.LazyStringList relWalRef_ = com.google.protobuf.LazyStringArrayList.EMPTY; + private void ensureRelWalRefIsMutable() { + if (!((bitField0_ & 0x00000800) == 0x00000800)) { + relWalRef_ = new com.google.protobuf.LazyStringArrayList(relWalRef_); + bitField0_ |= 0x00000800; + } + } + /** + * repeated string rel_wal_ref = 12; + */ + public java.util.List + getRelWalRefList() { + return java.util.Collections.unmodifiableList(relWalRef_); + } + /** + * repeated string rel_wal_ref = 12; + */ + public int getRelWalRefCount() { + return relWalRef_.size(); + } + /** + * repeated string rel_wal_ref = 12; + */ + public java.lang.String getRelWalRef(int index) { + return relWalRef_.get(index); + } + /** + * repeated string rel_wal_ref = 12; + */ + public com.google.protobuf.ByteString + getRelWalRefBytes(int index) { + return relWalRef_.getByteString(index); + } + /** + * repeated string rel_wal_ref = 12; + */ + public Builder setRelWalRef( + int index, java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureRelWalRefIsMutable(); + relWalRef_.set(index, value); + onChanged(); + return this; + } + /** + * repeated string rel_wal_ref = 12; + */ + public Builder addRelWalRef( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureRelWalRefIsMutable(); + relWalRef_.add(value); + onChanged(); + return this; + } + /** + * repeated string rel_wal_ref = 12; + */ + public Builder addAllRelWalRef( + java.lang.Iterable values) { + ensureRelWalRefIsMutable(); + super.addAll(values, relWalRef_); + onChanged(); + return this; + } + /** + * repeated string rel_wal_ref = 12; + */ + public Builder clearRelWalRef() { + relWalRef_ = com.google.protobuf.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000800); + onChanged(); + return this; + } + /** + * repeated string rel_wal_ref = 12; + */ + public Builder addRelWalRefBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + ensureRelWalRefIsMutable(); + relWalRef_.add(value); + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:hbase.pb.BackupManifest) } @@ -7872,6 +8063,36 @@ public final class BackupProtos { * optional uint32 progress = 12; */ int getProgress(); + + // optional bytes job_id = 13; + /** + * optional bytes job_id = 13; + */ + boolean hasJobId(); + /** + * optional bytes job_id = 13; + */ + com.google.protobuf.ByteString getJobId(); + + // required uint32 workers_number = 14; + /** + * required uint32 workers_number = 14; + */ + boolean hasWorkersNumber(); + /** + * required uint32 workers_number = 14; + */ + int getWorkersNumber(); + + // required uint32 bandwidth = 15; + /** + * required uint32 bandwidth = 15; + */ + boolean hasBandwidth(); + /** + * required uint32 bandwidth = 15; + */ + int getBandwidth(); } /** * Protobuf type {@code hbase.pb.BackupContext} @@ -8005,6 +8226,21 @@ public final class BackupProtos { progress_ = input.readUInt32(); break; } + case 106: { + bitField0_ |= 0x00000800; + jobId_ = input.readBytes(); + break; + } + case 112: { + bitField0_ |= 0x00001000; + workersNumber_ = input.readUInt32(); + break; + } + case 120: { + bitField0_ |= 0x00002000; + bandwidth_ = input.readUInt32(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -8595,6 +8831,54 @@ public final class BackupProtos { return progress_; } + // optional bytes job_id = 13; + public static final int JOB_ID_FIELD_NUMBER = 13; + private com.google.protobuf.ByteString jobId_; + /** + * optional bytes job_id = 13; + */ + public boolean hasJobId() { + return ((bitField0_ & 0x00000800) == 0x00000800); + } + /** + * optional bytes job_id = 13; + */ + public com.google.protobuf.ByteString getJobId() { + return jobId_; + } + + // required uint32 workers_number = 14; + public static final int WORKERS_NUMBER_FIELD_NUMBER = 14; + private int workersNumber_; + /** + * required uint32 workers_number = 14; + */ + public boolean hasWorkersNumber() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + /** + * required uint32 workers_number = 14; + */ + public int getWorkersNumber() { + return workersNumber_; + } + + // required uint32 bandwidth = 15; + public static final int BANDWIDTH_FIELD_NUMBER = 15; + private int bandwidth_; + /** + * required uint32 bandwidth = 15; + */ + public boolean hasBandwidth() { + return ((bitField0_ & 0x00002000) == 0x00002000); + } + /** + * required uint32 bandwidth = 15; + */ + public int getBandwidth() { + return bandwidth_; + } + private void initFields() { backupId_ = ""; type_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL; @@ -8608,6 +8892,9 @@ public final class BackupProtos { totalBytesCopied_ = 0L; hlogTargetDir_ = ""; progress_ = 0; + jobId_ = com.google.protobuf.ByteString.EMPTY; + workersNumber_ = 0; + bandwidth_ = 0; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -8626,6 +8913,14 @@ public final class BackupProtos { memoizedIsInitialized = 0; return false; } + if (!hasWorkersNumber()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasBandwidth()) { + memoizedIsInitialized = 0; + return false; + } for (int i = 0; i < getTableBackupStatusCount(); i++) { if (!getTableBackupStatus(i).isInitialized()) { memoizedIsInitialized = 0; @@ -8675,6 +8970,15 @@ public final class BackupProtos { if (((bitField0_ & 0x00000400) == 0x00000400)) { output.writeUInt32(12, progress_); } + if (((bitField0_ & 0x00000800) == 0x00000800)) { + output.writeBytes(13, jobId_); + } + if (((bitField0_ & 0x00001000) == 0x00001000)) { + output.writeUInt32(14, workersNumber_); + } + if (((bitField0_ & 0x00002000) == 0x00002000)) { + output.writeUInt32(15, bandwidth_); + } getUnknownFields().writeTo(output); } @@ -8732,6 +9036,18 @@ public final class BackupProtos { size += com.google.protobuf.CodedOutputStream .computeUInt32Size(12, progress_); } + if (((bitField0_ & 0x00000800) == 0x00000800)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(13, jobId_); + } + if (((bitField0_ & 0x00001000) == 0x00001000)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt32Size(14, workersNumber_); + } + if (((bitField0_ & 0x00002000) == 0x00002000)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt32Size(15, bandwidth_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -8812,6 +9128,21 @@ public final class BackupProtos { result = result && (getProgress() == other.getProgress()); } + result = result && (hasJobId() == other.hasJobId()); + if (hasJobId()) { + result = result && getJobId() + .equals(other.getJobId()); + } + result = result && (hasWorkersNumber() == other.hasWorkersNumber()); + if (hasWorkersNumber()) { + result = result && (getWorkersNumber() + == other.getWorkersNumber()); + } + result = result && (hasBandwidth() == other.hasBandwidth()); + if (hasBandwidth()) { + result = result && (getBandwidth() + == other.getBandwidth()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -8873,6 +9204,18 @@ public final class BackupProtos { hash = (37 * hash) + PROGRESS_FIELD_NUMBER; hash = (53 * hash) + getProgress(); } + if (hasJobId()) { + hash = (37 * hash) + JOB_ID_FIELD_NUMBER; + hash = (53 * hash) + getJobId().hashCode(); + } + if (hasWorkersNumber()) { + hash = (37 * hash) + WORKERS_NUMBER_FIELD_NUMBER; + hash = (53 * hash) + getWorkersNumber(); + } + if (hasBandwidth()) { + hash = (37 * hash) + BANDWIDTH_FIELD_NUMBER; + hash = (53 * hash) + getBandwidth(); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -9011,6 +9354,12 @@ public final class BackupProtos { bitField0_ = (bitField0_ & ~0x00000400); progress_ = 0; bitField0_ = (bitField0_ & ~0x00000800); + jobId_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00001000); + workersNumber_ = 0; + bitField0_ = (bitField0_ & ~0x00002000); + bandwidth_ = 0; + bitField0_ = (bitField0_ & ~0x00004000); return this; } @@ -9092,6 +9441,18 @@ public final class BackupProtos { to_bitField0_ |= 0x00000400; } result.progress_ = progress_; + if (((from_bitField0_ & 0x00001000) == 0x00001000)) { + to_bitField0_ |= 0x00000800; + } + result.jobId_ = jobId_; + if (((from_bitField0_ & 0x00002000) == 0x00002000)) { + to_bitField0_ |= 0x00001000; + } + result.workersNumber_ = workersNumber_; + if (((from_bitField0_ & 0x00004000) == 0x00004000)) { + to_bitField0_ |= 0x00002000; + } + result.bandwidth_ = bandwidth_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -9175,6 +9536,15 @@ public final class BackupProtos { if (other.hasProgress()) { setProgress(other.getProgress()); } + if (other.hasJobId()) { + setJobId(other.getJobId()); + } + if (other.hasWorkersNumber()) { + setWorkersNumber(other.getWorkersNumber()); + } + if (other.hasBandwidth()) { + setBandwidth(other.getBandwidth()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -9192,6 +9562,14 @@ public final class BackupProtos { return false; } + if (!hasWorkersNumber()) { + + return false; + } + if (!hasBandwidth()) { + + return false; + } for (int i = 0; i < getTableBackupStatusCount(); i++) { if (!getTableBackupStatus(i).isInitialized()) { @@ -9996,6 +10374,108 @@ public final class BackupProtos { return this; } + // optional bytes job_id = 13; + private com.google.protobuf.ByteString jobId_ = com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes job_id = 13; + */ + public boolean hasJobId() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + /** + * optional bytes job_id = 13; + */ + public com.google.protobuf.ByteString getJobId() { + return jobId_; + } + /** + * optional bytes job_id = 13; + */ + public Builder setJobId(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00001000; + jobId_ = value; + onChanged(); + return this; + } + /** + * optional bytes job_id = 13; + */ + public Builder clearJobId() { + bitField0_ = (bitField0_ & ~0x00001000); + jobId_ = getDefaultInstance().getJobId(); + onChanged(); + return this; + } + + // required uint32 workers_number = 14; + private int workersNumber_ ; + /** + * required uint32 workers_number = 14; + */ + public boolean hasWorkersNumber() { + return ((bitField0_ & 0x00002000) == 0x00002000); + } + /** + * required uint32 workers_number = 14; + */ + public int getWorkersNumber() { + return workersNumber_; + } + /** + * required uint32 workers_number = 14; + */ + public Builder setWorkersNumber(int value) { + bitField0_ |= 0x00002000; + workersNumber_ = value; + onChanged(); + return this; + } + /** + * required uint32 workers_number = 14; + */ + public Builder clearWorkersNumber() { + bitField0_ = (bitField0_ & ~0x00002000); + workersNumber_ = 0; + onChanged(); + return this; + } + + // required uint32 bandwidth = 15; + private int bandwidth_ ; + /** + * required uint32 bandwidth = 15; + */ + public boolean hasBandwidth() { + return ((bitField0_ & 0x00004000) == 0x00004000); + } + /** + * required uint32 bandwidth = 15; + */ + public int getBandwidth() { + return bandwidth_; + } + /** + * required uint32 bandwidth = 15; + */ + public Builder setBandwidth(int value) { + bitField0_ |= 0x00004000; + bandwidth_ = value; + onChanged(); + return this; + } + /** + * required uint32 bandwidth = 15; + */ + public Builder clearBandwidth() { + bitField0_ = (bitField0_ & ~0x00004000); + bandwidth_ = 0; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:hbase.pb.BackupContext) } @@ -11009,7 +11489,7 @@ public final class BackupProtos { "\n\006server\030\001 \002(\t\022\021\n\ttimestamp\030\002 \002(\004\"o\n\024Tab", "leServerTimestamp\022\"\n\005table\030\001 \002(\0132\023.hbase" + ".pb.TableName\0223\n\020server_timestamp\030\002 \003(\0132" + - "\031.hbase.pb.ServerTimestamp\"\313\002\n\016BackupMan" + + "\031.hbase.pb.ServerTimestamp\"\340\002\n\016BackupMan" + "ifest\022\017\n\007version\030\001 \002(\t\022\021\n\tbackup_id\030\002 \002(" + "\t\022\"\n\004type\030\003 \002(\0162\024.hbase.pb.BackupType\022\'\n" + "\ntable_list\030\004 \003(\0132\023.hbase.pb.TableName\022\020" + @@ -11018,25 +11498,27 @@ public final class BackupProtos { "\007tst_map\030\t \003(\0132\036.hbase.pb.TableServerTim" + "estamp\0225\n\026dependent_backup_image\030\n \003(\0132\025", ".hbase.pb.BackupImage\022\021\n\tcompacted\030\013 \002(\010" + - "\"]\n\021TableBackupStatus\022\"\n\005table\030\001 \002(\0132\023.h" + - "base.pb.TableName\022\022\n\ntarget_dir\030\002 \002(\t\022\020\n" + - "\010snapshot\030\003 \001(\t\"\323\004\n\rBackupContext\022\021\n\tbac" + - "kup_id\030\001 \002(\t\022\"\n\004type\030\002 \002(\0162\024.hbase.pb.Ba" + - "ckupType\022\027\n\017target_root_dir\030\003 \002(\t\0222\n\005sta" + - "te\030\004 \001(\0162#.hbase.pb.BackupContext.Backup" + - "State\0222\n\005phase\030\005 \001(\0162#.hbase.pb.BackupCo" + - "ntext.BackupPhase\022\026\n\016failed_message\030\006 \001(" + - "\t\0228\n\023table_backup_status\030\007 \003(\0132\033.hbase.p", - "b.TableBackupStatus\022\020\n\010start_ts\030\010 \001(\004\022\016\n" + - "\006end_ts\030\t \001(\004\022\032\n\022total_bytes_copied\030\n \001(" + - "\003\022\027\n\017hlog_target_dir\030\013 \001(\t\022\020\n\010progress\030\014" + - " \001(\r\"P\n\013BackupState\022\013\n\007WAITING\020\000\022\013\n\007RUNN" + + "\022\023\n\013rel_wal_ref\030\014 \003(\t\"]\n\021TableBackupStat" + + "us\022\"\n\005table\030\001 \002(\0132\023.hbase.pb.TableName\022\022" + + "\n\ntarget_dir\030\002 \002(\t\022\020\n\010snapshot\030\003 \001(\t\"\216\005\n" + + "\rBackupContext\022\021\n\tbackup_id\030\001 \002(\t\022\"\n\004typ" + + "e\030\002 \002(\0162\024.hbase.pb.BackupType\022\027\n\017target_" + + "root_dir\030\003 \002(\t\0222\n\005state\030\004 \001(\0162#.hbase.pb" + + ".BackupContext.BackupState\0222\n\005phase\030\005 \001(" + + "\0162#.hbase.pb.BackupContext.BackupPhase\022\026" + + "\n\016failed_message\030\006 \001(\t\0228\n\023table_backup_s", + "tatus\030\007 \003(\0132\033.hbase.pb.TableBackupStatus" + + "\022\020\n\010start_ts\030\010 \001(\004\022\016\n\006end_ts\030\t \001(\004\022\032\n\022to" + + "tal_bytes_copied\030\n \001(\003\022\027\n\017hlog_target_di" + + "r\030\013 \001(\t\022\020\n\010progress\030\014 \001(\r\022\016\n\006job_id\030\r \001(" + + "\014\022\026\n\016workers_number\030\016 \002(\r\022\021\n\tbandwidth\030\017" + + " \002(\r\"P\n\013BackupState\022\013\n\007WAITING\020\000\022\013\n\007RUNN" + "ING\020\001\022\014\n\010COMPLETE\020\002\022\n\n\006FAILED\020\003\022\r\n\tCANCE" + "LLED\020\004\"}\n\013BackupPhase\022\013\n\007REQUEST\020\000\022\014\n\010SN" + "APSHOT\020\001\022\027\n\023PREPARE_INCREMENTAL\020\002\022\020\n\014SNA" + - "PSHOTCOPY\020\003\022\024\n\020INCREMENTAL_COPY\020\004\022\022\n\016STO" + + "PSHOTCOPY\020\003\022\024\n\020INCREMENTAL_COPY\020\004\022\022\n\016STO", "RE_MANIFEST\020\005\"n\n\021BackupProcContext\022$\n\003ct" + - "x\030\001 \002(\0132\027.hbase.pb.BackupContext\0223\n\020serv", + "x\030\001 \002(\0132\027.hbase.pb.BackupContext\0223\n\020serv" + "er_timestamp\030\002 \003(\0132\031.hbase.pb.ServerTime" + "stamp*k\n\024FullTableBackupState\022\026\n\022PRE_SNA" + "PSHOT_TABLE\020\001\022\023\n\017SNAPSHOT_TABLES\020\002\022\021\n\rSN" + @@ -11044,9 +11526,9 @@ public final class BackupProtos { "crementalTableBackupState\022\027\n\023PREPARE_INC" + "REMENTAL\020\001\022\024\n\020INCREMENTAL_COPY\020\002\022\030\n\024INCR" + "_BACKUP_COMPLETE\020\003*(\n\022SnapshotTableState" + - "\022\022\n\016SNAPSHOT_TABLE\020\001*\'\n\nBackupType\022\010\n\004FU" + + "\022\022\n\016SNAPSHOT_TABLE\020\001*\'\n\nBackupType\022\010\n\004FU", "LL\020\000\022\017\n\013INCREMENTAL\020\001BB\n*org.apache.hado" + - "op.hbase.protobuf.generatedB\014BackupProto", + "op.hbase.protobuf.generatedB\014BackupProto" + "sH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = @@ -11083,7 +11565,7 @@ public final class BackupProtos { internal_static_hbase_pb_BackupManifest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_BackupManifest_descriptor, - new java.lang.String[] { "Version", "BackupId", "Type", "TableList", "StartTs", "CompleteTs", "TotalBytes", "LogBytes", "TstMap", "DependentBackupImage", "Compacted", }); + new java.lang.String[] { "Version", "BackupId", "Type", "TableList", "StartTs", "CompleteTs", "TotalBytes", "LogBytes", "TstMap", "DependentBackupImage", "Compacted", "RelWalRef", }); internal_static_hbase_pb_TableBackupStatus_descriptor = getDescriptor().getMessageTypes().get(5); internal_static_hbase_pb_TableBackupStatus_fieldAccessorTable = new @@ -11095,7 +11577,7 @@ public final class BackupProtos { internal_static_hbase_pb_BackupContext_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_BackupContext_descriptor, - new java.lang.String[] { "BackupId", "Type", "TargetRootDir", "State", "Phase", "FailedMessage", "TableBackupStatus", "StartTs", "EndTs", "TotalBytesCopied", "HlogTargetDir", "Progress", }); + new java.lang.String[] { "BackupId", "Type", "TargetRootDir", "State", "Phase", "FailedMessage", "TableBackupStatus", "StartTs", "EndTs", "TotalBytesCopied", "HlogTargetDir", "Progress", "JobId", "WorkersNumber", "Bandwidth", }); internal_static_hbase_pb_BackupProcContext_descriptor = getDescriptor().getMessageTypes().get(7); internal_static_hbase_pb_BackupProcContext_fieldAccessorTable = new diff --git hbase-protocol/src/main/protobuf/Backup.proto hbase-protocol/src/main/protobuf/Backup.proto index 9b7cf1f..77ab5f4 100644 --- hbase-protocol/src/main/protobuf/Backup.proto +++ hbase-protocol/src/main/protobuf/Backup.proto @@ -85,6 +85,7 @@ message BackupManifest { repeated TableServerTimestamp tst_map = 9; repeated BackupImage dependent_backup_image = 10; required bool compacted = 11; + repeated string rel_wal_ref = 12; } message TableBackupStatus { @@ -106,6 +107,9 @@ message BackupContext { optional int64 total_bytes_copied = 10; optional string hlog_target_dir = 11; optional uint32 progress = 12; + optional bytes job_id = 13; + required uint32 workers_number = 14; + required uint32 bandwidth = 15; enum BackupState { WAITING = 0; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java new file mode 100644 index 0000000..a1b7ef7 --- /dev/null +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java @@ -0,0 +1,125 @@ +/** + * 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.backup; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; + +public interface BackupClient extends Configurable{ + + public void setConf(Configuration conf); + +// /** +// * Sends backup request to server, and monitor the progress if necessary +// * @param backupType : full or incremental +// * @param targetRootDir : the root path specified by user +// * @param tableList : the table list specified by user +// * @return backupId backup id +// * @throws IOException exception +// */ +// public String create(BackupType backupType, List tableList, +// String targetRootDir) throws IOException; +// +// /** +// * Sends backup request to server and monitor progress if necessary +// * @param backupType : full or incremental +// * @param tableList : table list +// * @param targetRootDir: target root directory +// * @param snapshot : using existing snapshot +// * @param workers : number of workers for parallel data copy +// * @param bandwidth : maximum bandwidth (in MB/sec) of copy job +// * @return backup id +// * @throws IOException +// */ +// public String create(BackupType backupType, List tableList, String targetRootDir, +// int workers, int bandwidth) throws IOException; +// + /** + * Describe backup image command + * @param backupId - backup id + * @throws IOException + */ + public void describeBackupImage(String backupId) throws IOException; + + /** + * Show backup progress command + * @param backupId - backup id + * @throws IOException + */ + public void showProgress(String backupId) throws IOException; + + /** + * Delete backup image command + * @param backupIds - backup id + * @throws IOException + */ + public void deleteBackups(String[] backupIds) throws IOException; + +// /** +// * Cancel current active backup command +// * @param backupId - backup id +// * @throws IOException +// */ +// public void cancelBackup(String backupId) throws IOException; + + /** + * Show backup history command + * @param n - last n backup sessions + * @throws IOException + */ + public void showHistory(int n) throws IOException; + + /** + * Backup set list command + * @throws IOException + */ + public void backupSetList() throws IOException; + + /** + * Backup set describe command + * @param name set name + * @throws IOException + */ + public void backupSetDescribe(String name) throws IOException; + + /** + * Delete backup set command + * @param name - backup set name + * @throws IOException + */ + public void backupSetDelete(String name) throws IOException; + + /** + * Add tables to backup set command + * @param name set name + * @param tables - list of tables + * @throws IOException + */ + public void backupSetAdd(String name, String[] tablesOrNamespaces) throws IOException; + + /** + * Remove tables from backup set + * @param name - backup set name + * @param tables - list of tables + * @throws IOException + */ + public void backupSetRemove(String name, String[] tablesOrNamepsaces) throws IOException; + + } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java index 015c80b..cf2e821 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java @@ -20,9 +20,6 @@ package org.apache.hadoop.hbase.backup; import java.io.IOException; import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.ParseException; -import org.apache.commons.cli.PosixParser; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -39,13 +36,28 @@ import org.apache.log4j.Logger; public class BackupDriver extends AbstractHBaseTool { private static final Log LOG = LogFactory.getLog(BackupDriver.class); - private Options opt; private CommandLine cmd; - + private static Configuration conf; + + public static void setStaticConf(Configuration conf) + { + BackupDriver.conf = conf; + } + + public BackupDriver() throws IOException + { + init(); + } + protected void init() throws IOException { // define supported options - opt = new Options(); - opt.addOption("debug", false, "Enable debug loggings"); + addOptNoArg("debug", "Enable debug loggings"); + addOptNoArg("all", "All tables"); + addOptWithArg("t", "Table name"); + addOptWithArg("b", "Bandwidth (MB/s)"); + addOptWithArg("w", "Number of workers"); + addOptWithArg("n", "History length"); + addOptWithArg("set", "Backup set name"); // disable irrelevant loggers to avoid it mess up command output LogUtils.disableUselessLoggers(LOG); @@ -64,13 +76,6 @@ public class BackupDriver extends AbstractHBaseTool { System.arraycopy(args, 1, remainArgs, 0, args.length - 1); } } - CommandLine cmdline = null; - try { - cmdline = new PosixParser().parse(opt, remainArgs); - } catch (ParseException e) { - LOG.error("Could not parse command", e); - return -1; - } BackupCommand type = BackupCommand.HELP; if (BackupCommand.CREATE.name().equalsIgnoreCase(cmd)) { @@ -84,14 +89,18 @@ public class BackupDriver extends AbstractHBaseTool { // enable debug logging Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup"); - if (cmdline.hasOption("debug")) { + if (this.cmd.hasOption("debug")) { backupClientLogger.setLevel(Level.DEBUG); } else { backupClientLogger.setLevel(Level.INFO); } // TODO: get rid of Command altogether? - BackupCommands.createCommand(getConf(), type, cmdline).execute(); + BackupCommands.Command command = BackupCommands.createCommand(getConf(), type, this.cmd); + if( type == BackupCommand.CREATE && conf != null) { + ((BackupCommands.CreateCommand) command).setConf(conf); + } + command.execute(); return 0; } @@ -106,14 +115,18 @@ public class BackupDriver extends AbstractHBaseTool { @Override protected int doWork() throws Exception { - init(); return parseAndRun(cmd.getArgs()); } public static void main(String[] args) throws Exception { - Configuration conf = HBaseConfiguration.create(); + Configuration conf = + BackupDriver.conf == null? HBaseConfiguration.create():BackupDriver.conf; int ret = ToolRunner.run(conf, new BackupDriver(), args); - System.exit(ret); + if(BackupDriver.conf == null) { + System.exit(ret); + } else if(ret != 0){ + throw new IOException("Backup command failed, please check log file for error."); + } } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java index e0c6483..30882bd 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.backup; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.backup.impl.BackupClientImpl; import org.apache.hadoop.hbase.backup.impl.BackupCopyService; import org.apache.hadoop.hbase.backup.impl.IncrementalRestoreService; import org.apache.hadoop.hbase.backup.impl.RestoreClientImpl; @@ -33,6 +34,7 @@ public final class BackupRestoreFactory { public final static String HBASE_INCR_RESTORE_IMPL_CLASS = "hbase.incremental.restore.class"; public final static String HBASE_BACKUP_COPY_IMPL_CLASS = "hbase.backup.copy.class"; + public final static String HBASE_BACKUP_CLIENT_IMPL_CLASS = "hbase.backup.client.class"; public final static String HBASE_RESTORE_CLIENT_IMPL_CLASS = "hbase.restore.client.class"; private BackupRestoreFactory(){ @@ -48,7 +50,9 @@ public final class BackupRestoreFactory { Class cls = conf.getClass(HBASE_INCR_RESTORE_IMPL_CLASS, MapReduceRestoreService.class, IncrementalRestoreService.class); - return ReflectionUtils.newInstance(cls, conf); + IncrementalRestoreService service = ReflectionUtils.newInstance(cls, conf); + service.setConf(conf); + return service; } /** @@ -60,7 +64,22 @@ public final class BackupRestoreFactory { Class cls = conf.getClass(HBASE_BACKUP_COPY_IMPL_CLASS, MapReduceBackupCopyService.class, BackupCopyService.class); - return ReflectionUtils.newInstance(cls, conf); + BackupCopyService service = ReflectionUtils.newInstance(cls, conf);; + service.setConf(conf); + return service; + } + /** + * Gets backup client implementation + * @param conf - configuration + * @return backup client + */ + public static BackupClient getBackupClient(Configuration conf) { + Class cls = + conf.getClass(HBASE_BACKUP_CLIENT_IMPL_CLASS, BackupClientImpl.class, + BackupClient.class); + BackupClient client = ReflectionUtils.newInstance(cls, conf); + client.setConf(conf); + return client; } /** diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java index 4423bcc..a928fd6 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java @@ -87,7 +87,7 @@ public class HBackupFileSystem { * @param backupId backup Id * @return backupPath for the particular table */ - public static Path getTableBackupPath(Path backupRootPath, TableName tableName, String backupId) { + public static Path getTableBackupPath(TableName tableName, Path backupRootPath, String backupId) { return new Path(backupRootPath, tableName.getNamespaceAsString() + Path.SEPARATOR + tableName.getQualifierAsString() + Path.SEPARATOR + backupId); } @@ -110,7 +110,7 @@ public class HBackupFileSystem { private static Path getManifestPath(TableName tableName, Configuration conf, Path backupRootPath, String backupId) throws IOException { - Path manifestPath = new Path(getTableBackupPath(backupRootPath, tableName, backupId), + Path manifestPath = new Path(getTableBackupPath(tableName, backupRootPath, backupId), BackupManifest.MANIFEST_FILE_NAME); FileSystem fs = backupRootPath.getFileSystem(conf); if (!fs.exists(manifestPath)) { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java index 541882a..2dd165a 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java @@ -26,7 +26,6 @@ import org.apache.commons.cli.PosixParser; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.impl.BackupUtil; @@ -38,8 +37,7 @@ import org.apache.log4j.Logger; public class RestoreDriver extends AbstractHBaseTool { - private static final Log LOG = LogFactory.getLog(BackupDriver.class); - private Options opt; + private static final Log LOG = LogFactory.getLog(RestoreDriver.class); private CommandLine cmd; private static final String OPTION_OVERWRITE = "overwrite"; @@ -70,25 +68,17 @@ public class RestoreDriver extends AbstractHBaseTool { protected void init() throws IOException { // define supported options - opt = new Options(); - opt.addOption(OPTION_OVERWRITE, false, + addOptNoArg(OPTION_OVERWRITE, "Overwrite the data if any of the restore target tables exists"); - opt.addOption(OPTION_CHECK, false, "Check restore sequence and dependencies"); - opt.addOption(OPTION_AUTOMATIC, false, "Restore all dependencies"); - opt.addOption("debug", false, "Enable debug logging"); + addOptNoArg(OPTION_CHECK, "Check restore sequence and dependencies"); + addOptNoArg(OPTION_AUTOMATIC, "Restore all dependencies"); + addOptNoArg("debug", "Enable debug logging"); // disable irrelevant loggers to avoid it mess up command output LogUtils.disableUselessLoggers(LOG); } private int parseAndRun(String[] args) { - CommandLine cmd = null; - try { - cmd = new PosixParser().parse(opt, args); - } catch (ParseException e) { - LOG.error("Could not parse command", e); - return -1; - } // enable debug logging Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup"); @@ -119,17 +109,17 @@ public class RestoreDriver extends AbstractHBaseTool { // parse main restore command options String[] remainArgs = cmd.getArgs(); - if (remainArgs.length < 3) { + if (remainArgs.length < 4) { System.out.println("ERROR: missing arguments"); System.out.println(USAGE); return -1; } - String backupRootDir = remainArgs[0]; - String backupId = remainArgs[1]; - String tables = remainArgs[2]; + String backupRootDir = remainArgs[1]; + String backupId = remainArgs[2]; + String tables = remainArgs[3]; - String tableMapping = (remainArgs.length > 3) ? remainArgs[3] : null; + String tableMapping = (remainArgs.length >= 4) ? remainArgs[4] : null; TableName[] sTableArray = BackupUtil.parseTableNames(tables); TableName[] tTableArray = BackupUtil.parseTableNames(tableMapping); @@ -145,7 +135,7 @@ public class RestoreDriver extends AbstractHBaseTool { client.restore(backupRootDir, backupId, check, autoRestore, sTableArray, tTableArray, isOverwrite); } catch (IOException e) { - System.err.println("ERROR: " + e.getMessage()); + LOG.error(e); return -1; } return 0; @@ -168,7 +158,7 @@ public class RestoreDriver extends AbstractHBaseTool { public static void main(String[] args) throws Exception { Configuration conf = HBaseConfiguration.create(); - int ret = ToolRunner.run(conf, new BackupDriver(), args); + int ret = ToolRunner.run(conf, new RestoreDriver(), args); System.exit(ret); } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java new file mode 100644 index 0000000..637ef83 --- /dev/null +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java @@ -0,0 +1,231 @@ +/** + * 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.backup.impl; + +import java.io.IOException; +import java.util.ArrayList; +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.DoNotRetryIOException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupClient; +import org.apache.hadoop.hbase.backup.BackupRestoreFactory; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.impl.BackupHandler.BackupState; +import org.apache.hadoop.hbase.backup.impl.BackupUtil.BackupCompleteData; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; + +/** + * Backup HBase tables locally or on a remote cluster Serve as client entry point for the following + * features: - Full Backup provide local and remote back/restore for a list of tables - Incremental + * backup to build on top of full backup as daily/weekly backup - Convert incremental backup WAL + * files into hfiles - Merge several backup images into one(like merge weekly into monthly) - Add + * and remove table to and from Backup image - Cancel a backup process - Full backup based on + * existing snapshot - Describe information of a backup image + */ + +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class BackupClientImpl implements BackupClient{ + private static final Log LOG = LogFactory.getLog(BackupClientImpl.class); + private Configuration conf; + + public BackupClientImpl() { + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + + @Override + public void describeBackupImage(String backupId) throws IOException { + BackupContext backupContext = null; + try (final BackupSystemTable table = new BackupSystemTable(conf)) { + backupContext = table.readBackupStatus(backupId); + if (backupContext != null) { + System.out.println(backupContext.getShortDescription()); + } else { + System.out.println("No information found for backupID=" + backupId); + } + } + } + + @Override + public void showProgress(String backupId) throws IOException { + BackupContext backupContext = null; + try (final BackupSystemTable table = new BackupSystemTable(conf)) { + + if (backupId == null) { + ArrayList recentSessions = + table.getBackupContexts(BackupState.RUNNING); + if (recentSessions.isEmpty()) { + System.out.println("No ongonig sessions found."); + return; + } + // else show status for all ongoing sessions + // must be one maximum + for (BackupContext context : recentSessions) { + System.out.println(context.getStatusAndProgressAsString()); + } + } else { + + backupContext = table.readBackupStatus(backupId); + if (backupContext != null) { + System.out.println(backupContext.getStatusAndProgressAsString()); + } else { + System.out.println("No information found for backupID=" + backupId); + } + } + } + } + + @Override + public void deleteBackups(String[] backupIds) throws IOException { + BackupContext backupContext = null; + String backupId = null; + try (final BackupSystemTable table = new BackupSystemTable(conf)) { + for (int i = 0; i < backupIds.length; i++) { + backupId = backupIds[i]; + backupContext = table.readBackupStatus(backupId); + if (backupContext != null) { + BackupUtil.cleanupBackupData(backupContext, conf); + table.deleteBackupStatus(backupContext.getBackupId()); + System.out.println("Delete backup for backupID=" + backupId + " completed."); + } else { + System.out.println("Delete backup failed: no information found for backupID=" + backupId); + } + } + } + } + +//TODO: Cancel backup? + +// @Override +// public void cancelBackup(String backupId) throws IOException { +// // Kill distributed job if active +// // Backup MUST not be in COMPLETE state +// try (final BackupSystemTable table = new BackupSystemTable(conf)) { +// BackupContext backupContext = table.readBackupStatus(backupId); +// String errMessage = null; +// if (backupContext != null && backupContext.getState() != BackupState.COMPLETE) { +// BackupUtil.cleanupBackupData(backupContext, conf); +// table.deleteBackupStatus(backupContext.getBackupId()); +// byte[] jobId = backupContext.getJobId(); +// if(jobId != null) { +// BackupCopyService service = BackupRestoreFactory.getBackupCopyService(conf); +// service.cancelCopyJob(jobId); +// } else{ +// errMessage = "Distributed Job ID is null for backup "+backupId + +// " in "+ backupContext.getState() + " state."; +// } +// } else if( backupContext == null){ +// errMessage = "No information found for backupID=" + backupId; +// } else { +// errMessage = "Can not cancel "+ backupId + " in " + backupContext.getState()+" state"; +// } +// +// if( errMessage != null) { +// throw new IOException(errMessage); +// } +// } +// // then clean backup image +// deleteBackups(new String[] { backupId }); +// } + + @Override + public void showHistory(int n) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conf)) { + List history = table.getBackupHistory(); + int max = Math.min(n, history.size()); + for (int i = 0; i < max; i++) { + printBackupCompleteData(history.get(i)); + } + } + } + + private void printBackupCompleteData(BackupCompleteData backupCompleteData) { + System.out.println(backupCompleteData.toString()); + } + + @Override + public void backupSetList() throws IOException{ + try (final BackupSystemTable table = new BackupSystemTable(conf)) { + List list = table.backupSetList(); + for (String s : list) { + System.out.println(s); + } + System.out.println("Found " + list.size() + " records"); + } + } + + @Override + public void backupSetDescribe(String name) throws IOException{ + try (final BackupSystemTable table = new BackupSystemTable(conf)) { + String[] list = table.backupSetDescribe(name); + for (String s : list) { + System.out.println(s); + } + System.out.println("Found " + list.length + " records"); + } + } + + @Override + public void backupSetDelete(String name) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conf)) { + table.backupSetDelete(name); + System.out.println("Deleted " + name); + } + } + + @Override + public void backupSetAdd(String name, String[] tablesOrNamespaces) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conf)) { + table.backupSetAdd(name, tablesOrNamespaces); + System.out.println("Added tables to '" + name + "'"); + } + } + + @Override + public void backupSetRemove(String name, String[] tablesOrNamepsaces) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conf)) { + table.backupSetRemove(name, tablesOrNamepsaces); + System.out.println("Removed tables from '" + name + "'"); + } + } + + @Override + public Configuration getConf() { + return conf; + } + +} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java index 9b54f6b..5d741aa 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java @@ -19,9 +19,14 @@ package org.apache.hadoop.hbase.backup.impl; import java.io.IOException; + import org.apache.commons.cli.CommandLine; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupClient; +import org.apache.hadoop.hbase.backup.BackupRestoreFactory; import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants.BackupCommand; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -40,18 +45,56 @@ import com.google.common.collect.Lists; public final class BackupCommands { private static final String USAGE = "Usage: hbase backup COMMAND\n" - + "where COMMAND is one of:\n" + " create create a new backup image\n" + + "where COMMAND is one of:\n" + + " create create a new backup image\n" + + " cancel cancel an ongoing backup\n" + + " delete delete an existing backup image\n" + + " describe show the detailed information of a backup image\n" + + " history show history of all successful backups\n" + + " progress show the progress of the latest backup request\n" + + " convert convert incremental backup WAL files into HFiles\n" + + " merge merge backup images\n" + + " set backup set management\n" + "Enter \'help COMMAND\' to see help message for each command\n"; private static final String CREATE_CMD_USAGE = - "Usage: hbase backup create [tables] [-convert] " - + "\n" + " type \"full\" to create a full backup image;\n" + "Usage: hbase backup create [tables] [-s name] [-convert] " + + "[-silent] [-w workers][-b bandwith]\n" + " type \"full\" to create a full backup image;\n" + " \"incremental\" to create an incremental backup image\n" - + " backup_root_path The full root path to store the backup image,\n" - + " the prefix can be hdfs, webhdfs, gpfs, etc\n" + " Options:\n" - + " tables If no tables (\"\") are specified, all tables are backed up. " + + " backup_root_path The full root path to store the backup image,\n" + + " the prefix can be gpfs, hdfs or webhdfs\n" + " Options:\n" + + " tables If no tables (\"\") are specified, all tables are backed up. " + "Otherwise it is a\n" + " comma separated list of tables.\n" - + " -convert For an incremental backup, convert WAL files to HFiles\n"; + + " -s name Use the specified snapshot for full backup\n" + + " -convert For an incremental backup, convert WAL files to HFiles\n" + + " -w number of parallel workers.\n" + + " -b bandwith per one worker (in MB sec)" ; + + private static final String PROGRESS_CMD_USAGE = "Usage: hbase backup progress \n" + + " backupId backup image id;\n"; + + private static final String DESCRIBE_CMD_USAGE = "Usage: hbase backup decsribe \n" + + " backupId backup image id\n"; + + private static final String HISTORY_CMD_USAGE = "Usage: hbase backup history [-n N]\n" + + " -n N show up to N last backup sessions, default - 10;\n"; + + private static final String DELETE_CMD_USAGE = "Usage: hbase backup delete \n" + + " backupId backup image id;\n"; + + private static final String CANCEL_CMD_USAGE = "Usage: hbase backup progress \n" + + " backupId backup image id;\n"; + + private static final String SET_CMD_USAGE = "Usage: hbase set COMMAND [name] [tables]\n" + + " name Backup set name\n" + + " tables If no tables (\"\") are specified, all tables will belong to the set. " + + "Otherwise it is a\n" + " comma separated list of tables.\n" + + "where COMMAND is one of:\n" + + " add add tables to a set, crete set if needed\n" + + " remove remove tables from set\n" + + " list list all sets\n" + + " describe describes set\n" + + " delete delete backup set\n"; public static abstract class Command extends Configured { Command(Configuration conf) { @@ -67,25 +110,44 @@ public final class BackupCommands { public static Command createCommand(Configuration conf, BackupCommand type, CommandLine cmdline) { Command cmd = null; switch (type) { - case CREATE: - cmd = new CreateCommand(conf, cmdline); - break; - case HELP: - default: - cmd = new HelpCommand(conf, cmdline); - break; + case CREATE: + cmd = new CreateCommand(conf, cmdline); + break; + case DESCRIBE: + cmd = new DescribeCommand(conf, cmdline); + break; + case PROGRESS: + cmd = new ProgressCommand(conf, cmdline); + break; + case DELETE: + cmd = new DeleteCommand(conf, cmdline); + break; + case CANCEL: + cmd = new CancelCommand(conf, cmdline); + break; + case HISTORY: + cmd = new HistoryCommand(conf, cmdline); + break; + case SET: + cmd = new BackupSetCommand(conf, cmdline); + break; + case HELP: + default: + cmd = new HelpCommand(conf, cmdline); + break; } return cmd; } - private static class CreateCommand extends Command { + + public static class CreateCommand extends Command { CommandLine cmdline; CreateCommand(Configuration conf, CommandLine cmdline) { super(conf); this.cmdline = cmdline; } - + @Override public void execute() throws IOException { if (cmdline == null || cmdline.getArgs() == null) { @@ -94,32 +156,62 @@ public final class BackupCommands { System.exit(-1); } String[] args = cmdline.getArgs(); - if (args.length < 2 || args.length > 3) { + if (args.length < 3 || args.length > 4) { System.out.println("ERROR: wrong number of arguments"); System.out.println(CREATE_CMD_USAGE); System.exit(-1); } - if (!BackupType.FULL.toString().equalsIgnoreCase(args[0]) - && !BackupType.INCREMENTAL.toString().equalsIgnoreCase(args[0])) { + if (!BackupType.FULL.toString().equalsIgnoreCase(args[1]) + && !BackupType.INCREMENTAL.toString().equalsIgnoreCase(args[1])) { System.out.println("ERROR: invalid backup type"); System.out.println(CREATE_CMD_USAGE); System.exit(-1); } - String tables = (args.length == 3) ? args[2] : null; + String tables = null; + Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create(); + // Check backup set + if (cmdline.hasOption("set")) { + String setName = cmdline.getOptionValue("set"); + tables = getTablesForSet(setName, conf); + + if (tables == null) throw new IOException("Backup set '" + setName + + "' is either empty or does not exist"); + } else { + tables = (args.length == 4) ? args[3] : null; + } + int bandwidth = cmdline.hasOption('b') ? Integer.parseInt(cmdline.getOptionValue('b')) : -1; + int workers = cmdline.hasOption('w') ? Integer.parseInt(cmdline.getOptionValue('w')) : -1; try (Connection conn = ConnectionFactory.createConnection(getConf()); Admin admin = conn.getAdmin();) { - admin.backupTables(BackupType.valueOf(args[0].toUpperCase()), - Lists.newArrayList(BackupUtil.parseTableNames(tables)), args[1], -1, -1); + TableName[] tableNames = BackupUtil.parseTableNames(tables); + admin.backupTablesSync(BackupType.valueOf(args[1].toUpperCase()), + Lists.newArrayList(tableNames), args[2], workers, bandwidth); } catch (IOException e) { - System.err.println("ERROR: " + e.getMessage()); - System.exit(-1); + throw e; + } + } + + private String getTablesForSet(String name, Configuration conf) + throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conf)) { + String[] tables = table.backupSetDescribe(name); + if (tables == null) return null; + StringBuffer sb = new StringBuffer(); + for (int i = 0; i < tables.length; i++) { + sb.append(tables[i]); + if (i < tables.length - 1) { + sb.append(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND); + } + } + return sb.toString(); } } } + private static class HelpCommand extends Command { CommandLine cmdline; @@ -151,9 +243,302 @@ public final class BackupCommands { if (BackupCommand.CREATE.name().equalsIgnoreCase(type)) { System.out.println(CREATE_CMD_USAGE); - } // other commands will be supported in future jira + } else if (BackupCommand.DESCRIBE.name().equalsIgnoreCase(type)) { + System.out.println(DESCRIBE_CMD_USAGE); + } else if (BackupCommand.HISTORY.name().equalsIgnoreCase(type)) { + System.out.println(HISTORY_CMD_USAGE); + } else if (BackupCommand.PROGRESS.name().equalsIgnoreCase(type)) { + System.out.println(PROGRESS_CMD_USAGE); + } else if (BackupCommand.DELETE.name().equalsIgnoreCase(type)) { + System.out.println(DELETE_CMD_USAGE); + } + if (BackupCommand.CANCEL.name().equalsIgnoreCase(type)) { + System.out.println(CANCEL_CMD_USAGE); + } + if (BackupCommand.SET.name().equalsIgnoreCase(type)) { + System.out.println(SET_CMD_USAGE); + } else { + System.out.println("Unknown command : " + type); + System.out.println(USAGE); + } System.exit(0); } } + private static class DescribeCommand extends Command { + CommandLine cmdline; + + DescribeCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + if (cmdline == null || cmdline.getArgs() == null) { + System.out.println("ERROR: missing arguments"); + System.out.println(DESCRIBE_CMD_USAGE); + System.exit(-1); + } + String[] args = cmdline.getArgs(); + if (args.length != 1) { + System.out.println("ERROR: wrong number of arguments"); + System.out.println(DESCRIBE_CMD_USAGE); + System.exit(-1); + } + + String backupId = args[0]; + try { + Configuration conf = HBaseConfiguration.create(); + BackupClient client = BackupRestoreFactory.getBackupClient(conf); + client.describeBackupImage(backupId); + } catch (RuntimeException e) { + System.out.println("ERROR: " + e.getMessage()); + System.exit(-1); + } + } + } + + private static class ProgressCommand extends Command { + CommandLine cmdline; + + ProgressCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + if (cmdline == null || cmdline.getArgs() == null) { + System.out.println("No backup id was specified, " + + "will retrieve the most recent (ongoing) sessions"); + } + String[] args = cmdline.getArgs(); + if (args.length > 1) { + System.out.println("ERROR: wrong number of arguments: " + args.length); + System.out.println(PROGRESS_CMD_USAGE); + System.exit(-1); + } + + String backupId = args == null ? null : args[0]; + try { + Configuration conf = HBaseConfiguration.create(); + BackupClient client = BackupRestoreFactory.getBackupClient(conf); + client.showProgress(backupId); + } catch (RuntimeException e) { + System.out.println("ERROR: " + e.getMessage()); + System.exit(-1); + } + } + } + + private static class DeleteCommand extends Command { + + CommandLine cmdline; + DeleteCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + if (cmdline == null || cmdline.getArgs() == null) { + System.out.println("No backup id(s) was specified"); + System.out.println(PROGRESS_CMD_USAGE); + System.exit(-1); + } + String[] args = cmdline.getArgs(); + + try { + Configuration conf = HBaseConfiguration.create(); + BackupClient client = BackupRestoreFactory.getBackupClient(conf); + client.deleteBackups(args); + } catch (RuntimeException e) { + System.out.println("ERROR: " + e.getMessage()); + System.exit(-1); + } + } + } + +// TODO Cancel command + + private static class CancelCommand extends Command { + CommandLine cmdline; + + CancelCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + if (cmdline == null || cmdline.getArgs() == null) { + System.out.println("No backup id(s) was specified, will use the most recent one"); + } + String[] args = cmdline.getArgs(); + String backupId = args == null || args.length == 0 ? null : args[0]; + try { + Configuration conf = HBaseConfiguration.create(); + BackupClient client = BackupRestoreFactory.getBackupClient(conf); +//TODO +// client.cancelBackup(backupId); + } catch (RuntimeException e) { + System.out.println("ERROR: " + e.getMessage()); + System.exit(-1); + } + } + } + + private static class HistoryCommand extends Command { + CommandLine cmdline; + + HistoryCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + + int n = + cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length == 0 ? 10 + : parseHistoryLength(); + try { + Configuration conf = HBaseConfiguration.create(); + BackupClient client = BackupRestoreFactory.getBackupClient(conf); + client.showHistory(n); + } catch (RuntimeException e) { + System.out.println("ERROR: " + e.getMessage()); + System.exit(-1); + } + } + + private int parseHistoryLength() { + String value = cmdline.getOptionValue("n"); + if (value == null) throw new RuntimeException("command line format"); + return Integer.parseInt(value); + } + } + + private static class BackupSetCommand extends Command { + private final static String SET_ADD_CMD = "add"; + private final static String SET_REMOVE_CMD = "remove"; + private final static String SET_DELETE_CMD = "delete"; + private final static String SET_DESCRIBE_CMD = "describe"; + private final static String SET_LIST_CMD = "list"; + + CommandLine cmdline; + + BackupSetCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + + // Command-line must have at least one element + if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length == 0) { + throw new IOException("command line format"); + } + String[] args = cmdline.getArgs(); + String cmdStr = args[0]; + BackupCommand cmd = getCommand(cmdStr); + + try { + + switch (cmd) { + case SET_ADD: + processSetAdd(args); + break; + case SET_REMOVE: + processSetRemove(args); + break; + case SET_DELETE: + processSetDelete(args); + break; + case SET_DESCRIBE: + processSetDescribe(args); + break; + case SET_LIST: + processSetList(args); + break; + default: + break; + + } + } catch (RuntimeException e) { + System.out.println("ERROR: " + e.getMessage()); + System.exit(-1); + } + } + + private void processSetList(String[] args) throws IOException { + // List all backup set names + // does not expect any args + Configuration conf = HBaseConfiguration.create(); + BackupClient client = BackupRestoreFactory.getBackupClient(conf); + client.backupSetList(); + } + + private void processSetDescribe(String[] args) throws IOException { + if (args == null || args.length != 2) { + throw new RuntimeException("Wrong args"); + } + String setName = args[1]; + Configuration conf = HBaseConfiguration.create(); + BackupClient client = BackupRestoreFactory.getBackupClient(conf); + client.backupSetDescribe(setName); + } + + private void processSetDelete(String[] args) throws IOException { + if (args == null || args.length != 2) { + throw new RuntimeException("Wrong args"); + } + String setName = args[1]; + Configuration conf = HBaseConfiguration.create(); + BackupClient client = BackupRestoreFactory.getBackupClient(conf); + client.backupSetDelete(setName); + } + + private void processSetRemove(String[] args) throws IOException { + if (args == null || args.length != 3) { + throw new RuntimeException("Wrong args"); + } + String setName = args[1]; + String[] tables = args[2].split(","); + Configuration conf = HBaseConfiguration.create(); + BackupClient client = BackupRestoreFactory.getBackupClient(conf); + client.backupSetRemove(setName, tables); + } + + private void processSetAdd(String[] args) throws IOException { + if (args == null || args.length != 3) { + throw new RuntimeException("Wrong args"); + } + String setName = args[1]; + String[] tables = args[2].split(","); + Configuration conf = HBaseConfiguration.create(); + BackupClient client = BackupRestoreFactory.getBackupClient(conf); + client.backupSetAdd(setName, tables); + } + + private BackupCommand getCommand(String cmdStr) throws IOException { + if (cmdStr.equals(SET_ADD_CMD)) { + return BackupCommand.SET_ADD; + } else if (cmdStr.equals(SET_REMOVE_CMD)) { + return BackupCommand.SET_REMOVE; + } else if (cmdStr.equals(SET_DELETE_CMD)) { + return BackupCommand.SET_DELETE; + } else if (cmdStr.equals(SET_DESCRIBE_CMD)) { + return BackupCommand.SET_DESCRIBE; + } else if (cmdStr.equals(SET_LIST_CMD)) { + return BackupCommand.SET_LIST; + } else { + throw new IOException("Unknown command for 'set' :" + cmdStr); + } + } + + } + } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupContext.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupContext.java index d436931..ea8e58e 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupContext.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupContext.java @@ -21,12 +21,16 @@ package org.apache.hadoop.hbase.backup.impl; import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; +import java.util.Calendar; +import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; 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.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.HBackupFileSystem; @@ -38,49 +42,15 @@ import org.apache.hadoop.hbase.protobuf.generated.BackupProtos; import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupContext.Builder; import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.TableBackupStatus; +import com.google.protobuf.ByteString; + /** * An object to encapsulate the information for each backup request */ @InterfaceAudience.Private @InterfaceStability.Evolving public class BackupContext { - - public Map getBackupStatusMap() { - return backupStatusMap; - } - - public void setBackupStatusMap(Map backupStatusMap) { - this.backupStatusMap = backupStatusMap; - } - - public HashMap> getTableSetTimestampMap() { - return tableSetTimestampMap; - } - - public void setTableSetTimestampMap( - HashMap> tableSetTimestampMap) { - this.tableSetTimestampMap = tableSetTimestampMap; - } - - public String getHlogTargetDir() { - return hlogTargetDir; - } - - public void setType(BackupType type) { - this.type = type; - } - - public void setTargetRootDir(String targetRootDir) { - this.targetRootDir = targetRootDir; - } - - public void setTotalBytesCopied(long totalBytesCopied) { - this.totalBytesCopied = totalBytesCopied; - } - - public void setCancelled(boolean cancelled) { - this.state = BackupState.CANCELLED;; - } + private static final Log LOG = LogFactory.getLog(BackupContext.class); // backup id: a timestamp when we request the backup private String backupId; @@ -123,9 +93,22 @@ public class BackupContext { transient private HashMap> tableSetTimestampMap; // backup progress in %% (0-100) - private int progress; - + + // distributed job id + private byte[] jobId; + + // Number of paralle workers. -1 - system defined + private int workers = -1; + + // Bandwidth per worker in MB per sec. -1 - unlimited + private int bandwidth = -1; + + // For incremental backup: + // List of WAL files (backupid/wal_file_name) we keep references only. + + private List relWALRefs; + public BackupContext() { } @@ -135,7 +118,9 @@ public class BackupContext { this.backupId = backupId; this.type = type; this.targetRootDir = targetRootDir; - + if(LOG.isDebugEnabled()){ + LOG.debug("CreateBackupContext: " + tables.length+" "+tables[0] ); + } this.addTables(tables); if (type == BackupType.INCREMENTAL) { @@ -146,6 +131,62 @@ public class BackupContext { this.endTs = 0; } + public byte[] getJobId() { + return jobId; + } + + public void setJobId(byte[] jobId) { + this.jobId = jobId; + } + + public int getWorkers() { + return workers; + } + + public void setWorkers(int workers) { + this.workers = workers; + } + + public int getBandwidth() { + return bandwidth; + } + + public void setBandwidth(int bandwidth) { + this.bandwidth = bandwidth; + } + + public void setBackupStatusMap(Map backupStatusMap) { + this.backupStatusMap = backupStatusMap; + } + + public HashMap> getTableSetTimestampMap() { + return tableSetTimestampMap; + } + + public void setTableSetTimestampMap(HashMap> tableSetTimestampMap) { + this.tableSetTimestampMap = tableSetTimestampMap; + } + + public String getHlogTargetDir() { + return hlogTargetDir; + } + + public void setType(BackupType type) { + this.type = type; + } + + public void setTargetRootDir(String targetRootDir) { + this.targetRootDir = targetRootDir; + } + + public void setTotalBytesCopied(long totalBytesCopied) { + this.totalBytesCopied = totalBytesCopied; + } + + public void setCancelled(boolean cancelled) { + this.state = BackupState.CANCELLED;; + } + /** * Set progress string * @param msg progress message @@ -333,6 +374,11 @@ public class BackupContext { builder.setTargetRootDir(getTargetRootDir()); builder.setTotalBytesCopied(getTotalBytesCopied()); builder.setType(BackupProtos.BackupType.valueOf(getType().name())); + builder.setWorkersNumber(workers); + builder.setBandwidth(bandwidth); + if(jobId != null) { + builder.setJobId(ByteString.copyFrom(jobId)); + } return builder.build(); } @@ -378,6 +424,11 @@ public class BackupContext { context.setTargetRootDir(proto.getTargetRootDir()); context.setTotalBytesCopied(proto.getTotalBytesCopied()); context.setType(BackupType.valueOf(proto.getType().name())); + context.setWorkers(proto.getWorkersNumber()); + context.setBandwidth(proto.getBandwidth()); + if(proto.hasJobId()){ + context.setJobId(proto.getJobId().toByteArray()); + } return context; } @@ -389,4 +440,58 @@ public class BackupContext { return map; } + public String getShortDescription() { + StringBuffer sb = new StringBuffer(); + sb.append("ID : " + backupId).append("\n"); + sb.append("Tables : " + getTableListAsString()).append("\n"); + sb.append("State : " + getState()).append("\n"); + Date date = null; + Calendar cal = Calendar.getInstance(); + cal.setTimeInMillis(getStartTs()); + date = cal.getTime(); + sb.append("Start time : " + date).append("\n"); + if (state == BackupState.FAILED) { + sb.append("Failed message : " + getFailedMsg()).append("\n"); + } else if (state == BackupState.RUNNING) { + sb.append("Phase : " + getPhase()).append("\n"); + sb.append("Progress : " + getProgress()).append("\n"); + } else if (state == BackupState.COMPLETE) { + cal = Calendar.getInstance(); + cal.setTimeInMillis(getEndTs()); + date = cal.getTime(); + sb.append("Start time : " + date).append("\n"); + } + return sb.toString(); + } + + public String getStatusAndProgressAsString() { + StringBuffer sb = new StringBuffer(); + sb.append("id: ").append(getBackupId()).append(" state: ").append(getState()) + .append(" progress: ").append(getProgress()); + return sb.toString(); + } + + public String getTableListAsString() { + return concat(backupStatusMap.keySet(), ";"); + } + + private String concat(Set col, String separator) { + if (col.size() == 0) { + return ""; + } + StringBuilder sb = new StringBuilder(); + for (Object s : col) { + sb.append(s.toString() + separator); + } + sb.deleteCharAt(sb.lastIndexOf(";")); + return sb.toString(); + } + + public List getRelWALRefs() { + return relWALRefs; + } + + public void setRelWALRefs(List relWALRefs) { + this.relWALRefs = relWALRefs; + } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCopyService.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCopyService.java index 1e8da63..a905350 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCopyService.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCopyService.java @@ -32,6 +32,25 @@ public interface BackupCopyService extends Configurable { FULL, INCREMENTAL } + /** + * Copy backup data + * @param backupContext - context + * @param backupManager - manager + * @param conf - configuration + * @param copyType - copy type + * @param options - list of options + * TODO: result object + * @return result (0 - success) + * @throws IOException + */ public int copy(BackupContext backupContext, BackupManager backupManager, Configuration conf, BackupCopyService.Type copyType, String[] options) throws IOException; + + + /** + * TODO refactor + * @param jobHandler - copy job handler + * @throws IOException + */ + public void cancelCopyJob(byte[] jobHandler) throws IOException; } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java index 78e767a..f28c673 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java @@ -168,7 +168,7 @@ public class BackupHandler // implements Callable // copy out the table and region info files for each table BackupUtil.copyTableRegionInfo(backupContext, conf); this.incrementalCopy(backupContext); - // Save list of WAL files copied + // Save list of WAL files being copied backupManager.recordWALFiles(backupContext.getIncrBackupFileList()); } catch (Exception e) { // fail the overall backup and return @@ -235,7 +235,6 @@ public class BackupHandler // implements Callable */ private void snapshotForFullBackup(BackupContext backupContext) throws IOException { LOG.info("HBase snapshot full backup for " + backupContext.getBackupId()); - // avoid action if has been cancelled if (backupContext.isCancelled()) { return; @@ -244,6 +243,7 @@ public class BackupHandler // implements Callable try (Admin admin = conn.getAdmin()) { // we do HBase snapshot for tables in the table list one by one currently for (TableName table : backupContext.getTables()) { + // avoid action if it has been cancelled if (backupContext.isCancelled()) { return; @@ -431,11 +431,26 @@ public class BackupHandler // implements Callable // TODO this below // backupCopier.setSubTaskPercntgInWholeTask(1f / numOfSnapshots); int res = 0; - String[] args = new String[4]; + int argsLen = 4; + if (backupContext.getWorkers() > 0) argsLen += 2; + if (backupContext.getBandwidth() > 0) argsLen += 2; + String[] args = new String[argsLen]; args[0] = "-snapshot"; args[1] = backupContext.getSnapshotName(table); args[2] = "-copy-to"; args[3] = backupContext.getBackupStatus(table).getTargetDir(); + if (backupContext.getWorkers() > 0 && backupContext.getBandwidth() > 0) { + args[4] = "-mappers"; + args[5] = Integer.toString(backupContext.getWorkers()); + args[6] = "-bandwidth"; + args[7] = Integer.toString(backupContext.getBandwidth()); + } else if (backupContext.getWorkers() > 0) { + args[4] = "-mappers"; + args[5] = Integer.toString(backupContext.getWorkers()); + } else if (backupContext.getBandwidth() > 0) { + args[4] = "-bandwidth"; + args[5] = Integer.toString(backupContext.getBandwidth()); + } LOG.debug("Copy snapshot " + args[1] + " to " + args[3]); res = copyService.copy(backupContext, backupManager, conf, BackupCopyService.Type.FULL, args); @@ -601,7 +616,6 @@ public class BackupHandler // implements Callable private void addManifest(BackupContext backupContext) throws IOException, BackupException { // set the overall backup phase : store manifest backupContext.setPhase(BackupPhase.STORE_MANIFEST); - // avoid action if has been cancelled if (backupContext.isCancelled()) { return; @@ -612,6 +626,7 @@ public class BackupHandler // implements Callable // Since we have each table's backup in its own directory structure, // we'll store its manifest with the table directory. for (TableName table : backupContext.getTables()) { + manifest = new BackupManifest(backupContext, table); ArrayList ancestors = this.backupManager.getAncestors(backupContext, table); for (BackupImage image : ancestors) { @@ -624,7 +639,12 @@ public class BackupHandler // implements Callable new HashMap>(); tableTimestampMap.put(table, backupContext.getIncrTimestampMap().get(table)); manifest.setIncrTimestampMap(tableTimestampMap); + ArrayList ancestorss = this.backupManager.getAncestors(backupContext); + for (BackupImage image : ancestorss) { + manifest.addDependentImage(image); + } } + manifest.setRelativeWALReferences(backupContext.getRelWALRefs()); manifest.store(conf); } @@ -639,6 +659,7 @@ public class BackupHandler // implements Callable for (BackupImage image : ancestors) { manifest.addDependentImage(image); } + manifest.setRelativeWALReferences(backupContext.getRelWALRefs()); manifest.store(conf); } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java index 7efddfd..9a14ba1 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java @@ -86,12 +86,20 @@ public class BackupManager implements Closeable { HConstants.BACKUP_ENABLE_KEY + " setting."); } this.conf = conf; - this.conn = ConnectionFactory.createConnection(conf); // TODO: get Connection from elsewhere? - this.systemTable = new BackupSystemTable(conn); + this.systemTable = new BackupSystemTable(this.conf); + this.conn = BackupSystemTable.getConnection(); Runtime.getRuntime().addShutdownHook(new ExitHandler()); + } /** + * Return backup context + */ + protected BackupContext getBackupContext() + { + return backupContext; + } + /** * This method modifies the master's configuration in order to inject backup-related features * @param conf configuration */ @@ -178,13 +186,13 @@ public class BackupManager implements Closeable { LOG.error(e); } } - if (conn != null) { - try { - conn.close(); - } catch (IOException e) { - LOG.error(e); - } - } +// if (conn != null) { +// try { +// conn.close(); +// } catch (IOException e) { +// LOG.error(e); +// } +// } } /** @@ -198,13 +206,14 @@ public class BackupManager implements Closeable { * @throws BackupException exception */ protected BackupContext createBackupContext(String backupId, BackupType type, - List tableList, String targetRootDir) throws BackupException { + List tableList, String targetRootDir, int workers, int bandwidth) + throws BackupException { if (targetRootDir == null) { throw new BackupException("Wrong backup request parameter: target backup root directory"); } - if (type == BackupType.FULL && tableList == null) { + if (type == BackupType.FULL && tableList == null || tableList.size() == 0) { // If table list is null for full backup, which means backup all tables. Then fill the table // list with all user tables from meta. It no table available, throw the request exception. @@ -228,8 +237,12 @@ public class BackupManager implements Closeable { } // there are one or more tables in the table list - return new BackupContext(backupId, type, tableList.toArray(new TableName[tableList.size()]), + backupContext = new BackupContext(backupId, type, + tableList.toArray(new TableName[tableList.size()]), targetRootDir); + backupContext.setBandwidth(bandwidth); + backupContext.setWorkers(workers); + return backupContext; } /** @@ -298,7 +311,7 @@ public class BackupManager implements Closeable { ArrayList allHistoryList = getBackupHistory(); for (BackupCompleteData backup : allHistoryList) { BackupImage image = - new BackupImage(backup.getBackupToken(), BackupType.valueOf(backup.getType()), + new BackupImage(backup.getBackupId(), BackupType.valueOf(backup.getType()), backup.getBackupRootPath(), backup.getTableList(), Long.parseLong(backup.getStartTime()), Long.parseLong(backup .getEndTime())); @@ -325,11 +338,10 @@ public class BackupManager implements Closeable { } else { Path logBackupPath = HBackupFileSystem.getLogBackupPath(backup.getBackupRootPath(), - backup.getBackupToken()); + backup.getBackupId()); LOG.debug("Current backup has an incremental backup ancestor, " + "touching its image manifest in " + logBackupPath.toString() + " to construct the dependency."); - BackupManifest lastIncrImgManifest = new BackupManifest(conf, logBackupPath); BackupImage lastIncrImage = lastIncrImgManifest.getBackupImage(); ancestors.add(lastIncrImage); @@ -447,7 +459,7 @@ public class BackupManager implements Closeable { * @throws IOException exception */ public Set getIncrementalBackupTableSet() throws IOException { - return BackupSystemTable.getIncrementalBackupTableSet(getConnection()); + return BackupSystemTable.getIncrementalBackupTableSet(); } /** @@ -465,7 +477,8 @@ public class BackupManager implements Closeable { * safely purged. */ public void recordWALFiles(List files) throws IOException { - systemTable.addWALFiles(files, backupContext.getBackupId()); + systemTable.addWALFiles(files, + backupContext.getBackupId(), backupContext.getTargetRootDir()); } /** @@ -473,7 +486,7 @@ public class BackupManager implements Closeable { * @return WAL files iterator from hbase:backup * @throws IOException */ - public Iterator getWALFilesFromBackupSystem() throws IOException { + public Iterator getWALFilesFromBackupSystem() throws IOException { return systemTable.getWALFilesIterator(); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java index 6264fc5..95a1a25 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.BackupProtos; +import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupManifest.Builder; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.util.FSUtils; @@ -102,11 +103,18 @@ public class BackupManifest { for(HBaseProtos.TableName tn : tableListList) { tableList.add(ProtobufUtil.toTableName(tn)); } + + List ancestorList = im.getAncestorsList(); + BackupType type = im.getBackupType() == BackupProtos.BackupType.FULL ? BackupType.FULL: BackupType.INCREMENTAL; - return new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs); + BackupImage image = new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs); + for(BackupProtos.BackupImage img: ancestorList) { + image.addAncestor(fromProto(img)); + } + return image; } BackupProtos.BackupImage toProto() { @@ -204,7 +212,7 @@ public class BackupManifest { public boolean hasTable(TableName table) { for (TableName t : tableList) { - if (t.getNameAsString().equals(table)) { + if (t.getNameAsString().equals(table.getNameAsString())) { return true; } } @@ -266,6 +274,13 @@ public class BackupManifest { // the indicator of the image compaction private boolean isCompacted = false; + + // WAL files relative references + // For WAL files which have been backuped + // during other backup sessions we keep references in + // backup manifest + // Format: backupId/wal_file_name + private List relWALRefs; /** * Construct manifest for a ongoing backup. * @param backupCtx The ongoing backup context @@ -283,6 +298,17 @@ public class BackupManifest { this.loadTableList(backupCtx.getTableNames()); } + public void setRelativeWALReferences(List refs) + { + relWALRefs = refs; + } + + public List getRelativeWALReferences() + { + return relWALRefs; + } + + /** * Construct a table level manifest for a backup of the named table. * @param backupCtx The ongoing backup context @@ -370,6 +396,7 @@ public class BackupManifest { loadIncrementalTimestampMap(proto); loadDependency(proto); this.isCompacted = proto.getCompacted(); + loadRelativeWALRefs(proto); //TODO: merge will be implemented by future jira LOG.debug("Loaded manifest instance from manifest file: " + FSUtils.getPath(subFile.getPath())); @@ -386,6 +413,15 @@ public class BackupManifest { } } + private void loadRelativeWALRefs( + org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupManifest proto) { + List list = proto.getRelWalRefList(); + if(list != null && list.size() > 0){ + this.relWALRefs = new ArrayList(); + this.relWALRefs.addAll(list); + } + } + private void loadIncrementalTimestampMap(BackupProtos.BackupManifest proto) { List list = proto.getTstMapList(); if(list == null || list.size() == 0) return; @@ -405,10 +441,15 @@ public class BackupManifest { } private void loadDependency(BackupProtos.BackupManifest proto) { + if(LOG.isDebugEnabled()) { + LOG.debug("load dependency for: "+proto.getBackupId()); + } + dependency = new HashMap(); List list = proto.getDependentBackupImageList(); for (BackupProtos.BackupImage im : list) { - dependency.put(im.getBackupId(), BackupImage.fromProto(im)); + BackupImage bim = BackupImage.fromProto(im); + dependency.put(im.getBackupId(), bim); } } @@ -463,22 +504,25 @@ public class BackupManifest { public void store(Configuration conf) throws BackupException { byte[] data = toByteArray(); + // write the file, overwrite if already exist Path manifestFilePath = new Path(new Path((this.tableBackupDir != null ? this.tableBackupDir : this.logBackupDir)) ,MANIFEST_FILE_NAME); + LOG.info("Manifestfilestored_to " + (this.tableBackupDir != null ? this.tableBackupDir + : this.logBackupDir) + Path.SEPARATOR + MANIFEST_FILE_NAME); try { FSDataOutputStream out = manifestFilePath.getFileSystem(conf).create(manifestFilePath, true); out.write(data); out.close(); } catch (IOException e) { - LOG.error(e); + throw new BackupException(e.getMessage()); } - LOG.debug("Manifestfilestored_to " + this.tableBackupDir != null ? this.tableBackupDir - : this.logBackupDir + Path.SEPARATOR + MANIFEST_FILE_NAME); + LOG.info("Manifestfilestored_to " + (this.tableBackupDir != null ? this.tableBackupDir + : this.logBackupDir) + Path.SEPARATOR + MANIFEST_FILE_NAME); } /** @@ -500,11 +544,23 @@ public class BackupManifest { setIncrementalTimestampMap(builder); setDependencyMap(builder); builder.setCompacted(this.isCompacted); + setRelativeWALRefs(builder); return builder.build().toByteArray(); } + private void setRelativeWALRefs(Builder builder) { + if(this.relWALRefs == null) { + return; + } + for(String ref : relWALRefs) { + builder.addRelWalRef(ref); + } + } + private void setIncrementalTimestampMap(BackupProtos.BackupManifest.Builder builder) { - if (this.incrTimeRanges == null) return; + if (this.incrTimeRanges == null) { + return; + } for (Entry> entry: this.incrTimeRanges.entrySet()) { TableName key = entry.getKey(); HashMap value = entry.getValue(); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java index d0ce059..7233bfa 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java @@ -37,7 +37,8 @@ public final class BackupRestoreConstants { public static final String BACKUPID_PREFIX = "backup_"; public static enum BackupCommand { - CREATE, CANCEL, DELETE, DESCRIBE, HISTORY, STATUS, CONVERT, MERGE, STOP, SHOW, HELP, + CREATE, CANCEL, DELETE, DESCRIBE, HISTORY, STATUS, CONVERT, MERGE, STOP, SHOW, HELP, PROGRESS, SET, + SET_ADD, SET_REMOVE, SET_DELETE, SET_DESCRIBE, SET_LIST } private BackupRestoreConstants() { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java index 02cc9c5..da377d9 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java @@ -27,14 +27,22 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; import java.util.TreeSet; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.NamespaceNotFoundException; +import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.impl.BackupHandler.BackupState; import org.apache.hadoop.hbase.backup.impl.BackupUtil.BackupCompleteData; @@ -42,6 +50,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -58,46 +67,118 @@ import org.apache.hadoop.hbase.protobuf.generated.BackupProtos; @InterfaceAudience.Private @InterfaceStability.Evolving public final class BackupSystemTable implements Closeable { + + static class WALItem { + String backupId; + String walFile; + String backupRoot; + + WALItem(String backupId, String walFile, String backupRoot) + { + this.backupId = backupId; + this.walFile = walFile; + this.backupRoot = backupRoot; + } + + public String getBackupId() { + return backupId; + } + public String getWalFile() { + return walFile; + } + + public String getBackupRoot() { + return backupRoot; + } + + public String toString() { + return backupRoot+"/"+backupId + "/" + walFile; + } + + } + private static final Log LOG = LogFactory.getLog(BackupSystemTable.class); private final static TableName tableName = TableName.BACKUP_TABLE_NAME; final static byte[] familyName = "f".getBytes(); // Connection to HBase cluster, shared // among all instances - private final Connection connection; + private static Connection connection; // Cluster configuration - private final Configuration conf; + private static Configuration config; - /** - * Create a BackupSystemTable object for the given Connection. Connection is NOT owned by this - * instance and has to be closed explicitly. - * @param connection - * @throws IOException - */ - public BackupSystemTable(Connection connection) throws IOException { - this.connection = connection; - this.conf = connection.getConfiguration(); + private static int refCount = 0; + private static WriteLock refCountLock = new ReentrantReadWriteLock().writeLock(); - createSystemTableIfNotExists(); + public static Connection getConnection() + { + return connection; + } + +//TODO: namespace 'backup' +// private static void verifySystemNamespaceExists(Admin admin) throws IOException { +// try { +// admin.getNamespaceDescriptor(TABLE_NAMESPACE); +// } catch (NamespaceNotFoundException e) { +// admin.createNamespace(NamespaceDescriptor.create(TABLE_NAMESPACE).build()); +// } +// } + + public BackupSystemTable(Configuration conf) throws IOException { + try{ + refCountLock.lock(); + if (connection == null) { + connection = ConnectionFactory.createConnection(conf); + config = conf; + // Verify hbase:backup exists - FOR TESTS ONLY + // hbase:backup is created by HMaster on start up + createSystemTableIfNotExists(); + } + refCount++; + } finally{ + refCountLock.unlock(); + } } - @Override + /** + * Closes HBase connection, to avoid connection leak + * we perform reference counting and when refNumber reaches 0 + * we close underlying connection + */ public void close() { + try{ + BackupSystemTable.refCountLock.lock(); + BackupSystemTable.refCount--; + if(connection != null && BackupSystemTable.refCount == 0) { + try{ + connection.close(); + connection = null; + } catch(Exception e){ + LOG.error(e); + } + } + } finally{ + BackupSystemTable.refCountLock.unlock(); + } } private void createSystemTableIfNotExists() throws IOException { try(Admin admin = connection.getAdmin()) { if (admin.tableExists(tableName) == false) { +// verifySystemNamespaceExists(admin); HTableDescriptor tableDesc = new HTableDescriptor(tableName); HColumnDescriptor colDesc = new HColumnDescriptor(familyName); colDesc.setMaxVersions(1); int ttl = - conf.getInt(HConstants.BACKUP_SYSTEM_TTL_KEY, HConstants.BACKUP_SYSTEM_TTL_DEFAULT); + config.getInt(HConstants.BACKUP_SYSTEM_TTL_KEY, HConstants.BACKUP_SYSTEM_TTL_DEFAULT); colDesc.setTimeToLive(ttl); tableDesc.addFamily(colDesc); admin.createTable(tableDesc); } + + } catch (TableExistsException ee){ + //just ignore that } catch (IOException e) { LOG.error(e); throw e; @@ -275,7 +356,7 @@ public final class BackupSystemTable implements Closeable { } BackupCompleteData history = new BackupCompleteData(); - history.setBackupToken(context.getBackupId()); + history.setBackupId(context.getBackupId()); history.setStartTime(Long.toString(context.getStartTs())); history.setEndTime(Long.toString(context.getEndTs())); history.setBackupRootPath(context.getTargetRootDir()); @@ -416,7 +497,7 @@ public final class BackupSystemTable implements Closeable { * @return set of tableNames * @throws IOException exception */ - public static Set getIncrementalBackupTableSet(Connection connection) + public static Set getIncrementalBackupTableSet() throws IOException { if (LOG.isDebugEnabled()) { LOG.debug("get incr backup table set from hbase:backup"); @@ -458,12 +539,17 @@ public final class BackupSystemTable implements Closeable { * @param files files * @throws IOException exception */ - public void addWALFiles(List files, String backupId) throws IOException { + public void addWALFiles(List files, String backupId, + String backupRoot) throws IOException { if (LOG.isDebugEnabled()) { - LOG.debug("add WAL files to hbase:backup"); + LOG.debug("add WAL files to hbase:backup: "+backupId +" "+backupRoot); + for(String f: files){ + LOG.debug(f); + } } try (Table table = connection.getTable(tableName)) { - List puts = BackupSystemTableHelper.createPutsForAddWALFiles(files, backupId); + List puts = + BackupSystemTableHelper.createPutsForAddWALFiles(files, backupId, backupRoot); table.put(puts); } } @@ -473,7 +559,7 @@ public final class BackupSystemTable implements Closeable { * @param files files * @throws IOException exception */ - public Iterator getWALFilesIterator() throws IOException { + public Iterator getWALFilesIterator() throws IOException { if (LOG.isDebugEnabled()) { LOG.debug("get WAL files from hbase:backup"); } @@ -481,7 +567,7 @@ public final class BackupSystemTable implements Closeable { Scan scan = BackupSystemTableHelper.createScanForGetWALs(); final ResultScanner scanner = table.getScanner(scan); final Iterator it = scanner.iterator(); - return new Iterator() { + return new Iterator() { @Override public boolean hasNext() { @@ -499,13 +585,23 @@ public final class BackupSystemTable implements Closeable { } @Override - public String next() { + public WALItem next() { Result next = it.next(); List cells = next.listCells(); byte[] buf = cells.get(0).getValueArray(); int len = cells.get(0).getValueLength(); int offset = cells.get(0).getValueOffset(); - return new String(buf, offset, len); + String backupId = new String(buf, offset, len); + buf = cells.get(1).getValueArray(); + len = cells.get(1).getValueLength(); + offset = cells.get(1).getValueOffset(); + String walFile = new String(buf, offset, len); + //walFile = walFile.substring(walFile.lastIndexOf(Path.SEPARATOR) +1); + buf = cells.get(2).getValueArray(); + len = cells.get(2).getValueLength(); + offset = cells.get(2).getValueOffset(); + String backupRoot = new String(buf, offset, len); + return new WALItem(backupId, walFile, backupRoot); } @Override @@ -559,4 +655,207 @@ public final class BackupSystemTable implements Closeable { return result; } } + + /** + * BACKUP SETS + */ + + /** + * Get backup set list + * @return backup set list + * @throws IOException + */ + public List backupSetList() throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug(" backup set list"); + } + List list = new ArrayList(); + Table table = null; + ResultScanner scanner = null; + try { + table = connection.getTable(tableName); + Scan scan = BackupSystemTableHelper.createScanForBackupSetList(); + scan.setMaxVersions(1); + scanner = table.getScanner(scan); + Result res = null; + while ((res = scanner.next()) != null) { + res.advance(); + list.add(BackupSystemTableHelper.cellKeyToBackupSetName(res.current())); + } + return list; + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * Get backup set description (list of tables) + * @param setName set's name + * @return list of tables in a backup set + * @throws IOException + */ + public String[] backupSetDescribe(String name) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug(" backup set describe: "+name); + } + Table table = null; + try { + table = connection.getTable(tableName); + Get get = BackupSystemTableHelper.createGetForBackupSet(name); + Result res = table.get(get); + if(res.isEmpty()) return null; + res.advance(); + String[] tables = + BackupSystemTableHelper.cellValueToBackupSet(res.current()); + return tables; + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * Add backup set (list of tables) + * @param name - set name + * @param tables - list of tables, comma-separated + * @throws IOException + */ + public void backupSetAdd(String name, String[] newTables) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug(" backup set add: "+name); + } + Table table = null; + String[] union = null; + try { + table = connection.getTable(tableName); + Get get = BackupSystemTableHelper.createGetForBackupSet(name); + Result res = table.get(get); + if(res.isEmpty()) { + union = newTables; + } else { + res.advance(); + String[] tables = + BackupSystemTableHelper.cellValueToBackupSet(res.current()); + union = merge(tables, newTables); + } + Put put = BackupSystemTableHelper.createPutForBackupSet(name, union); + table.put(put); + } finally { + if (table != null) { + table.close(); + } + } + } + + private String[] merge(String[] tables, String[] newTables) { + List list = new ArrayList(); + // Add all from tables + for(String t: tables){ + list.add(t); + } + for(String nt: newTables){ + if(list.contains(nt)) continue; + list.add(nt); + } + String[] arr = new String[list.size()]; + list.toArray(arr); + return arr; + } + + /** + * Remove tables from backup set (list of tables) + * @param name - set name + * @param tables - list of tables, comma-separated + * @throws IOException + */ + public void backupSetRemove(String name, String[] toRemove) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug(" backup set describe: " + name); + } + Table table = null; + String[] disjoint = null; + try { + table = connection.getTable(tableName); + Get get = BackupSystemTableHelper.createGetForBackupSet(name); + Result res = table.get(get); + if (res.isEmpty()) { + return; + } else { + res.advance(); + String[] tables = BackupSystemTableHelper.cellValueToBackupSet(res.current()); + disjoint = disjoin(tables, toRemove); + } + if (disjoint.length > 0) { + Put put = BackupSystemTableHelper.createPutForBackupSet(name, disjoint); + table.put(put); + } else { + // Delete + backupSetDescribe(name); + } + } finally { + if (table != null) { + table.close(); + } + } + } + + private String[] disjoin(String[] tables, String[] toRemove) { + List list = new ArrayList(); + // Add all from tables + for (String t : tables) { + list.add(t); + } + for (String nt : toRemove) { + if (list.contains(nt)) { + list.remove(nt); + } + } + String[] arr = new String[list.size()]; + list.toArray(arr); + return arr; + } + + /** + * Delete backup set + * @param name set's name + * @throws IOException + */ + public void backupSetDelete(String name) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug(" backup set delete: " + name); + } + Table table = null; + try { + table = connection.getTable(tableName); + Delete del = BackupSystemTableHelper.createDeleteForBackupSet(name); + table.delete(del); + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * Get backup system table descriptor + * @return descriptor + */ + public static HTableDescriptor getSystemTableDescriptor() { + HTableDescriptor tableDesc = new HTableDescriptor(tableName); + HColumnDescriptor colDesc = new HColumnDescriptor(familyName); + colDesc.setMaxVersions(1); + if(config == null) config = HBaseConfiguration.create(); + int ttl = + config.getInt(HConstants.BACKUP_SYSTEM_TTL_KEY, HConstants.BACKUP_SYSTEM_TTL_DEFAULT); + colDesc.setTimeToLive(ttl); + tableDesc.addFamily(colDesc); + return tableDesc; + } + + public static String getTableName() { + return tableName.getNameAsString(); + } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java index ac096b7..bbadba0 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.util.Bytes; +import org.mortbay.log.Log; /** @@ -61,9 +62,11 @@ public final class BackupSystemTableHelper { private final static String TABLE_RS_LOG_MAP_PREFIX = "trslm."; private final static String RS_LOG_TS_PREFIX = "rslogts."; private final static String WALS_PREFIX = "wals."; + private final static String SET_KEY_PREFIX = "set."; private final static byte[] col1 = "col1".getBytes(); private final static byte[] col2 = "col2".getBytes(); + private final static byte[] col3 = "col3".getBytes(); private final static byte[] EMPTY_VALUE = new byte[] {}; @@ -280,7 +283,8 @@ public final class BackupSystemTableHelper { * @return put list * @throws IOException exception */ - public static List createPutsForAddWALFiles(List files, String backupId) + public static List createPutsForAddWALFiles(List files, + String backupId, String backupRoot) throws IOException { List puts = new ArrayList(); @@ -289,6 +293,7 @@ public final class BackupSystemTableHelper { Put put = new Put(row); put.addColumn(BackupSystemTable.familyName, col1, backupId.getBytes()); put.addColumn(BackupSystemTable.familyName, col2, file.getBytes()); + put.addColumn(BackupSystemTable.familyName, col3, backupRoot.getBytes()); puts.add(put); } return puts; @@ -305,7 +310,8 @@ public final class BackupSystemTableHelper { stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); scan.setStartRow(startRow); scan.setStopRow(stopRow); - scan.addColumn(BackupSystemTable.familyName, col2); + //TODO we will need backupId and backup root in the future + scan.addFamily(BackupSystemTable.familyName); return scan; } /** @@ -322,4 +328,93 @@ public final class BackupSystemTableHelper { return get; } + + /** + * Creates Scan operation to load backup set list + * @return scan operation + */ + static Scan createScanForBackupSetList() { + Scan scan = new Scan(); + byte[] startRow = SET_KEY_PREFIX.getBytes(); + byte[] stopRow = Arrays.copyOf(startRow, startRow.length); + stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); + scan.setStartRow(startRow); + scan.setStopRow(stopRow); + scan.addFamily(BackupSystemTable.familyName); + return scan; + } + + /** + * Creates Get operation to load backup set content + * @return get operation + */ + static Get createGetForBackupSet(String name) { + byte[] row = (SET_KEY_PREFIX + name).getBytes(); + Get get = new Get(row); + get.addFamily(BackupSystemTable.familyName); + return get; + } + + /** + * Creates Delete operation to delete backup set content + * @return delete operation + */ + static Delete createDeleteForBackupSet(String name) { + byte[] row = (SET_KEY_PREFIX + name).getBytes(); + Delete del = new Delete(row); + del.addFamily(BackupSystemTable.familyName); + return del; + } + + + /** + * Creates Put operation to update backup set content + * @return put operation + */ + static Put createPutForBackupSet(String name, String[] tables) { + byte[] row = (SET_KEY_PREFIX + name).getBytes(); + Put put = new Put(row); + byte[] value = convertToByteArray(tables); + put.addColumn(BackupSystemTable.familyName, col1, value); + return put; + } + + private static byte[] convertToByteArray(String[] tables) { + StringBuffer sb = new StringBuffer(); + for(int i=0; i < tables.length; i++){ + sb.append(tables[i]); + if(i < tables.length -1){ + sb.append(","); + } + } + return sb.toString().getBytes(); + } + + + /** + * Converts cell to backup set list. + * @param current - cell + * @return backup set + * @throws IOException + */ + static String[] cellValueToBackupSet(Cell current) throws IOException { + byte[] data = CellUtil.cloneValue(current); + if( data != null && data.length > 0){ + return new String(data).split(","); + } else{ + return new String[0]; + } + } + + /** + * Converts cell key to backup set name. + * @param current - cell + * @return backup set namd + * @throws IOException + */ + static String cellKeyToBackupSetName(Cell current) throws IOException { + byte[] data = CellUtil.cloneRow(current); + return new String(data).substring(SET_KEY_PREFIX.length()); + } + } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupUtil.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupUtil.java index 96812c9..c3aed31 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupUtil.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupUtil.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.net.URLDecoder; import java.util.ArrayList; import java.util.Collections; +import java.util.Date; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -139,7 +140,6 @@ public final class BackupUtil { descriptors.createTableDescriptorForTableDirectory(target, orig, false); LOG.debug("Finished copying tableinfo."); - HBaseAdmin hbadmin = null; // TODO: optimize List regions = null; try(Connection conn = ConnectionFactory.createConnection(conf); @@ -254,7 +254,7 @@ public final class BackupUtil { private String type; private String backupRootPath; private List tableList; - private String backupToken; + private String backupId; private String bytesCopied; private List ancestors; @@ -277,12 +277,12 @@ public final class BackupUtil { this.bytesCopied = bytesCopied; } - public String getBackupToken() { - return backupToken; + public String getBackupId() { + return backupId; } - public void setBackupToken(String backupToken) { - this.backupToken = backupToken; + public void setBackupId(String backupToken) { + this.backupId = backupToken; } public String getStartTime() { @@ -328,14 +328,40 @@ public final class BackupUtil { @Override public int compareTo(BackupCompleteData o) { Long thisTS = - new Long(this.getBackupToken().substring(this.getBackupToken().lastIndexOf("_") + 1)); + new Long(this.getBackupId().substring(this.getBackupId().lastIndexOf("_") + 1)); Long otherTS = - new Long(o.getBackupToken().substring(o.getBackupToken().lastIndexOf("_") + 1)); + new Long(o.getBackupId().substring(o.getBackupId().lastIndexOf("_") + 1)); return thisTS.compareTo(otherTS); } + + public String toString() { + StringBuffer sb = new StringBuffer(); + sb.append("BackupId =").append(backupId).append("\n"); + sb.append("Type =").append(type).append("\n"); + sb.append("Start time =").append(new Date(Long.parseLong(startTime))).append("\n"); + sb.append("End time =").append(new Date(Long.parseLong(endTime))).append("\n"); + sb.append("Root path =").append(backupRootPath).append("\n"); + sb.append("Type =").append(type).append("\n"); + sb.append("Table list =").append(tableList).append("\n"); + sb.append("Bytes copied =").append(type).append("\n"); + sb.append("Ancestors =").append(BackupUtil.merge(ancestors)).append("\n\n"); + return sb.toString(); + } } + public static String merge(List list) { + if (list == null || list.size() == 0) return ""; + StringBuffer sb = new StringBuffer(); + for (int i = 0; i < list.size(); i++) { + sb.append(list.get(i)); + if (i < list.size() - 1) { + sb.append(","); + } + } + return sb.toString(); + } + /** * Sort history list by start time in descending order. * @param historyList history list @@ -450,4 +476,78 @@ public final class BackupUtil { } return ret; } + + public static void cleanupBackupData(BackupContext context, Configuration conf) + throws IOException + { + cleanupHLogDir(context, conf); + cleanupTargetDir(context, conf); + } + + /** + * Clean up directories which are generated when DistCp copying hlogs. + * @throws IOException + */ + private static void cleanupHLogDir(BackupContext backupContext, Configuration conf) + throws IOException { + + String logDir = backupContext.getHLogTargetDir(); + if (logDir == null) { + LOG.warn("No log directory specified for " + backupContext.getBackupId()); + return; + } + + Path rootPath = new Path(logDir).getParent(); + FileSystem fs = FileSystem.get(rootPath.toUri(), conf); + FileStatus[] files = FSUtils.listStatus(fs, rootPath); + if (files == null) { + return; + } + for (FileStatus file : files) { + LOG.debug("Delete log files: " + file.getPath().getName()); + FSUtils.delete(fs, file.getPath(), true); + } + } + + /** + * Clean up the data at target directory + */ + private static void cleanupTargetDir(BackupContext backupContext, Configuration conf) { + try { + // clean up the data at target directory + LOG.debug("Trying to cleanup up target dir : " + backupContext.getBackupId()); + String targetDir = backupContext.getTargetRootDir(); + if (targetDir == null) { + LOG.warn("No target directory specified for " + backupContext.getBackupId()); + return; + } + + FileSystem outputFs = + FileSystem.get(new Path(backupContext.getTargetRootDir()).toUri(), conf); + + for (TableName table : backupContext.getTables()) { + Path targetDirPath = + new Path(HBackupFileSystem.getTableBackupDir(backupContext.getTargetRootDir(), + backupContext.getBackupId(), table)); + if (outputFs.delete(targetDirPath, true)) { + LOG.info("Cleaning up backup data at " + targetDirPath.toString() + " done."); + } else { + LOG.info("No data has been found in " + targetDirPath.toString() + "."); + } + + Path tableDir = targetDirPath.getParent(); + FileStatus[] backups = FSUtils.listStatus(outputFs, tableDir); + if (backups == null || backups.length == 0) { + outputFs.delete(tableDir, true); + LOG.debug(tableDir.toString() + " is empty, remove it."); + } + } + + } catch (IOException e1) { + LOG.error("Cleaning up backup data of " + backupContext.getBackupId() + " at " + + backupContext.getTargetRootDir() + " failed due to " + e1.getMessage() + "."); + } + } + + } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupProcedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupProcedure.java index 3d03312..4355115 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupProcedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupProcedure.java @@ -84,7 +84,8 @@ public class FullTableBackupProcedure this.tableList = tableList; this.targetRootDir = targetRootDir; backupContext = - backupManager.createBackupContext(backupId, BackupType.FULL, tableList, targetRootDir); + backupManager.createBackupContext(backupId, BackupType.FULL, + tableList, targetRootDir, workers, (int)bandwidth); } @Override diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java index 61e15c4..e04c51e 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable.WALItem; /** * After a full backup was created, the incremental backup will only store the changes made @@ -84,7 +85,7 @@ public class IncrementalBackupManager { HashMap> previousTimestampMap = backupManager.readLogTimestampMap(); - previousTimestampMins = BackupUtil.getRSLogTimestampMins(previousTimestampMap); + previousTimestampMins = BackupUtil.getRSLogTimestampMins(previousTimestampMap); if (LOG.isDebugEnabled()) { LOG.debug("StartCode " + savedStartCode + "for backupID " + backupContext.getBackupId()); @@ -106,42 +107,81 @@ public class IncrementalBackupManager { newTimestamps = backupManager.readRegionServerLastLogRollResult(); logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode); - logList.addAll(getLogFilesFromBackupSystem(previousTimestampMins, newTimestamps)); + List logFromSystemTable = + getLogFilesFromBackupSystem(previousTimestampMins, + newTimestamps, backupManager.getBackupContext().getTargetRootDir()); + addLogsFromBackupSystemToContext(logFromSystemTable); + + logList = excludeAlreadyBackupedWALs(logList, logFromSystemTable); backupContext.setIncrBackupFileList(logList); return newTimestamps; } + + private List excludeAlreadyBackupedWALs(List logList, + List logFromSystemTable) { + + List backupedWALList = toWALList(logFromSystemTable); + logList.removeAll(backupedWALList); + return logList; + } + + private List toWALList(List logFromSystemTable) { + + List list = new ArrayList(logFromSystemTable.size()); + for(WALItem item : logFromSystemTable){ + list.add(item.getWalFile()); + } + return list; + } + + private void addLogsFromBackupSystemToContext(List logFromSystemTable) { + BackupContext context = backupManager.getBackupContext(); + List walFiles = new ArrayList(); + for(WALItem item : logFromSystemTable){ + Path p = new Path(item.getWalFile()); + String walFileName = p.getName(); + String backupId = item.getBackupId(); + String relWALPath = backupId + Path.SEPARATOR+walFileName; + walFiles.add(relWALPath); + } + context.setRelWALRefs(walFiles); + } + + /** - * For each region server: get all log files newer than the last timestamps but not newer than the - * newest timestamps. FROM hbase:backup table + * For each region server: get all log files newer than the last timestamps, + * but not newer than the newest timestamps. FROM hbase:backup table * @param olderTimestamps - the timestamp for each region server of the last backup. * @param newestTimestamps - the timestamp for each region server that the backup should lead to. * @return list of log files which needs to be added to this backup * @throws IOException */ - private List getLogFilesFromBackupSystem(HashMap olderTimestamps, - HashMap newestTimestamps) throws IOException { - List logFiles = new ArrayList(); - Iterator it = backupManager.getWALFilesFromBackupSystem(); - + private List getLogFilesFromBackupSystem(HashMap olderTimestamps, + HashMap newestTimestamps, String backupRoot) throws IOException { + List logFiles = new ArrayList(); + Iterator it = backupManager.getWALFilesFromBackupSystem(); while (it.hasNext()) { - String walFileName = it.next(); + WALItem item = it.next(); + String rootDir = item.getBackupRoot(); + if(!rootDir.equals(backupRoot)) { + continue; + } + String walFileName = item.getWalFile(); String server = BackupUtil.parseHostNameFromLogFile(new Path(walFileName)); - //String server = getServer(walFileName); Long tss = getTimestamp(walFileName); Long oldTss = olderTimestamps.get(server); + Long newTss = newestTimestamps.get(server); if (oldTss == null){ - logFiles.add(walFileName); + logFiles.add(item); continue; } - Long newTss = newestTimestamps.get(server); if (newTss == null) { newTss = Long.MAX_VALUE; } - if (tss > oldTss && tss < newTss) { - logFiles.add(walFileName); + logFiles.add(item); } } return logFiles; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalRestoreService.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalRestoreService.java index 12ecbe9..8904184 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalRestoreService.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalRestoreService.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.backup.impl; import java.io.IOException; import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -29,6 +30,13 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; @InterfaceStability.Evolving public interface IncrementalRestoreService extends Configurable{ - public void run(String logDirectory, TableName[] fromTables, TableName[] toTables) + /** + * Run restore operation + * @param logDirectoryPaths - path array of WAL log directories + * @param fromTables - from tables + * @param toTables - to tables + * @throws IOException + */ + public void run(Path[] logDirectoryPaths, TableName[] fromTables, TableName[] toTables) throws IOException; } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupProcedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupProcedure.java index d8c11dd..01c09f5 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupProcedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupProcedure.java @@ -74,8 +74,8 @@ public class IncrementalTableBackupProcedure this.backupId = backupId; this.tableList = tableList; this.targetRootDir = targetRootDir; - backupContext = backupManager.createBackupContext(backupId, BackupType.INCREMENTAL, tableList, - targetRootDir); + backupContext = backupManager.createBackupContext(backupId, + BackupType.INCREMENTAL, tableList, targetRootDir, workers, (int)bandwidth); } @Override diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java index c0c5220..fb05cef 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map.Entry; import java.util.Set; @@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.util.StringUtils; /** * The main class which interprets the given arguments and trigger restore operation. @@ -52,7 +54,6 @@ public final class RestoreClientImpl implements RestoreClient { private static final Log LOG = LogFactory.getLog(RestoreClientImpl.class); private Configuration conf; - private Set lastRestoreImagesSet; public RestoreClientImpl() { } @@ -111,11 +112,10 @@ public final class RestoreClientImpl implements RestoreClient { checkTargetTables(tTableArray, isOverwrite); // start restore process - Set restoreImageSet = - restoreStage(backupManifestMap, sTableArray, tTableArray, autoRestore); + + restoreStage(backupManifestMap, sTableArray, tTableArray, autoRestore); LOG.info("Restore for " + Arrays.asList(sTableArray) + " are successful!"); - lastRestoreImagesSet = restoreImageSet; } catch (IOException e) { LOG.error("ERROR: restore failed with error: " + e.getMessage()); @@ -126,13 +126,6 @@ public final class RestoreClientImpl implements RestoreClient { return false; } - /** - * Get last restore image set. The value is globally set for the latest finished restore. - * @return the last restore image set - */ - public Set getLastRestoreImagesSet() { - return lastRestoreImagesSet; - } private boolean validate(HashMap backupManifestMap) throws IOException { @@ -147,10 +140,6 @@ public final class RestoreClientImpl implements RestoreClient { imageSet.addAll(depList); } - // todo merge - LOG.debug("merge will be implemented in future jira"); - // BackupUtil.clearMergedImages(table, imageSet, conf); - LOG.info("Dependent image(s) from old to new:"); for (BackupImage image : imageSet) { String imageDir = @@ -189,7 +178,7 @@ public final class RestoreClientImpl implements RestoreClient { } } else { LOG.info("HBase table " + tableName - + " does not exist. It will be create during backup process"); + + " does not exist. It will be created during restore process"); } } } @@ -223,25 +212,27 @@ public final class RestoreClientImpl implements RestoreClient { * @return set of BackupImages restored * @throws IOException exception */ - private Set restoreStage( + private void restoreStage( HashMap backupManifestMap, TableName[] sTableArray, TableName[] tTableArray, boolean autoRestore) throws IOException { TreeSet restoreImageSet = new TreeSet(); for (int i = 0; i < sTableArray.length; i++) { - restoreImageSet.clear(); + //restoreImageSet.clear(); TableName table = sTableArray[i]; + BackupManifest manifest = backupManifestMap.get(table); if (autoRestore) { // Get the image list of this backup for restore in time order from old // to new. + List list = new ArrayList(); + list.add(manifest.getBackupImage()); + List depList = manifest.getDependentListByTable(table); + list.addAll(depList); TreeSet restoreList = - new TreeSet(manifest.getDependentListByTable(table)); + new TreeSet(list); LOG.debug("need to clear merged Image. to be implemented in future jira"); - - for (BackupImage image : restoreList) { - restoreImage(image, table, tTableArray[i]); - } + restoreImages(restoreList.iterator(), table, tTableArray[i]); restoreImageSet.addAll(restoreList); } else { BackupImage image = manifest.getBackupImage(); @@ -268,9 +259,7 @@ public final class RestoreClientImpl implements RestoreClient { } } } - } - return restoreImageSet; } /** @@ -289,9 +278,9 @@ public final class RestoreClientImpl implements RestoreClient { RestoreUtil restoreTool = new RestoreUtil(conf, rootPath, backupId); BackupManifest manifest = HBackupFileSystem.getManifest(sTable, conf, rootPath, backupId); - Path tableBackupPath = HBackupFileSystem.getTableBackupPath(rootPath, sTable, backupId); + Path tableBackupPath = HBackupFileSystem.getTableBackupPath(sTable, rootPath, backupId); - // todo: convert feature will be provided in a future jira + // TODO: convert feature will be provided in a future JIRA boolean converted = false; if (manifest.getType() == BackupType.FULL || converted) { @@ -303,10 +292,77 @@ public final class RestoreClientImpl implements RestoreClient { HBackupFileSystem.getLogBackupDir(image.getRootDir(), image.getBackupId()); LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from incremental backup image " + logBackupDir); - restoreTool.incrementalRestoreTable(logBackupDir, new TableName[] { sTable }, + restoreTool.incrementalRestoreTable(new Path[]{ new Path(logBackupDir)}, new TableName[] { sTable }, new TableName[] { tTable }); } LOG.info(sTable + " has been successfully restored to " + tTable); } + + /** + * Restore operation handle each backupImage in iterator + * @param it: backupImage iterator - ascending + * @param sTable: table to be restored + * @param tTable: table to be restored to + * @throws IOException exception + */ + private void restoreImages(Iterator it, TableName sTable, TableName tTable) + throws IOException { + + // First image MUST be image of a FULL backup + BackupImage image = it.next(); + + String rootDir = image.getRootDir(); + String backupId = image.getBackupId(); + Path backupRoot = new Path(rootDir); + + //HBackupFileSystem hFS = new HBackupFileSystem(conf, new Path(rootDir), backupId); + // We need hFS only for full restore (see the code) + RestoreUtil restoreTool = new RestoreUtil(conf, backupRoot, backupId); + BackupManifest manifest = HBackupFileSystem.getManifest(sTable, conf, backupRoot, backupId); + + Path tableBackupPath = HBackupFileSystem.getTableBackupPath(sTable, backupRoot, backupId); + + // TODO: convert feature will be provided in a future JIRA + boolean converted = false; + + if (manifest.getType() == BackupType.FULL || converted) { + LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from " + + (converted ? "converted" : "full") + " backup image " + tableBackupPath.toString()); + restoreTool.fullRestoreTable(tableBackupPath, sTable, tTable, converted); + + } else { // incremental Backup + throw new IOException("Unexpected backup type " + image.getType()); + } + + // The rest one are incremental + if (it.hasNext()) { + List logDirList = new ArrayList(); + while (it.hasNext()) { + BackupImage im = it.next(); + String logBackupDir = HBackupFileSystem.getLogBackupDir(im.getRootDir(), im.getBackupId()); + logDirList.add(logBackupDir); + } + String logDirs = concat(logDirList); + LOG.info("Restoring '" + sTable + "' to '" + tTable + + "' from log dirs: " + logDirs); + String[] sarr = logDirs.split(","); + Path[] paths = StringUtils.stringToPath(sarr); + restoreTool.incrementalRestoreTable(paths, new TableName[] { sTable }, + new TableName[] { tTable }); + } + LOG.info(sTable + " has been successfully restored to " + tTable); + } + + + private String concat(List logDirList) { + StringBuffer sb = new StringBuffer(); + for(int i=0; i < logDirList.size(); i++){ + sb.append(logDirList.get(i)); + if(i < logDirList.size() -1) { + sb.append(","); + } + } + return sb.toString(); + } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreUtil.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreUtil.java index 9139b6e..d9354c8 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreUtil.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreUtil.java @@ -98,7 +98,7 @@ public class RestoreUtil { */ Path getTableArchivePath(TableName tableName) throws IOException { - Path baseDir = new Path(HBackupFileSystem.getTableBackupPath(backupRootPath, tableName, + Path baseDir = new Path(HBackupFileSystem.getTableBackupPath(tableName, backupRootPath, backupId), HConstants.HFILE_ARCHIVE_DIRECTORY); Path dataDir = new Path(baseDir, HConstants.BASE_NAMESPACE_DIR); Path archivePath = new Path(dataDir, tableName.getNamespaceAsString()); @@ -140,7 +140,7 @@ public class RestoreUtil { * @param newTableNames : target tableNames(table names to be restored to) * @throws IOException exception */ - void incrementalRestoreTable(String logDir, + void incrementalRestoreTable(Path[] logDirs, TableName[] tableNames, TableName[] newTableNames) throws IOException { if (tableNames.length != newTableNames.length) { @@ -161,7 +161,7 @@ public class RestoreUtil { IncrementalRestoreService restoreService = BackupRestoreFactory.getIncrementalRestoreService(conf); - restoreService.run(logDir, tableNames, newTableNames); + restoreService.run(logDirs, tableNames, newTableNames); } } @@ -180,7 +180,7 @@ public class RestoreUtil { */ static Path getTableSnapshotPath(Path backupRootPath, TableName tableName, String backupId) { - return new Path(HBackupFileSystem.getTableBackupPath(backupRootPath, tableName, backupId), + return new Path(HBackupFileSystem.getTableBackupPath(tableName, backupRootPath, backupId), HConstants.SNAPSHOT_DIR_NAME); } @@ -221,10 +221,12 @@ public class RestoreUtil { SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, tableInfoPath); SnapshotManifest manifest = SnapshotManifest.open(conf, fs, tableInfoPath, desc); HTableDescriptor tableDescriptor = manifest.getTableDescriptor(); - if (!tableDescriptor.getNameAsString().equals(tableName)) { + if (!tableDescriptor.getNameAsString().equals(tableName.getNameAsString())) { LOG.error("couldn't find Table Desc for table: " + tableName + " under tableInfoPath: " + tableInfoPath.toString()); LOG.error("tableDescriptor.getNameAsString() = " + tableDescriptor.getNameAsString()); + throw new FileNotFoundException("couldn't find Table Desc for table: " + tableName + + " under tableInfoPath: " + tableInfoPath.toString()); } return tableDescriptor; } @@ -464,6 +466,9 @@ public class RestoreUtil { this.conf.setInt("hbase.rpc.timeout", resultMillis); } + // By default, it is 32 and loader will fail if # of files in any region exceed this + // limit. Bad for snapshot restore. + this.conf.setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE); LoadIncrementalHFiles loader = null; try { loader = new LoadIncrementalHFiles(this.conf); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java index 14235ce..2f3f1ba 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.backup.mapreduce; +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -38,7 +40,9 @@ import org.apache.hadoop.hbase.backup.impl.BackupUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.snapshot.ExportSnapshot; +import org.apache.hadoop.mapreduce.Cluster; import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobID; import org.apache.hadoop.tools.DistCp; import org.apache.hadoop.tools.DistCpConstants; import org.apache.hadoop.tools.DistCpOptions; @@ -294,4 +298,26 @@ public class MapReduceBackupCopyService implements BackupCopyService { } } + @Override + public void cancelCopyJob(byte[] jobId) throws IOException { + JobID id = new JobID(); + id.readFields(new DataInputStream(new ByteArrayInputStream(jobId))); + Cluster cluster = new Cluster(getConf()); + try { + Job job = cluster.getJob(id); + if (job == null) { + LOG.error("No job found for " + id); + // should we throw exception + } + if (job.isComplete() || job.isRetired()) { + return; + } + + job.killJob(); + LOG.debug("Killed job " + id); + } catch (InterruptedException e) { + throw new IOException(e); + } + } + } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java index 203c9a3..03547da 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java @@ -18,16 +18,22 @@ package org.apache.hadoop.hbase.backup.mapreduce; import java.io.IOException; +import java.util.ArrayList; +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.TableName; -import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.impl.BackupException; +import org.apache.hadoop.hbase.backup.impl.BackupManifest; import org.apache.hadoop.hbase.backup.impl.BackupUtil; import org.apache.hadoop.hbase.backup.impl.IncrementalRestoreService; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles; import org.apache.hadoop.hbase.mapreduce.WALPlayer; @InterfaceAudience.Private @@ -42,23 +48,142 @@ public class MapReduceRestoreService implements IncrementalRestoreService { } @Override - public void run(String logDir, TableName[] tableNames, TableName[] newTableNames) throws IOException { - String tableStr = BackupUtil.join(tableNames); - String newTableStr = BackupUtil.join(newTableNames); + public void run(Path[] logDirPaths, TableName[] tableNames, TableName[] newTableNames) + throws IOException { + logDirPaths = getAllWithRelativePaths(logDirPaths); // WALPlayer reads all files in arbitrary directory structure and creates a Map task for each // log file + String logDirs = concat(logDirPaths); + LOG.info("Restore incremental backup from directory " + logDirs + " from hbase tables " + + BackupUtil.join(tableNames) + " to tables " + BackupUtil.join(newTableNames)); - String[] playerArgs = { logDir, tableStr, newTableStr }; - LOG.info("Restore incremental backup from directory " + logDir + " from hbase tables " - + BackupUtil.join(tableNames) + " to tables " - + BackupUtil.join(newTableNames)); + for (int i = 0; i < tableNames.length; i++) { + + LOG.info("Restore "+ tableNames[i] + " into "+ newTableNames[i]); + + Path bulkOutputPath = getBulkOutputDir(newTableNames[i].getNameAsString()); + String[] playerArgs = + { logDirs, tableNames[i].getNameAsString(), newTableNames[i].getNameAsString()/*, + "-Dwal.bulk.output=" + bulkOutputPath.toString() */}; + + int result = 0; + int loaderResult = 0; + try { + Configuration conf = getConf(); + conf.set("wal.bulk.output", bulkOutputPath.toString()); + player.setConf(getConf()); + result = player.run(playerArgs); + if (succeeded(result)) { + // do bulk load + LoadIncrementalHFiles loader = createLoader(); + if (LOG.isDebugEnabled()) { + LOG.debug("Restoring HFiles from directory " + bulkOutputPath); + } + String[] args = { bulkOutputPath.toString(), newTableNames[i].getNameAsString() }; + loaderResult = loader.run(args); + if(failed(loaderResult)) { + throw new IOException("Can not restore from backup directory " + logDirs + + " (check Hadoop and HBase logs). Bulk loader return code =" + loaderResult); + } + } else { + throw new IOException("Can not restore from backup directory " + logDirs + + " (check Hadoop/MR and HBase logs). WALPlayer return code =" + result); + } + LOG.debug("Restore Job finished:" + result); + } catch (Exception e) { + throw new IOException("Can not restore from backup directory " + logDirs + + " (check Hadoop and HBase logs) ", e); + } + + } + } + + private Path[] getAllWithRelativePaths(Path[] logDirPaths) throws BackupException { + List list = new ArrayList(); + for(Path p: logDirPaths){ + list.addAll(getWithRelativePaths(p)); + } + Path[] arr = new Path[list.size()]; + list.toArray(arr); + return arr; + } + + private List getWithRelativePaths(Path p) throws BackupException { + + // We have to load manifest + List list = new ArrayList(); + list.add(p); + Path backupRootPathWAL = getBackupRootPath(p); + BackupManifest manifest = new BackupManifest(getConf(), p); + List relRefs = manifest.getRelativeWALReferences(); + if(relRefs != null && relRefs.size() > 0){ + for(String ref: relRefs){ + String[] split = ref.split(Path.SEPARATOR); + String backupId = split[0]; + String walFileName = split[1]; + Path walPath = new Path(backupRootPathWAL, backupId + Path.SEPARATOR + walFileName); + list.add(walPath); + } + } + return list; + } + + private Path getBackupRootPath(Path p) { + return p.getParent().getParent(); + } + + private boolean failed(int result) { + return result != 0; + } + + private boolean succeeded(int result) { + return result == 0; + } + + private LoadIncrementalHFiles createLoader() + throws IOException { + // set configuration for restore: + // LoadIncrementalHFile needs more time + // hbase.rpc.timeout 600000 + // calculates + Integer milliSecInHour = 3600000; + Configuration conf = new Configuration(getConf()); + conf.setInt("hbase.rpc.timeout", milliSecInHour); + + // By default, it is 32 and loader will fail if # of files in any region exceed this + // limit. Bad for snapshot restore. + conf.setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE); + conf.setBoolean("hbase.loadincremental.validate.hfile", false); + LoadIncrementalHFiles loader = null; try { - player.run(playerArgs); + loader = new LoadIncrementalHFiles(conf); } catch (Exception e) { - throw new IOException("cannot restore from backup directory " + logDir - + " (check Hadoop and HBase logs) " + e); + throw new IOException(e); + } + return loader; + } + + private Path getBulkOutputDir(String tableName) throws IOException + { + Configuration conf = getConf(); + FileSystem fs = FileSystem.get(conf); + String tmp = conf.get("hbase.tmp.dir"); + Path path = new Path(tmp + Path.SEPARATOR + "bulk_output-"+tableName + "-" + + System.currentTimeMillis()); + fs.deleteOnExit(path); + return path; + } + + private String concat(Path[] logDirPaths) { + StringBuffer sb = new StringBuffer(); + for(int i=0; i < logDirPaths.length ; i++){ + sb.append(logDirPaths[i]); + if(i < logDirPaths.length -1){ + sb.append(","); + } } + return sb.toString(); } @Override diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java index dae24a6..898881b 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java @@ -18,9 +18,6 @@ */ package org.apache.hadoop.hbase.backup.master; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; - import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -34,8 +31,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate; /** @@ -64,11 +59,12 @@ public class BackupLogCleaner extends BaseLogCleanerDelegate { // TODO: LogCleaners do not have a way to get the Connection from Master. We should find a // way to pass it down here, so that this connection is not re-created every time. // It is expensive - try(Connection connection = ConnectionFactory.createConnection(this.getConf()); - final BackupSystemTable table = new BackupSystemTable(connection)) { + try (final BackupSystemTable table = new BackupSystemTable(this.getConf())) { // If we do not have recorded backup sessions if (!table.hasBackupSessions()) { + LOG.debug("BackupLogCleaner has no backup sessions"); + return files; } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java index d524140..f08e976 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java @@ -75,7 +75,7 @@ public class LogRollBackupSubprocedure extends Subprocedure { LOG.info("After roll log in backup subprocedure, current log number: " + hlog.getFilenum()); Connection connection = rss.getConnection(); - try(final BackupSystemTable table = new BackupSystemTable(connection)) { + try(final BackupSystemTable table = new BackupSystemTable(connection.getConfiguration())) { // sanity check, good for testing HashMap serverTimestampMap = table.readRegionServerLastLogRollResult(); String host = rss.getServerName().getHostname(); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java index 02fcbba..736b8a5 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java @@ -27,22 +27,24 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; -import org.apache.hadoop.hbase.wal.WALFactory; -import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.wal.WAL.Reader; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.util.StringUtils; /** * Simple {@link InputFormat} for {@link org.apache.hadoop.hbase.wal.WAL} files. @@ -231,21 +233,31 @@ public class WALInputFormat extends InputFormat { List getSplits(final JobContext context, final String startKey, final String endKey) throws IOException, InterruptedException { Configuration conf = context.getConfiguration(); - Path inputDir = new Path(conf.get("mapreduce.input.fileinputformat.inputdir")); + + Path[] inputPaths = getInputPaths(conf); long startTime = conf.getLong(startKey, Long.MIN_VALUE); long endTime = conf.getLong(endKey, Long.MAX_VALUE); - FileSystem fs = inputDir.getFileSystem(conf); - List files = getFiles(fs, inputDir, startTime, endTime); - - List splits = new ArrayList(files.size()); - for (FileStatus file : files) { + FileSystem fs = FileSystem.get(conf); + + List allFiles = new ArrayList(); + for(Path inputPath: inputPaths){ + List files = getFiles(fs, inputPath, startTime, endTime); + allFiles.addAll(files); + } + List splits = new ArrayList(allFiles.size()); + for (FileStatus file : allFiles) { splits.add(new WALSplit(file.getPath().toString(), file.getLen(), startTime, endTime)); } return splits; } + private Path[] getInputPaths(Configuration conf) { + String inpDirs = conf.get("mapreduce.input.fileinputformat.inputdir"); + return StringUtils.stringToPath(inpDirs.split(",")); + } + private List getFiles(FileSystem fs, Path dir, long startTime, long endTime) throws IOException { List result = new ArrayList(); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java index 2ceeda5..516dca1 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java @@ -47,6 +47,8 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.mapreduce.Counter; +import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; @@ -139,7 +141,8 @@ public class WALPlayer extends Configured implements Tool { protected static class WALMapper extends Mapper { private Map tables = new TreeMap(); - + + @Override public void map(WALKey key, WALEdit value, Context context) throws IOException { @@ -152,6 +155,7 @@ public class WALPlayer extends Configured implements Tool { Put put = null; Delete del = null; Cell lastCell = null; + for (Cell cell : value.getCells()) { // filtering WAL meta entries if (WALEdit.isMetaEditFamily(cell)) { @@ -210,6 +214,13 @@ public class WALPlayer extends Configured implements Tool { } @Override + protected void + cleanup(Mapper.Context context) + throws IOException, InterruptedException { + super.cleanup(context); + } + + @Override public void setup(Context context) throws IOException { String[] tableMap = context.getConfiguration().getStrings(TABLE_MAP_KEY); String[] tablesToUse = context.getConfiguration().getStrings(TABLES_KEY); @@ -261,7 +272,7 @@ public class WALPlayer extends Configured implements Tool { Configuration conf = getConf(); setupTime(conf, HLogInputFormat.START_TIME_KEY); setupTime(conf, HLogInputFormat.END_TIME_KEY); - Path inputDir = new Path(args[0]); + String inputDirs = args[0]; String[] tables = args[1].split(","); String[] tableMap; if (args.length > 2) { @@ -275,13 +286,18 @@ public class WALPlayer extends Configured implements Tool { } conf.setStrings(TABLES_KEY, tables); conf.setStrings(TABLE_MAP_KEY, tableMap); - Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + inputDir)); + Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + System.currentTimeMillis())); job.setJarByClass(WALPlayer.class); - FileInputFormat.setInputPaths(job, inputDir); + + FileInputFormat.addInputPaths(job, inputDirs); + job.setInputFormatClass(WALInputFormat.class); job.setMapOutputKeyClass(ImmutableBytesWritable.class); + String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY); if (hfileOutPath != null) { + LOG.debug("add incremental job :"+hfileOutPath); + // the bulk HFile case if (tables.length != 1) { throw new IOException("Exactly one table must be specified for the bulk export option"); @@ -297,6 +313,8 @@ public class WALPlayer extends Configured implements Tool { RegionLocator regionLocator = conn.getRegionLocator(tableName)) { HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator); } + LOG.debug("success configuring load incremental job"); + TableMapReduceUtil.addDependencyJars(job.getConfiguration(), com.google.common.base.Preconditions.class); } else { @@ -311,6 +329,7 @@ public class WALPlayer extends Configured implements Tool { return job; } + /** * Print usage * @param errorMsg Error message. Can be null. @@ -360,6 +379,7 @@ public class WALPlayer extends Configured implements Tool { System.exit(-1); } Job job = createSubmittableJob(args); - return job.waitForCompletion(true) ? 0 : 1; + int result =job.waitForCompletion(true) ? 0 : 1; + return result; } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index eefc8ee..fed9a69 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -2630,7 +2630,7 @@ public class HMaster extends HRegionServer implements MasterServices { String backupId = BackupRestoreConstants.BACKUPID_PREFIX + EnvironmentEdgeManager.currentTime(); if (type == BackupType.INCREMENTAL) { Set incrTableSet = - BackupSystemTable.getIncrementalBackupTableSet(clusterConnection); + BackupSystemTable.getIncrementalBackupTableSet(); if (incrTableSet.isEmpty()) { LOG.warn("Incremental backup table set contains no table.\n" + "Use 'backup create full' or 'backup stop' to \n " diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java index ba1081f..5c94e50 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java @@ -18,6 +18,8 @@ */ package org.apache.hadoop.hbase.backup; +import static org.apache.hadoop.hbase.wal.WALFactory.WAL_PROVIDER; + import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -31,9 +33,10 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.backup.impl.BackupHandler.BackupState; import org.apache.hadoop.hbase.backup.impl.BackupContext; +import org.apache.hadoop.hbase.backup.impl.BackupHandler.BackupState; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.master.BackupController; import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; import org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager; import org.apache.hadoop.hbase.client.Connection; @@ -61,17 +64,17 @@ public class TestBackupBase { protected static HBaseTestingUtility TEST_UTIL; protected static HBaseTestingUtility TEST_UTIL2; - protected static TableName table1; - protected static TableName table2; - protected static TableName table3; - protected static TableName table4; + protected static TableName table1 = TableName.valueOf("table1"); + protected static TableName table2 = TableName.valueOf("table2"); + protected static TableName table3 = TableName.valueOf("table3"); + protected static TableName table4 = TableName.valueOf("table4"); protected static TableName table1_restore = TableName.valueOf("table1_restore"); protected static TableName table2_restore = TableName.valueOf("table2_restore"); protected static TableName table3_restore = TableName.valueOf("table3_restore"); protected static TableName table4_restore = TableName.valueOf("table4_restore"); - protected static final int NB_ROWS_IN_BATCH = 100; + protected static final int NB_ROWS_IN_BATCH = 999; protected static final byte[] qualName = Bytes.toBytes("q1"); protected static final byte[] famName = Bytes.toBytes("f"); @@ -88,15 +91,20 @@ public class TestBackupBase { @BeforeClass public static void setUpBeforeClass() throws Exception { TEST_UTIL = new HBaseTestingUtility(); - TEST_UTIL.getConfiguration().set("hbase.procedure.regionserver.classes", + conf1 = TEST_UTIL.getConfiguration(); + conf1.set("hbase.procedure.regionserver.classes", LogRollRegionServerProcedureManager.class.getName()); - TEST_UTIL.getConfiguration().set("hbase.procedure.master.classes", + conf1.set("hbase.procedure.master.classes", LogRollMasterProcedureManager.class.getName()); - TEST_UTIL.getConfiguration().set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1"); + // Set Master Observer - Backup Controller + conf1.set("hbase.coprocessor.master.classes", + BackupController.class.getName()); + conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1"); + // Set MultiWAL (with 2 default WAL files per RS) + //conf1.set(WAL_PROVIDER, "multiwal"); TEST_UTIL.startMiniZKCluster(); MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster(); - conf1 = TEST_UTIL.getConfiguration(); conf2 = HBaseConfiguration.create(conf1); conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2"); TEST_UTIL2 = new HBaseTestingUtility(conf2); @@ -150,6 +158,10 @@ public class TestBackupBase { return backupTables(BackupType.FULL, tables, BACKUP_ROOT_DIR); } + protected String incrementalTableBackup(List tables) throws IOException { + return backupTables(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + } + protected static void loadTable(HTable table) throws Exception { Put p; // 100 + 1 row to t1_syncup @@ -164,7 +176,7 @@ public class TestBackupBase { long tid = System.currentTimeMillis(); table1 = TableName.valueOf("test-" + tid); - BackupSystemTable backupTable = new BackupSystemTable(TEST_UTIL.getConnection()); + //BackupSystemTable backupTable = new BackupSystemTable(TEST_UTIL.getConfiguration()); HBaseAdmin ha = TEST_UTIL.getHBaseAdmin(); HTableDescriptor desc = new HTableDescriptor(table1); HColumnDescriptor fam = new HColumnDescriptor(famName); @@ -204,14 +216,17 @@ public class TestBackupBase { } private BackupContext getBackupContext(String backupId) throws IOException { - Configuration conf = conf1;//BackupClientImpl.getConf(); - try (Connection connection = ConnectionFactory.createConnection(conf); - BackupSystemTable table = new BackupSystemTable(connection)) { + Configuration conf = conf1; + try (BackupSystemTable table = new BackupSystemTable(conf)) { BackupContext status = table.readBackupStatus(backupId); return status; } } + protected BackupClient getBackupClient(){ + return BackupRestoreFactory.getBackupClient(conf1); + } + protected RestoreClient getRestoreClient() { return BackupRestoreFactory.getRestoreClient(conf1); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java new file mode 100644 index 0000000..4316fa8 --- /dev/null +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java @@ -0,0 +1,65 @@ +/** + * 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.backup; + +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupDelete extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestBackupDelete.class); + + /** + * Verify that full backup is created on a single table with data correctly. Verify that history + * works as expected + * @throws Exception + */ + @Test + public void testBackupDelete() throws Exception { + LOG.info("test backup delete on a single table with data"); + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + String[] backupIds = new String[] { backupId }; + getBackupClient().deleteBackups(backupIds); + + LOG.info("delete_backup"); + String output = baos.toString(); + LOG.info(baos.toString()); + assertTrue(output.indexOf("Delete backup for backupID=" + backupId + " completed.") >= 0); + } + +} diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java new file mode 100644 index 0000000..04fd234 --- /dev/null +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java @@ -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.backup; + +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupContext; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupDescribe extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestBackupDescribe.class); + + /** + * Verify that full backup is created on a single table with data correctly. Verify that describe + * works as expected + * @throws Exception + */ + @Test + public void testBackupDescribe() throws Exception { + + LOG.info("test backup describe on a single table with data"); + + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + getBackupClient().describeBackupImage(backupId); + String responce = baos.toString(); + assertTrue(responce.indexOf(backupId) > 0); + assertTrue(responce.indexOf("COMPLETE") > 0); + + BackupSystemTable table = new BackupSystemTable(conf1); + BackupContext status = table.readBackupStatus(backupId); + String desc = status.getShortDescription(); + table.close(); + assertTrue(responce.indexOf(desc) >= 0); + + } + +} diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java index 175bce0..09989dc 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java @@ -65,8 +65,7 @@ public class TestBackupLogCleaner extends TestBackupBase { List tableSetFullList = Lists.newArrayList(table1, table2, table3, table4); - try (Connection connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); - BackupSystemTable systemTable = new BackupSystemTable(connection)) { + try (BackupSystemTable systemTable = new BackupSystemTable(TEST_UTIL.getConfiguration())) { // Verify that we have no backup sessions yet assertFalse(systemTable.hasBackupSessions()); @@ -79,7 +78,7 @@ public class TestBackupLogCleaner extends TestBackupBase { // We can delete all files because we do not have yet recorded backup sessions assertTrue(Iterables.size(deletable) == walFiles.size()); - systemTable.addWALFiles(swalFiles, "backup"); + systemTable.addWALFiles(swalFiles, "backup", "root"); String backupIdFull = fullTableBackup(tableSetFullList); // getBackupClient().create(BackupType.FULL, tableSetFullList, BACKUP_ROOT_DIR); // Check one more time diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java new file mode 100644 index 0000000..25909bb --- /dev/null +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupShowHistory extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestBackupShowHistory.class); + + /** + * Verify that full backup is created on a single table with data correctly. Verify that history + * works as expected + * @throws Exception + */ + @Test + public void testBackupHistory() throws Exception { + + LOG.info("test backup history on a single table with data"); + + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + getBackupClient().showHistory(10); + + LOG.info("show_history"); + String output = baos.toString(); + LOG.info(baos.toString()); + assertTrue(output.indexOf(backupId) > 0); + } + +} \ No newline at end of file diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java new file mode 100644 index 0000000..8518d0b --- /dev/null +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java @@ -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.backup; + +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupStatusProgress extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestBackupStatusProgress.class); + + /** + * Verify that full backup is created on a single table with data correctly. + * @throws Exception + */ + @Test + public void testBackupStatusProgress() throws Exception { + + LOG.info("test backup status/progress on a single table with data"); + + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + getBackupClient().describeBackupImage(backupId); + String responce = baos.toString(); + assertTrue(responce.indexOf(backupId) > 0); + assertTrue(responce.indexOf("COMPLETE") > 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + getBackupClient().showProgress(backupId); + responce = baos.toString(); + LOG.info(responce); + assertTrue(responce.indexOf(backupId) > 0); + assertTrue(responce.indexOf("COMPLETE") > 0); + + } + +} diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java index bf902e9..0d716c5 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java @@ -42,9 +42,9 @@ import org.apache.hadoop.hbase.backup.impl.BackupContext; import org.apache.hadoop.hbase.backup.impl.BackupHandler.BackupState; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; import org.apache.hadoop.hbase.backup.impl.BackupUtil.BackupCompleteData; +import org.apache.hadoop.hbase.backup.master.BackupController; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.After; import org.junit.AfterClass; @@ -68,13 +68,15 @@ public class TestBackupSystemTable { @BeforeClass public static void setUp() throws Exception { - cluster = UTIL.startMiniCluster(); - conn = ConnectionFactory.createConnection(UTIL.getConfiguration()); + UTIL.getConfiguration().set("hbase.coprocessor.master.classes", + BackupController.class.getName()); + cluster = UTIL.startMiniCluster(); } @Before public void before() throws IOException { - table = new BackupSystemTable(conn); + table = new BackupSystemTable(conf); + //conn = UTIL.getConnection(); } @After @@ -82,6 +84,7 @@ public class TestBackupSystemTable { if (table != null) { table.close(); } + } @Test @@ -148,6 +151,43 @@ public class TestBackupSystemTable { } @Test + public void testBackupDelete() throws IOException { + + try (BackupSystemTable table = new BackupSystemTable(conf)) { + + int n = 10; + List list = createBackupContextList(n); + + // Load data + for (BackupContext bc : list) { + // Make sure we set right status + bc.setState(BackupState.COMPLETE); + table.updateBackupStatus(bc); + } + + // Verify exists + for (BackupContext bc : list) { + assertNotNull(table.readBackupStatus(bc.getBackupId())); + } + + // Delete all + for (BackupContext bc : list) { + table.deleteBackupStatus(bc.getBackupId()); + } + + // Verify do not exists + for (BackupContext bc : list) { + assertNull(table.readBackupStatus(bc.getBackupId())); + } + + cleanBackupTable(); + } + + } + + + + @Test public void testRegionServerLastLogRollResults() throws IOException { String[] servers = new String[] { "server1", "server2", "server3" }; Long[] timestamps = new Long[] { 100L, 102L, 107L }; @@ -189,7 +229,7 @@ public class TestBackupSystemTable { table.addIncrementalBackupTableSet(tables1); TreeSet res1 = (TreeSet) BackupSystemTable - .getIncrementalBackupTableSet(conn); + .getIncrementalBackupTableSet(); assertTrue(tables1.size() == res1.size()); Iterator desc1 = tables1.descendingIterator(); Iterator desc2 = res1.descendingIterator(); @@ -199,7 +239,7 @@ public class TestBackupSystemTable { table.addIncrementalBackupTableSet(tables2); TreeSet res2 = (TreeSet) BackupSystemTable - .getIncrementalBackupTableSet(conn); + .getIncrementalBackupTableSet(); assertTrue((tables2.size() + tables1.size() - 1) == res2.size()); tables1.addAll(tables2); @@ -294,7 +334,7 @@ public class TestBackupSystemTable { "hdfs://server/WALs/srv3,103,17777/srv3,103,17777.default.3"); String newFile = "hdfs://server/WALs/srv1,101,15555/srv1,101,15555.default.5"; - table.addWALFiles(files, "backup"); + table.addWALFiles(files, "backup", "root"); assertTrue(table.checkWALFile(files.get(0))); assertTrue(table.checkWALFile(files.get(1))); @@ -304,9 +344,148 @@ public class TestBackupSystemTable { cleanBackupTable(); } + + /** + * Backup set tests + */ + + @Test + public void testBackupSetAddNotExists() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conf)) { + + String[] tables = new String[] { "table1", "table2", "table3" }; + String setName = "name"; + table.backupSetAdd(setName, tables); + String[] tnames = table.backupSetDescribe(setName); + assertTrue(tnames != null); + assertTrue(tnames.length == tables.length); + for (int i = 0; i < tnames.length; i++) { + assertTrue(tnames[i].equals(tables[i])); + } + + cleanBackupTable(); + } + + } + + @Test + public void testBackupSetAddExists() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conf)) { + + String[] tables = new String[] { "table1", "table2", "table3" }; + String setName = "name"; + table.backupSetAdd(setName, tables); + String[] addTables = new String[] { "table4", "table5", "table6" }; + table.backupSetAdd(setName, addTables); + + String[] tnames = table.backupSetDescribe(setName); + assertTrue(tnames != null); + assertTrue(tnames.length == tables.length + addTables.length); + for (int i = 0; i < tnames.length; i++) { + assertTrue(tnames[i].equals("table" + (i + 1))); + } + cleanBackupTable(); + } + } + + @Test + public void testBackupSetAddExistsIntersects() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conf)) { + + String[] tables = new String[] { "table1", "table2", "table3" }; + String setName = "name"; + table.backupSetAdd(setName, tables); + String[] addTables = new String[] { "table3", "table4", "table5", "table6" }; + table.backupSetAdd(setName, addTables); + + String[] tnames = table.backupSetDescribe(setName); + assertTrue(tnames != null); + assertTrue(tnames.length == tables.length + addTables.length - 1); + for (int i = 0; i < tnames.length; i++) { + assertTrue(tnames[i].equals("table" + (i + 1))); + } + cleanBackupTable(); + } + } + + @Test + public void testBackupSetRemoveSomeNotExists() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conf)) { + + String[] tables = new String[] { "table1", "table2", "table3", "table4" }; + String setName = "name"; + table.backupSetAdd(setName, tables); + String[] removeTables = new String[] { "table4", "table5", "table6" }; + table.backupSetRemove(setName, removeTables); + + String[] tnames = table.backupSetDescribe(setName); + assertTrue(tnames != null); + assertTrue(tnames.length == tables.length - 1); + for (int i = 0; i < tnames.length; i++) { + assertTrue(tnames[i].equals("table" + (i + 1))); + } + cleanBackupTable(); + } + } + + @Test + public void testBackupSetRemove() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conf)) { + + String[] tables = new String[] { "table1", "table2", "table3", "table4" }; + String setName = "name"; + table.backupSetAdd(setName, tables); + String[] removeTables = new String[] { "table4", "table3" }; + table.backupSetRemove(setName, removeTables); + + String[] tnames = table.backupSetDescribe(setName); + assertTrue(tnames != null); + assertTrue(tnames.length == tables.length - 2); + for (int i = 0; i < tnames.length; i++) { + assertTrue(tnames[i].equals("table" + (i + 1))); + } + cleanBackupTable(); + } + } + + @Test + public void testBackupSetDelete() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conf)) { + + String[] tables = new String[] { "table1", "table2", "table3", "table4" }; + String setName = "name"; + table.backupSetAdd(setName, tables); + table.backupSetDelete(setName); + + String[] tnames = table.backupSetDescribe(setName); + assertTrue(tnames == null); + cleanBackupTable(); + } + } + + @Test + public void testBackupSetList() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conf)) { + + String[] tables = new String[] { "table1", "table2", "table3", "table4" }; + String setName1 = "name1"; + String setName2 = "name2"; + table.backupSetAdd(setName1, tables); + table.backupSetAdd(setName2, tables); + + List list = table.backupSetList(); + + assertTrue(list.size() == 2); + assertTrue(list.get(0).equals(setName1)); + assertTrue(list.get(1).equals(setName2)); + + cleanBackupTable(); + } + } + private boolean compare(BackupContext ctx, BackupCompleteData data) { - return ctx.getBackupId().equals(data.getBackupToken()) + return ctx.getBackupId().equals(data.getBackupId()) && ctx.getTargetRootDir().equals(data.getBackupRootPath()) && ctx.getType().toString().equals(data.getType()) && ctx.getStartTs() == Long.parseLong(data.getStartTime()) diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java new file mode 100644 index 0000000..e62ef73 --- /dev/null +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java @@ -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.backup; + +import static org.junit.Assert.*; + +import java.util.ArrayList; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.impl.BackupUtil.BackupCompleteData; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(LargeTests.class) +public class TestFullBackupSet extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestFullBackupSet.class); + + + /** + * Verify that full backup is created on a single table with data correctly. + * @throws Exception + */ + @Test + public void testFullBackupSetExist() throws Exception { + + LOG.info("TFBSE test full backup, backup set exists"); + + //Create set + BackupSystemTable table = new BackupSystemTable(conf1); + String name = "name"; + table.backupSetAdd(name, new String[]{ table1.getNameAsString()}); + String[] names = table.backupSetDescribe(name); + + assertNotNull(names); + assertTrue(names.length == 1); + assertTrue(names[0].equals( table1.getNameAsString())); + + String[] args = new String[]{"create", "full", BACKUP_ROOT_DIR, "-set", name }; + // Run backup + BackupDriver.setStaticConf(conf1); + BackupDriver.main(args); + + ArrayList backups = table.getBackupHistory(); + System.out.println("DDEBUG backups.size="+ backups.size()); + assertTrue(backups.size() == 1); + String backupId = backups.get(0).getBackupId(); + assertTrue(checkSucceeded(backupId)); + LOG.info("TFBSE backup complete"); + + } + + @Test + public void testFullBackupSetDoesNotExist() throws Exception { + + LOG.info("TFBSE test full backup, backup set does not exist"); + String name = "name1"; + String[] args = new String[]{"create", "full", BACKUP_ROOT_DIR, "-set", name }; + // Run backup + try{ + BackupDriver.setStaticConf(conf1); + BackupDriver.main(args); + assertTrue(false); + } catch(Exception e){ + LOG.info("TFBSE Expected exception ", e); + } + + } + +} \ No newline at end of file diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java index 63e864c..6d7b252 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; @@ -43,7 +44,7 @@ import com.google.common.collect.Lists; @Category(LargeTests.class) public class TestIncrementalBackup extends TestBackupBase { private static final Log LOG = LogFactory.getLog(TestIncrementalBackup.class); - //implement all testcases in 1 test since incremental backup/restore has dependencies + //implement all test cases in 1 test since incremental backup/restore has dependencies @Test public void TestIncBackupRestore() throws Exception { // #1 - create full backup for all tables diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupNoDataLoss.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupNoDataLoss.java new file mode 100644 index 0000000..3fca9c3 --- /dev/null +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupNoDataLoss.java @@ -0,0 +1,131 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertTrue; + +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestIncrementalBackupNoDataLoss extends TestBackupBase { + private static final Log LOG = LogFactory.getLog(TestIncrementalBackupNoDataLoss.class); + + // implement all test cases in 1 test since incremental backup/restore has dependencies + @Test + public void TestIncBackupRestore() throws Exception { + HBackupFileSystem hbfs; + + // #1 - create full backup for all tables + LOG.info("create full backup image for all tables"); + + List tables = Lists.newArrayList(table1, table2); + + String backupIdFull = fullTableBackup(tables); + + assertTrue(checkSucceeded(backupIdFull)); + + Connection conn = ConnectionFactory.createConnection(conf1); + // #2 - insert some data to table + HTable t1 = (HTable) conn.getTable(table1); + Put p1; + for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { + p1 = new Put(Bytes.toBytes("row-t1" + i)); + p1.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + t1.put(p1); + } + + Assert.assertThat(TEST_UTIL.countRows(t1), CoreMatchers.equalTo(NB_ROWS_IN_BATCH * 2)); + t1.close(); + + HTable t2 = (HTable) conn.getTable(table2); + Put p2; + for (int i = 0; i < 5; i++) { + p2 = new Put(Bytes.toBytes("row-t2" + i)); + p2.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + t2.put(p2); + } + + Assert.assertThat(TEST_UTIL.countRows(t2), CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5)); + t2.close(); + + // #3 - incremental backup for table1 + + tables = Lists.newArrayList(table1); + String backupIdInc1 = incrementalTableBackup(tables); + assertTrue(checkSucceeded(backupIdInc1)); + + // #4 - incremental backup for table2 + + tables = Lists.newArrayList(table2); + String backupIdInc2 = incrementalTableBackup(tables); + assertTrue(checkSucceeded(backupIdInc2)); + // #5 - restore incremental backup for table1 + TableName[] tablesRestoreInc1 = new TableName[] { table1 }; + TableName[] tablesMapInc1 = new TableName[] { table1_restore }; + + if (TEST_UTIL.getAdmin().tableExists(table1_restore)) { + TEST_UTIL.deleteTable(table1_restore); + } + if (TEST_UTIL.getAdmin().tableExists(table2_restore)) { + TEST_UTIL.deleteTable(table2_restore); + } + + //hbfs = new HBackupFileSystem(conf1, new Path(BACKUP_ROOT_DIR), backupIdInc1); + RestoreClient client = getRestoreClient(); + client.restore( BACKUP_ROOT_DIR, backupIdInc1, false, true, tablesRestoreInc1, + tablesMapInc1, false); + + HTable hTable = (HTable) conn.getTable(table1_restore); + Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH * 2)); + hTable.close(); + + // #5 - restore incremental backup for table2 + + TableName[] tablesRestoreInc2 = new TableName[] { table2 }; + TableName[] tablesMapInc2 = new TableName[] { table2_restore }; + + client = getRestoreClient(); + client.restore(BACKUP_ROOT_DIR, backupIdInc2, false, true, tablesRestoreInc2, + tablesMapInc2, false); + + hTable = (HTable) conn.getTable(table2_restore); + Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5)); + hTable.close(); + + conn.close(); + } + +}