diff --git a/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java b/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java index 7235e34de2c..0b783ced050 100644 --- a/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java +++ b/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java @@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.SoftValueSortedMap; import org.apache.hadoop.hbase.util.Writables; +import org.apache.hadoop.hbase.zookeeper.ClusterId; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; import org.apache.hadoop.hbase.zookeeper.RootRegionTracker; import org.apache.hadoop.hbase.zookeeper.ZKTable; @@ -608,8 +609,7 @@ public class HConnectionManager { ZooKeeperKeepAliveConnection zkw = null; try { zkw = getKeepAliveZooKeeperWatcher(); - this.clusterId = Bytes.toString( - ZKUtil.getData(zkw, zkw.clusterIdZNode)); + this.clusterId = ClusterId.readClusterIdZNode(zkw); if (clusterId == null) { LOG.info("ClusterId read in ZooKeeper is null"); } diff --git a/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java b/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java index 7304a9e14ce..8457bdc39f0 100644 --- a/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java +++ b/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java @@ -954,6 +954,838 @@ public final class ZooKeeperProtos { // @@protoc_insertion_point(class_scope:Master) } + public interface ClusterIdOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string clusterId = 1; + boolean hasClusterId(); + String getClusterId(); + } + public static final class ClusterId extends + com.google.protobuf.GeneratedMessage + implements ClusterIdOrBuilder { + // Use ClusterId.newBuilder() to construct. + private ClusterId(Builder builder) { + super(builder); + } + private ClusterId(boolean noInit) {} + + private static final ClusterId defaultInstance; + public static ClusterId getDefaultInstance() { + return defaultInstance; + } + + public ClusterId getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_ClusterId_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_ClusterId_fieldAccessorTable; + } + + private int bitField0_; + // required string clusterId = 1; + public static final int CLUSTERID_FIELD_NUMBER = 1; + private java.lang.Object clusterId_; + public boolean hasClusterId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getClusterId() { + java.lang.Object ref = clusterId_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + clusterId_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getClusterIdBytes() { + java.lang.Object ref = clusterId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + clusterId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private void initFields() { + clusterId_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasClusterId()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getClusterIdBytes()); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, getClusterIdBytes()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId) obj; + + boolean result = true; + result = result && (hasClusterId() == other.hasClusterId()); + if (hasClusterId()) { + result = result && getClusterId() + .equals(other.getClusterId()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasClusterId()) { + hash = (37 * hash) + CLUSTERID_FIELD_NUMBER; + hash = (53 * hash) + getClusterId().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterIdOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_ClusterId_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_ClusterId_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + clusterId_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId build() { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.clusterId_ = clusterId_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId.getDefaultInstance()) return this; + if (other.hasClusterId()) { + setClusterId(other.getClusterId()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasClusterId()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + clusterId_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string clusterId = 1; + private java.lang.Object clusterId_ = ""; + public boolean hasClusterId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getClusterId() { + java.lang.Object ref = clusterId_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + clusterId_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setClusterId(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + clusterId_ = value; + onChanged(); + return this; + } + public Builder clearClusterId() { + bitField0_ = (bitField0_ & ~0x00000001); + clusterId_ = getDefaultInstance().getClusterId(); + onChanged(); + return this; + } + void setClusterId(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + clusterId_ = value; + onChanged(); + } + + // @@protoc_insertion_point(builder_scope:ClusterId) + } + + static { + defaultInstance = new ClusterId(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:ClusterId) + } + + public interface ClusterUpOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string startDate = 1; + boolean hasStartDate(); + String getStartDate(); + } + public static final class ClusterUp extends + com.google.protobuf.GeneratedMessage + implements ClusterUpOrBuilder { + // Use ClusterUp.newBuilder() to construct. + private ClusterUp(Builder builder) { + super(builder); + } + private ClusterUp(boolean noInit) {} + + private static final ClusterUp defaultInstance; + public static ClusterUp getDefaultInstance() { + return defaultInstance; + } + + public ClusterUp getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_ClusterUp_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_ClusterUp_fieldAccessorTable; + } + + private int bitField0_; + // required string startDate = 1; + public static final int STARTDATE_FIELD_NUMBER = 1; + private java.lang.Object startDate_; + public boolean hasStartDate() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getStartDate() { + java.lang.Object ref = startDate_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + startDate_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getStartDateBytes() { + java.lang.Object ref = startDate_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + startDate_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private void initFields() { + startDate_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasStartDate()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getStartDateBytes()); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, getStartDateBytes()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp) obj; + + boolean result = true; + result = result && (hasStartDate() == other.hasStartDate()); + if (hasStartDate()) { + result = result && getStartDate() + .equals(other.getStartDate()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasStartDate()) { + hash = (37 * hash) + STARTDATE_FIELD_NUMBER; + hash = (53 * hash) + getStartDate().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUpOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_ClusterUp_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_ClusterUp_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + startDate_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp build() { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.startDate_ = startDate_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp.getDefaultInstance()) return this; + if (other.hasStartDate()) { + setStartDate(other.getStartDate()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasStartDate()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + startDate_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string startDate = 1; + private java.lang.Object startDate_ = ""; + public boolean hasStartDate() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getStartDate() { + java.lang.Object ref = startDate_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + startDate_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setStartDate(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + startDate_ = value; + onChanged(); + return this; + } + public Builder clearStartDate() { + bitField0_ = (bitField0_ & ~0x00000001); + startDate_ = getDefaultInstance().getStartDate(); + onChanged(); + return this; + } + void setStartDate(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + startDate_ = value; + onChanged(); + } + + // @@protoc_insertion_point(builder_scope:ClusterUp) + } + + static { + defaultInstance = new ClusterUp(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:ClusterUp) + } + private static com.google.protobuf.Descriptors.Descriptor internal_static_RootRegionServer_descriptor; private static @@ -964,6 +1796,16 @@ public final class ZooKeeperProtos { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_Master_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_ClusterId_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_ClusterId_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_ClusterUp_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_ClusterUp_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -975,9 +1817,11 @@ public final class ZooKeeperProtos { java.lang.String[] descriptorData = { "\n\017ZooKeeper.proto\032\013hbase.proto\"/\n\020RootRe" + "gionServer\022\033\n\006server\030\001 \002(\0132\013.ServerName\"" + - "%\n\006Master\022\033\n\006master\030\001 \002(\0132\013.ServerNameBE" + - "\n*org.apache.hadoop.hbase.protobuf.gener" + - "atedB\017ZooKeeperProtosH\001\210\001\001\240\001\001" + "%\n\006Master\022\033\n\006master\030\001 \002(\0132\013.ServerName\"\036" + + "\n\tClusterId\022\021\n\tclusterId\030\001 \002(\t\"\036\n\tCluste" + + "rUp\022\021\n\tstartDate\030\001 \002(\tBE\n*org.apache.had" + + "oop.hbase.protobuf.generatedB\017ZooKeeperP" + + "rotosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -1000,6 +1844,22 @@ public final class ZooKeeperProtos { new java.lang.String[] { "Master", }, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master.Builder.class); + internal_static_ClusterId_descriptor = + getDescriptor().getMessageTypes().get(2); + internal_static_ClusterId_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_ClusterId_descriptor, + new java.lang.String[] { "ClusterId", }, + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId.class, + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterId.Builder.class); + internal_static_ClusterUp_descriptor = + getDescriptor().getMessageTypes().get(3); + internal_static_ClusterUp_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_ClusterUp_descriptor, + new java.lang.String[] { "StartDate", }, + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp.class, + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ClusterUp.Builder.class); return null; } }; diff --git a/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterId.java b/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterId.java index 3fa83e61ab8..f804810d608 100644 --- a/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterId.java +++ b/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterId.java @@ -22,9 +22,12 @@ package org.apache.hadoop.hbase.zookeeper; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.zookeeper.KeeperException; +import com.google.protobuf.InvalidProtocolBufferException; + /** * Publishes and synchronizes a unique identifier specific to a given HBase * cluster. The stored identifier is read from the file system by the active @@ -61,9 +64,9 @@ public class ClusterId { public static String readClusterIdZNode(ZooKeeperWatcher watcher) throws KeeperException { if (ZKUtil.checkExists(watcher, watcher.clusterIdZNode) != -1) { - byte[] data = ZKUtil.getData(watcher, watcher.clusterIdZNode); + byte [] data = ZKUtil.getData(watcher, watcher.clusterIdZNode); if (data != null) { - return Bytes.toString(data); + return getZNodeClusterId(data); } } return null; @@ -71,6 +74,37 @@ public class ClusterId { public static void setClusterId(ZooKeeperWatcher watcher, String id) throws KeeperException { - ZKUtil.createSetData(watcher, watcher.clusterIdZNode, Bytes.toBytes(id)); + ZKUtil.createSetData(watcher, watcher.clusterIdZNode, getZNodeData(id)); } -} + + /** + * @param clusterid + * @return Content of the clusterid znode as a serialized pb with the pb + * magic as prefix. + */ + static byte [] getZNodeData(final String clusterid) { + ZooKeeperProtos.ClusterId.Builder builder = + ZooKeeperProtos.ClusterId.newBuilder(); + builder.setClusterId(clusterid); + return ProtobufUtil.prependPBMagic(builder.build().toByteArray()); + } + + /** + * @param data + * @return The clusterid extracted from the passed znode data + */ + static String getZNodeClusterId(final byte [] data) { + if (data == null || data.length <= 0) return null; + // If no magic, something is seriously wrong. Fail fast. + if (!ProtobufUtil.isPBMagicPrefix(data)) throw new RuntimeException("No magic preamble"); + int prefixLen = ProtobufUtil.lengthOfPBMagic(); + try { + ZooKeeperProtos.ClusterId clusterid = + ZooKeeperProtos.ClusterId.newBuilder().mergeFrom(data, prefixLen, data.length - prefixLen).build(); + return clusterid.getClusterId(); + } catch (InvalidProtocolBufferException e) { + // A failed parse of the znode is pretty catastrophic. Fail fast. + throw new RuntimeException(e); + } + } +} \ No newline at end of file diff --git a/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java b/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java index 61e7367ed5a..10356f91b66 100644 --- a/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java +++ b/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java @@ -24,9 +24,13 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.ClusterStatus; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.util.Bytes; import org.apache.zookeeper.KeeperException; +import com.google.protobuf.InvalidProtocolBufferException; + /** * Tracker on cluster settings up in zookeeper. * This is not related to {@link ClusterStatus}. That class is a data structure @@ -52,7 +56,8 @@ public class ClusterStatusTracker extends ZooKeeperNodeTracker { /** * Checks if cluster is up. - * @return true if root region location is available, false if not + * @return true if the cluster up ('shutdown' is its name up in zk) znode + * exists with data, false if not */ public boolean isClusterUp() { return super.getData(false) != null; @@ -64,7 +69,7 @@ public class ClusterStatusTracker extends ZooKeeperNodeTracker { */ public void setClusterUp() throws KeeperException { - byte [] upData = Bytes.toBytes(new java.util.Date().toString()); + byte [] upData = getZNodeData(); try { ZKUtil.createAndWatch(watcher, watcher.clusterStateZNode, upData); } catch(KeeperException.NodeExistsException nee) { @@ -85,4 +90,15 @@ public class ClusterStatusTracker extends ZooKeeperNodeTracker { "state node (" + watcher.clusterStateZNode + ") not found"); } } + + /** + * @return Content of the clusterup znode as a serialized pb with the pb + * magic as prefix. + */ + static byte [] getZNodeData() { + ZooKeeperProtos.ClusterUp.Builder builder = + ZooKeeperProtos.ClusterUp.newBuilder(); + builder.setStartDate(new java.util.Date().toString()); + return ProtobufUtil.prependPBMagic(builder.build().toByteArray()); + } } diff --git a/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java b/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java index d67e5ab381f..babde80286b 100644 --- a/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java +++ b/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java @@ -113,7 +113,7 @@ public class RootRegionTracker extends ZooKeeperNodeTracker { LOG.info("Setting ROOT region location in ZooKeeper as " + location); // Make the RootRegionServer pb and then get its bytes and save this as // the znode content. - byte [] data = getRootRegionServerZNodeContent(location); + byte [] data = getZNodeData(location); try { ZKUtil.createAndWatch(zookeeper, zookeeper.rootServerZNode, data); } catch(KeeperException.NodeExistsException nee) { @@ -127,7 +127,7 @@ public class RootRegionTracker extends ZooKeeperNodeTracker { * @param sn What to put into the znode. * @return The content of the root-region-server znode */ - static byte [] getRootRegionServerZNodeContent(final ServerName sn) { + static byte [] getZNodeData(final ServerName sn) { // ZNode content is a pb message preceeded by some pb magic. HBaseProtos.ServerName pbsn = HBaseProtos.ServerName.newBuilder().setHostName(sn.getHostname()). diff --git a/src/main/protobuf/ZooKeeper.proto b/src/main/protobuf/ZooKeeper.proto index dc85be99cac..961ab65aa41 100644 --- a/src/main/protobuf/ZooKeeper.proto +++ b/src/main/protobuf/ZooKeeper.proto @@ -42,3 +42,21 @@ message Master { // The ServerName of the current Master required ServerName master = 1; } + +// TODO: Put these two cluster attributes into the one znode. +/** + * Content of the '/hbase/hbaseid', cluster id, znode. + */ +message ClusterId { + // This is the cluster id, a uuid as a String + required string clusterId = 1; +} + +/** + * Content of the '/hbase/shutdown', cluster state, znode. + */ +message ClusterUp { + // If this znode is present, cluster is up. Currently + // the data is cluster startDate. + required string startDate = 1; +}