You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2012/05/09 19:08:34 UTC

svn commit: r1336302 - in /hbase/trunk/src: main/java/org/apache/hadoop/hbase/master/ main/java/org/apache/hadoop/hbase/protobuf/generated/ main/java/org/apache/hadoop/hbase/zookeeper/ main/protobuf/ test/java/org/apache/hadoop/hbase/

Author: stack
Date: Wed May  9 17:08:33 2012
New Revision: 1336302

URL: http://svn.apache.org/viewvc?rev=1336302&view=rev
Log:
HBASE-5944 Convert rs, shutdown, and table dir content to pb

Modified:
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MXBeanImpl.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
    hbase/trunk/src/main/protobuf/ZooKeeper.proto
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1336302&r1=1336301&r2=1336302&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Wed May  9 17:08:33 2012
@@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.Deseriali
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HServerLoad;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MXBeanImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MXBeanImpl.java?rev=1336302&r1=1336301&r2=1336302&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MXBeanImpl.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MXBeanImpl.java Wed May  9 17:08:33 2012
@@ -79,10 +79,9 @@ public class MXBeanImpl implements MXBea
   @Override
   public Map<String, ServerLoad> getRegionServers() {
     Map<String, ServerLoad> data = new HashMap<String, ServerLoad>();
-    for (final Entry<ServerName, ServerLoad> entry :
-      master.getServerManager().getOnlineServers().entrySet()) {
-      data.put(entry.getKey().getServerName(),
-          entry.getValue());
+    for (final Entry<ServerName, ServerLoad> entry:
+        this.master.getServerManager().getOnlineServers().entrySet()) {
+      data.put(entry.getKey().getServerName(), entry.getValue());
     }
     return data;
   }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java?rev=1336302&r1=1336301&r2=1336302&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java Wed May  9 17:08:33 2012
@@ -3183,6 +3183,469 @@ public final class ZooKeeperProtos {
     // @@protoc_insertion_point(class_scope:SplitLogTask)
   }
   
+  public interface TableOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .Table.State state = 1 [default = ENABLED];
+    boolean hasState();
+    org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State getState();
+  }
+  public static final class Table extends
+      com.google.protobuf.GeneratedMessage
+      implements TableOrBuilder {
+    // Use Table.newBuilder() to construct.
+    private Table(Builder builder) {
+      super(builder);
+    }
+    private Table(boolean noInit) {}
+    
+    private static final Table defaultInstance;
+    public static Table getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public Table getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_Table_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_Table_fieldAccessorTable;
+    }
+    
+    public enum State
+        implements com.google.protobuf.ProtocolMessageEnum {
+      ENABLED(0, 0),
+      DISABLED(1, 1),
+      DISABLING(2, 2),
+      ENABLING(3, 3),
+      ;
+      
+      public static final int ENABLED_VALUE = 0;
+      public static final int DISABLED_VALUE = 1;
+      public static final int DISABLING_VALUE = 2;
+      public static final int ENABLING_VALUE = 3;
+      
+      
+      public final int getNumber() { return value; }
+      
+      public static State valueOf(int value) {
+        switch (value) {
+          case 0: return ENABLED;
+          case 1: return DISABLED;
+          case 2: return DISABLING;
+          case 3: return ENABLING;
+          default: return null;
+        }
+      }
+      
+      public static com.google.protobuf.Internal.EnumLiteMap<State>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static com.google.protobuf.Internal.EnumLiteMap<State>
+          internalValueMap =
+            new com.google.protobuf.Internal.EnumLiteMap<State>() {
+              public State findValueByNumber(int number) {
+                return State.valueOf(number);
+              }
+            };
+      
+      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+          getValueDescriptor() {
+        return getDescriptor().getValues().get(index);
+      }
+      public final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptorForType() {
+        return getDescriptor();
+      }
+      public static final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDescriptor().getEnumTypes().get(0);
+      }
+      
+      private static final State[] VALUES = {
+        ENABLED, DISABLED, DISABLING, ENABLING, 
+      };
+      
+      public static State valueOf(
+          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        if (desc.getType() != getDescriptor()) {
+          throw new java.lang.IllegalArgumentException(
+            "EnumValueDescriptor is not for this type.");
+        }
+        return VALUES[desc.getIndex()];
+      }
+      
+      private final int index;
+      private final int value;
+      
+      private State(int index, int value) {
+        this.index = index;
+        this.value = value;
+      }
+      
+      // @@protoc_insertion_point(enum_scope:Table.State)
+    }
+    
+    private int bitField0_;
+    // required .Table.State state = 1 [default = ENABLED];
+    public static final int STATE_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State state_;
+    public boolean hasState() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State getState() {
+      return state_;
+    }
+    
+    private void initFields() {
+      state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.ENABLED;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasState()) {
+        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.writeEnum(1, state_.getNumber());
+      }
+      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
+          .computeEnumSize(1, state_.getNumber());
+      }
+      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.Table)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table) obj;
+      
+      boolean result = true;
+      result = result && (hasState() == other.hasState());
+      if (hasState()) {
+        result = result &&
+            (getState() == other.getState());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasState()) {
+        hash = (37 * hash) + STATE_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getState());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table 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.Table 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.Table parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table 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.Table parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table 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.Table 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.Table 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.Table parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table 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.Table prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_Table_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_Table_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.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();
+        state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.ENABLED;
+        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.Table.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table build() {
+        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.state_ = state_;
+        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.Table) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDefaultInstance()) return this;
+        if (other.hasState()) {
+          setState(other.getState());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasState()) {
+          
+          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 8: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State value = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(1, rawValue);
+              } else {
+                bitField0_ |= 0x00000001;
+                state_ = value;
+              }
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .Table.State state = 1 [default = ENABLED];
+      private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.ENABLED;
+      public boolean hasState() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State getState() {
+        return state_;
+      }
+      public Builder setState(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000001;
+        state_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearState() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        state_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State.ENABLED;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:Table)
+    }
+    
+    static {
+      defaultInstance = new Table(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:Table)
+  }
+  
   private static com.google.protobuf.Descriptors.Descriptor
     internal_static_RootRegionServer_descriptor;
   private static
@@ -3213,6 +3676,11 @@ public final class ZooKeeperProtos {
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_SplitLogTask_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_Table_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_Table_fieldAccessorTable;
   
   public static com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -3233,9 +3701,12 @@ public final class ZooKeeperProtos {
       "(\014\"\230\001\n\014SplitLogTask\022\"\n\005state\030\001 \002(\0162\023.Spl" +
       "itLogTask.State\022\037\n\nserverName\030\002 \002(\0132\013.Se",
       "rverName\"C\n\005State\022\016\n\nUNASSIGNED\020\000\022\t\n\005OWN" +
-      "ED\020\001\022\014\n\010RESIGNED\020\002\022\010\n\004DONE\020\003\022\007\n\003ERR\020\004BE\n" +
-      "*org.apache.hadoop.hbase.protobuf.genera" +
-      "tedB\017ZooKeeperProtosH\001\210\001\001\240\001\001"
+      "ED\020\001\022\014\n\010RESIGNED\020\002\022\010\n\004DONE\020\003\022\007\n\003ERR\020\004\"n\n" +
+      "\005Table\022$\n\005state\030\001 \002(\0162\014.Table.State:\007ENA" +
+      "BLED\"?\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DISABLED\020\001" +
+      "\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003BE\n*org.apa" +
+      "che.hadoop.hbase.protobuf.generatedB\017Zoo" +
+      "KeeperProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -3290,6 +3761,14 @@ public final class ZooKeeperProtos {
               new java.lang.String[] { "State", "ServerName", },
               org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.class,
               org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.Builder.class);
+          internal_static_Table_descriptor =
+            getDescriptor().getMessageTypes().get(6);
+          internal_static_Table_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_Table_descriptor,
+              new java.lang.String[] { "State", },
+              org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.class,
+              org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.Builder.class);
           return null;
         }
       };

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java?rev=1336302&r1=1336301&r2=1336302&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java Wed May  9 17:08:33 2012
@@ -26,11 +26,8 @@ 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
@@ -69,7 +66,7 @@ public class ClusterStatusTracker extend
    */
   public void setClusterUp()
   throws KeeperException {
-    byte [] upData = getZNodeData();
+    byte [] upData = toByteArray();
     try {
       ZKUtil.createAndWatch(watcher, watcher.clusterStateZNode, upData);
     } catch(KeeperException.NodeExistsException nee) {
@@ -95,10 +92,10 @@ public class ClusterStatusTracker extend
    * @return Content of the clusterup znode as a serialized pb with the pb
    * magic as prefix.
    */
-  static byte [] getZNodeData() {
+  static byte [] toByteArray() {
     ZooKeeperProtos.ClusterUp.Builder builder =
       ZooKeeperProtos.ClusterUp.newBuilder();
     builder.setStartDate(new java.util.Date().toString());
     return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
   }
-}
+}
\ No newline at end of file

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java?rev=1336302&r1=1336301&r2=1336302&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java Wed May  9 17:08:33 2012
@@ -27,7 +27,6 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.HServerAddress;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.zookeeper.KeeperException;
 
@@ -119,5 +118,4 @@ public class DrainingServerTracker exten
       }
     }
   }
-
-}
+}
\ No newline at end of file

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java?rev=1336302&r1=1336301&r2=1336302&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java Wed May  9 17:08:33 2012
@@ -182,4 +182,4 @@ public class RootRegionTracker extends Z
       return null;
     }
   }
-}
+}
\ No newline at end of file

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java?rev=1336302&r1=1336301&r2=1336302&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java Wed May  9 17:08:33 2012
@@ -37,7 +37,9 @@ import org.apache.zookeeper.KeeperExcept
  * If the current master instance fails, the ephemeral leader znode will
  * be removed, and all waiting instances will be notified, with the race
  * to claim the leader znode beginning all over again.
+ * @deprecated Not used
  */
+@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class ZKLeaderManager extends ZooKeeperListener {
@@ -176,4 +178,4 @@ public class ZKLeaderManager extends Zoo
   public boolean hasLeader() {
     return leaderExists.get();
   }
-}
+}
\ No newline at end of file

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java?rev=1336302&r1=1336301&r2=1336302&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java Wed May  9 17:08:33 2012
@@ -28,10 +28,14 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.master.AssignmentManager;
-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;
+
 /**
  * Helper class for table state tracking for use by {@link AssignmentManager}.
  * Reads, caches and sets state up in zookeeper.  If multiple read/write
@@ -58,22 +62,11 @@ public class ZKTable {
    * for every query.  Synchronize access rather than use concurrent Map because
    * synchronization needs to span query of zk.
    */
-  private final Map<String, TableState> cache =
-    new HashMap<String, TableState>();
+  private final Map<String, ZooKeeperProtos.Table.State> cache =
+    new HashMap<String, ZooKeeperProtos.Table.State>();
 
   // TODO: Make it so always a table znode. Put table schema here as well as table state.
   // Have watcher on table znode so all are notified of state or schema change.
-  /**
-   * States a Table can be in.
-   * {@link TableState#ENABLED} is not used currently; its the absence of state
-   * in zookeeper that indicates an enabled table currently.
-   */
-  public static enum TableState {
-    ENABLED,
-    DISABLED,
-    DISABLING,
-    ENABLING
-  };
 
   public ZKTable(final ZooKeeperWatcher zkw) throws KeeperException {
     super();
@@ -88,11 +81,10 @@ public class ZKTable {
   private void populateTableStates()
   throws KeeperException {
     synchronized (this.cache) {
-      List<String> children =
-        ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.tableZNode);
+      List<String> children = ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.tableZNode);
       if (children == null) return;
       for (String child: children) {
-        TableState state = getTableState(this.watcher, child);
+        ZooKeeperProtos.Table.State state = getTableState(this.watcher, child);
         if (state != null) this.cache.put(child, state);
       }
     }
@@ -104,20 +96,24 @@ public class ZKTable {
    * @return Null or {@link TableState} found in znode.
    * @throws KeeperException
    */
-  private static TableState getTableState(final ZooKeeperWatcher zkw,
+  private static ZooKeeperProtos.Table.State getTableState(final ZooKeeperWatcher zkw,
       final String child)
   throws KeeperException {
     String znode = ZKUtil.joinZNode(zkw.tableZNode, child);
     byte [] data = ZKUtil.getData(zkw, znode);
-    if (data == null || data.length <= 0) {
-      // Null if table is enabled.
-      return null;
-    }
-    String str = Bytes.toString(data);
+    if (data == null || data.length <= 0) return ZooKeeperProtos.Table.State.ENABLED;
     try {
-      return TableState.valueOf(str);
-    } catch (IllegalArgumentException e) {
-      throw new IllegalArgumentException(str);
+      ProtobufUtil.expectPBMagicPrefix(data);
+      ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
+      int magicLen = ProtobufUtil.lengthOfPBMagic();
+      ZooKeeperProtos.Table t = builder.mergeFrom(data, magicLen, data.length - magicLen).build();
+      return t.getState();
+    } catch (InvalidProtocolBufferException e) {
+      KeeperException ke = new KeeperException.DataInconsistencyException();
+      ke.initCause(e);
+      throw ke;
+    } catch (DeserializationException e) {
+      throw ZKUtil.convert(e);
     }
   }
 
@@ -134,7 +130,7 @@ public class ZKTable {
         LOG.warn("Moving table " + tableName + " state to disabled but was " +
           "not first in disabling state: " + this.cache.get(tableName));
       }
-      setTableState(tableName, TableState.DISABLED);
+      setTableState(tableName, ZooKeeperProtos.Table.State.DISABLED);
     }
   }
 
@@ -151,7 +147,7 @@ public class ZKTable {
         LOG.warn("Moving table " + tableName + " state to disabling but was " +
           "not first in enabled state: " + this.cache.get(tableName));
       }
-      setTableState(tableName, TableState.DISABLING);
+      setTableState(tableName, ZooKeeperProtos.Table.State.DISABLING);
     }
   }
 
@@ -168,7 +164,7 @@ public class ZKTable {
         LOG.warn("Moving table " + tableName + " state to enabling but was " +
           "not first in disabled state: " + this.cache.get(tableName));
       }
-      setTableState(tableName, TableState.ENABLING);
+      setTableState(tableName, ZooKeeperProtos.Table.State.ENABLING);
     }
   }
 
@@ -185,7 +181,7 @@ public class ZKTable {
       if (isEnablingTable(tableName)) {
         return false;
       }
-      setTableState(tableName, TableState.ENABLING);
+      setTableState(tableName, ZooKeeperProtos.Table.State.ENABLING);
       return true;
     }
   }
@@ -203,7 +199,7 @@ public class ZKTable {
       if (!isDisabledTable(tableName)) {
         return false;
       }
-      setTableState(tableName, TableState.ENABLING);
+      setTableState(tableName, ZooKeeperProtos.Table.State.ENABLING);
       return true;
     }
   }
@@ -221,25 +217,28 @@ public class ZKTable {
       if (this.cache.get(tableName) != null && !isEnabledTable(tableName)) {
         return false;
       }
-      setTableState(tableName, TableState.DISABLING);
+      setTableState(tableName, ZooKeeperProtos.Table.State.DISABLING);
       return true;
     }
   }
 
-  private void setTableState(final String tableName, final TableState state)
+  private void setTableState(final String tableName, final ZooKeeperProtos.Table.State state)
   throws KeeperException {
     String znode = ZKUtil.joinZNode(this.watcher.tableZNode, tableName);
     if (ZKUtil.checkExists(this.watcher, znode) == -1) {
       ZKUtil.createAndFailSilent(this.watcher, znode);
     }
     synchronized (this.cache) {
-      ZKUtil.setData(this.watcher, znode, Bytes.toBytes(state.toString()));
+      ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
+      builder.setState(state);
+      byte [] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
+      ZKUtil.setData(this.watcher, znode, data);
       this.cache.put(tableName, state);
     }
   }
 
   public boolean isDisabledTable(final String tableName) {
-    return isTableState(tableName, TableState.DISABLED);
+    return isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED);
   }
 
   /**
@@ -254,20 +253,20 @@ public class ZKTable {
   public static boolean isDisabledTable(final ZooKeeperWatcher zkw,
       final String tableName)
   throws KeeperException {
-    TableState state = getTableState(zkw, tableName);
-    return isTableState(TableState.DISABLED, state);
+    ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
+    return isTableState(ZooKeeperProtos.Table.State.DISABLED, state);
   }
 
   public boolean isDisablingTable(final String tableName) {
-    return isTableState(tableName, TableState.DISABLING);
+    return isTableState(tableName, ZooKeeperProtos.Table.State.DISABLING);
   }
 
   public boolean isEnablingTable(final String tableName) {
-    return isTableState(tableName, TableState.ENABLING);
+    return isTableState(tableName, ZooKeeperProtos.Table.State.ENABLING);
   }
 
   public boolean isEnabledTable(String tableName) {
-    return isTableState(tableName, TableState.ENABLED);
+    return isTableState(tableName, ZooKeeperProtos.Table.State.ENABLED);
   }
 
   /**
@@ -282,7 +281,7 @@ public class ZKTable {
   public static boolean isEnabledTable(final ZooKeeperWatcher zkw,
       final String tableName)
   throws KeeperException {
-    return getTableState(zkw, tableName) == TableState.ENABLED;
+    return getTableState(zkw, tableName) == ZooKeeperProtos.Table.State.ENABLED;
   }
 
   public boolean isDisablingOrDisabledTable(final String tableName) {
@@ -304,9 +303,9 @@ public class ZKTable {
   public static boolean isDisablingOrDisabledTable(final ZooKeeperWatcher zkw,
       final String tableName)
   throws KeeperException {
-    TableState state = getTableState(zkw, tableName);
-    return isTableState(TableState.DISABLING, state) ||
-      isTableState(TableState.DISABLED, state);
+    ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
+    return isTableState(ZooKeeperProtos.Table.State.DISABLING, state) ||
+      isTableState(ZooKeeperProtos.Table.State.DISABLED, state);
   }
 
   public boolean isEnabledOrDisablingTable(final String tableName) {
@@ -321,15 +320,15 @@ public class ZKTable {
     }
   }
 
-  private boolean isTableState(final String tableName, final TableState state) {
+  private boolean isTableState(final String tableName, final ZooKeeperProtos.Table.State state) {
     synchronized (this.cache) {
-      TableState currentState = this.cache.get(tableName);
+      ZooKeeperProtos.Table.State currentState = this.cache.get(tableName);
       return isTableState(currentState, state);
     }
   }
 
-  private static boolean isTableState(final TableState expectedState,
-      final TableState currentState) {
+  private static boolean isTableState(final ZooKeeperProtos.Table.State expectedState,
+      final ZooKeeperProtos.Table.State currentState) {
     return currentState != null && currentState.equals(expectedState);
   }
 
@@ -359,7 +358,7 @@ public class ZKTable {
    * @throws KeeperException
    */
   public void setEnabledTable(final String tableName) throws KeeperException {
-    setTableState(tableName, TableState.ENABLED);
+    setTableState(tableName, ZooKeeperProtos.Table.State.ENABLED);
   }
 
   /**
@@ -370,7 +369,7 @@ public class ZKTable {
    */
   public boolean isTablePresent(final String tableName) {
     synchronized (this.cache) {
-      TableState state = this.cache.get(tableName);
+      ZooKeeperProtos.Table.State state = this.cache.get(tableName);
       return !(state == null);
     }
   }
@@ -401,8 +400,8 @@ public class ZKTable {
     List<String> children =
       ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
     for (String child: children) {
-      TableState state = getTableState(zkw, child);
-      if (state == TableState.DISABLED) disabledTables.add(child);
+      ZooKeeperProtos.Table.State state = getTableState(zkw, child);
+      if (state == ZooKeeperProtos.Table.State.DISABLED) disabledTables.add(child);
     }
     return disabledTables;
   }
@@ -418,10 +417,11 @@ public class ZKTable {
     List<String> children =
       ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
     for (String child: children) {
-      TableState state = getTableState(zkw, child);
-      if (state == TableState.DISABLED || state == TableState.DISABLING)
+      ZooKeeperProtos.Table.State state = getTableState(zkw, child);
+      if (state == ZooKeeperProtos.Table.State.DISABLED ||
+          state == ZooKeeperProtos.Table.State.DISABLING)
         disabledTables.add(child);
     }
     return disabledTables;
   }
-}
+}
\ No newline at end of file

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java?rev=1336302&r1=1336301&r2=1336302&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java Wed May  9 17:08:33 2012
@@ -1058,8 +1058,7 @@ public class ZKUtil {
                                               zkw.backupMasterAddressesZNode)) {
         sb.append("\n ").append(child);
       }
-      sb.append("\nRegion server holding ROOT: ").append(
-          Bytes.toStringBinary(getData(zkw, zkw.rootServerZNode)));
+      sb.append("\nRegion server holding ROOT: " + RootRegionTracker.getRootRegionLocation(zkw));
       sb.append("\nRegion servers:");
       for (String child : listChildrenNoWatch(zkw, zkw.rsZNode)) {
         sb.append("\n ").append(child);

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java?rev=1336302&r1=1336301&r2=1336302&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java Wed May  9 17:08:33 2012
@@ -34,7 +34,6 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.thrift.generated.Hbase.deleteAll_args;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;

Modified: hbase/trunk/src/main/protobuf/ZooKeeper.proto
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/protobuf/ZooKeeper.proto?rev=1336302&r1=1336301&r2=1336302&view=diff
==============================================================================
--- hbase/trunk/src/main/protobuf/ZooKeeper.proto (original)
+++ hbase/trunk/src/main/protobuf/ZooKeeper.proto Wed May  9 17:08:33 2012
@@ -90,3 +90,20 @@ message SplitLogTask {
   required State state = 1;
   required ServerName serverName = 2;
 }
+
+/**
+ * The znode that holds state of table.
+ */
+message Table {
+  // Table's current state
+  enum State {
+    ENABLED = 0;
+    DISABLED = 1;
+    DISABLING = 2;
+    ENABLING = 3;
+  }
+  // This is the table's state.  If no znode for a table,
+  // its state is presumed enabled.  See o.a.h.h.zookeeper.ZKTable class
+  // for more.
+  required State state = 1 [default = ENABLED];
+}

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java?rev=1336302&r1=1336301&r2=1336302&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java Wed May  9 17:08:33 2012
@@ -112,8 +112,7 @@ public class TestZooKeeper {
   public void testClientSessionExpired() throws Exception {
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
 
-    // We don't want to share the connection as we will check
-    //  its state
+    // We don't want to share the connection as we will check its state
     c.set(HConstants.HBASE_CLIENT_INSTANCE_ID, "1111");
 
     HConnection connection = HConnectionManager.getConnection(c);