HBASE-7767 Get rid of ZKTable, and table enable/disable state in ZK (Andrey 
Stepachev)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/3cc5d190
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/3cc5d190
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/3cc5d190

Branch: refs/heads/master
Commit: 3cc5d19039904361f60c413f10f3cbca27a7ba96
Parents: f8c3a5b
Author: stack <st...@apache.org>
Authored: Mon Sep 15 09:34:10 2014 -0700
Committer: stack <st...@apache.org>
Committed: Mon Sep 15 10:12:49 2014 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/client/ConnectionAdapter.java  |    7 +-
 .../hadoop/hbase/client/ConnectionManager.java  |   27 +-
 .../apache/hadoop/hbase/client/HConnection.java |    9 +-
 .../apache/hadoop/hbase/client/Registry.java    |    8 +-
 .../apache/hadoop/hbase/client/TableState.java  |  203 +++
 .../hadoop/hbase/client/ZooKeeperRegistry.java  |   23 +-
 .../hadoop/hbase/protobuf/RequestConverter.java |   16 +
 .../zookeeper/ZKTableStateClientSideReader.java |  168 --
 .../hbase/zookeeper/ZooKeeperWatcher.java       |    1 +
 .../hbase/client/TestClientNoCluster.java       |    8 +-
 .../hbase/protobuf/generated/HBaseProtos.java   | 1720 +++++++++++++++++-
 .../hbase/protobuf/generated/MasterProtos.java  | 1425 ++++++++++++++-
 .../protobuf/generated/ZooKeeperProtos.java     |  211 +--
 hbase-protocol/src/main/protobuf/HBase.proto    |   21 +
 hbase-protocol/src/main/protobuf/Master.proto   |   12 +
 .../src/main/protobuf/ZooKeeper.proto           |    3 +-
 .../hadoop/hbase/CoordinatedStateManager.java   |    8 -
 .../apache/hadoop/hbase/TableDescriptor.java    |  154 ++
 .../apache/hadoop/hbase/TableDescriptors.java   |   26 +-
 .../apache/hadoop/hbase/TableStateManager.java  |  115 --
 .../hbase/client/CoprocessorHConnection.java    |    2 +-
 .../BaseCoordinatedStateManager.java            |    8 +-
 .../coordination/ZkCoordinatedStateManager.java |   15 -
 .../hadoop/hbase/master/AssignmentManager.java  |   93 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   30 +-
 .../hadoop/hbase/master/MasterFileSystem.java   |    6 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   23 +-
 .../hadoop/hbase/master/MasterServices.java     |    5 +
 .../hadoop/hbase/master/RegionStates.java       |   14 +-
 .../hbase/master/TableNamespaceManager.java     |    9 +-
 .../hadoop/hbase/master/TableStateManager.java  |  217 +++
 .../master/handler/CreateTableHandler.java      |   71 +-
 .../master/handler/DeleteTableHandler.java      |   91 +-
 .../master/handler/DisableTableHandler.java     |   30 +-
 .../master/handler/EnableTableHandler.java      |   41 +-
 .../master/handler/ModifyTableHandler.java      |   20 +-
 .../master/handler/ServerShutdownHandler.java   |   17 +-
 .../master/handler/TableAddFamilyHandler.java   |    5 +-
 .../handler/TableDeleteFamilyHandler.java       |    2 +-
 .../hbase/master/handler/TableEventHandler.java |   11 +-
 .../handler/TableModifyFamilyHandler.java       |    3 +-
 .../master/handler/TruncateTableHandler.java    |   90 +-
 .../hbase/master/snapshot/SnapshotManager.java  |    9 +-
 .../hbase/migration/NamespaceUpgrade.java       |    4 +-
 .../hbase/regionserver/CompactionTool.java      |   14 +-
 .../hbase/regionserver/wal/HLogSplitter.java    |   24 +-
 .../hbase/regionserver/wal/WALCellCodec.java    |    1 +
 .../hadoop/hbase/snapshot/SnapshotManifest.java |    8 +-
 .../hadoop/hbase/util/FSTableDescriptors.java   |  176 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |   30 +-
 .../org/apache/hadoop/hbase/util/HMerge.java    |    3 +-
 .../org/apache/hadoop/hbase/util/Merge.java     |    5 +-
 .../hadoop/hbase/util/ZKDataMigrator.java       |   83 +-
 .../hbase/zookeeper/ZKTableStateManager.java    |  330 ----
 .../hadoop/hbase/HBaseTestingUtility.java       |   42 +
 .../TestHColumnDescriptorDefaultVersions.java   |    4 +-
 .../hadoop/hbase/TestTableDescriptor.java       |   57 +
 .../apache/hadoop/hbase/client/TestAdmin.java   |   11 +-
 .../master/TestAssignmentManagerOnCluster.java  |   18 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |   33 +-
 .../apache/hadoop/hbase/master/TestMaster.java  |    4 +-
 .../TestMasterRestartAfterDisablingTable.java   |    8 +-
 .../hbase/master/TestTableLockManager.java      |    5 +-
 .../TestTableDescriptorModification.java        |    6 +-
 .../hbase/snapshot/SnapshotTestingUtils.java    |    7 +-
 .../hbase/util/TestFSTableDescriptors.java      |   46 +-
 .../apache/hadoop/hbase/util/TestMergeTool.java |    4 +-
 .../zookeeper/TestZKTableStateManager.java      |  115 --
 68 files changed, 4518 insertions(+), 1497 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3cc5d190/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java
index 9d1570d..5b4f7c7 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java
@@ -170,6 +170,11 @@ class ConnectionAdapter implements ClusterConnection {
   }
 
   @Override
+  public TableState getTableState(TableName tableName) throws IOException {
+    return wrappedConnection.getTableState(tableName);
+  }
+
+  @Override
   public HTableDescriptor[] listTables() throws IOException {
     return wrappedConnection.listTables();
   }
@@ -435,4 +440,4 @@ class ConnectionAdapter implements ClusterConnection {
   public AsyncProcess getAsyncProcess() {
     return wrappedConnection.getAsyncProcess();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3cc5d190/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
index 7c9c0b9..bbf180e 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
@@ -176,6 +176,8 @@ import com.google.protobuf.BlockingRpcChannel;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
+import static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.*;
+
 /**
  * An internal, A non-instantiable class that manages creation of {@link 
HConnection}s.
  */
@@ -893,7 +895,7 @@ class ConnectionManager {
 
     @Override
     public boolean isTableEnabled(TableName tableName) throws IOException {
-      return this.registry.isTableOnlineState(tableName, true);
+      return getTableState(tableName).inStates(TableState.State.ENABLED);
     }
 
     @Override
@@ -903,7 +905,7 @@ class ConnectionManager {
 
     @Override
     public boolean isTableDisabled(TableName tableName) throws IOException {
-      return this.registry.isTableOnlineState(tableName, false);
+      return getTableState(tableName).inStates(TableState.State.DISABLED);
     }
 
     @Override
@@ -1993,6 +1995,13 @@ class ConnectionManager {
         }
 
         @Override
+        public GetTableStateResponse getTableState(
+                RpcController controller, GetTableStateRequest request)
+                throws ServiceException {
+          return stub.getTableState(controller, request);
+        }
+
+        @Override
         public void close() {
           release(this.mss);
         }
@@ -2498,6 +2507,20 @@ class ConnectionManager {
     throws IOException {
       return getHTableDescriptor(TableName.valueOf(tableName));
     }
+
+    @Override
+    public TableState getTableState(TableName tableName) throws IOException {
+      MasterKeepAliveConnection master = getKeepAliveMasterService();
+      try {
+        GetTableStateResponse resp = master.getTableState(null,
+                RequestConverter.buildGetTableStateRequest(tableName));
+        return TableState.convert(resp.getTableState());
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
+      } finally {
+        master.close();
+      }
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/3cc5d190/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
index cd11a52..918c944 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
@@ -208,6 +208,13 @@ public interface HConnection extends Abortable, Closeable {
   boolean isTableDisabled(byte[] tableName) throws IOException;
 
   /**
+   * Retrieve TableState, represent current table state.
+   * @param tableName table state for
+   * @return state of the table
+   */
+  public TableState getTableState(TableName tableName)  throws IOException;
+
+  /**
    * @param tableName table name
    * @return true if all regions of the table are available, false otherwise
    * @throws IOException if a remote or network exception occurs
@@ -576,4 +583,4 @@ public interface HConnection extends Abortable, Closeable {
    * @deprecated internal method, do not use thru HConnection */
   @Deprecated
   public NonceGenerator getNonceGenerator();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3cc5d190/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java
index aab547e..89c8cef 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java
@@ -44,14 +44,8 @@ interface Registry {
   String getClusterId();
 
   /**
-   * @param enabled Return true if table is enabled
-   * @throws IOException
-   */
-  boolean isTableOnlineState(TableName tableName, boolean enabled) throws 
IOException;
-
-  /**
    * @return Count of 'running' regionservers
    * @throws IOException
    */
   int getCurrentNrHRS() throws IOException;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3cc5d190/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
new file mode 100644
index 0000000..bb7a028
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
@@ -0,0 +1,203 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+
+/**
+ * Represents table state.
+ */
+@InterfaceAudience.Private
+public class TableState {
+
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  public static enum State {
+    ENABLED,
+    DISABLED,
+    DISABLING,
+    ENABLING;
+
+    /**
+     * Covert from PB version of State
+     *
+     * @param state convert from
+     * @return POJO
+     */
+    public static State convert(HBaseProtos.TableState.State state) {
+      State ret;
+      switch (state) {
+      case ENABLED:
+        ret = State.ENABLED;
+        break;
+      case DISABLED:
+        ret = State.DISABLED;
+        break;
+      case DISABLING:
+        ret = State.DISABLING;
+        break;
+      case ENABLING:
+        ret = State.ENABLING;
+        break;
+      default:
+        throw new IllegalStateException(state.toString());
+      }
+      return ret;
+    }
+
+    /**
+     * Covert to PB version of State
+     *
+     * @return PB
+     */
+    public HBaseProtos.TableState.State convert() {
+      HBaseProtos.TableState.State state;
+      switch (this) {
+      case ENABLED:
+        state = HBaseProtos.TableState.State.ENABLED;
+        break;
+      case DISABLED:
+        state = HBaseProtos.TableState.State.DISABLED;
+        break;
+      case DISABLING:
+        state = HBaseProtos.TableState.State.DISABLING;
+        break;
+      case ENABLING:
+        state = HBaseProtos.TableState.State.ENABLING;
+        break;
+      default:
+        throw new IllegalStateException(this.toString());
+      }
+      return state;
+    }
+
+  }
+
+  private final long timestamp;
+  private final TableName tableName;
+  private final State state;
+
+  /**
+   * Create instance of TableState.
+   * @param state table state
+   */
+  public TableState(TableName tableName, State state, long timestamp) {
+    this.tableName = tableName;
+    this.state = state;
+    this.timestamp = timestamp;
+  }
+
+  /**
+   * Create instance of TableState with current timestamp
+   *
+   * @param tableName table for which state is created
+   * @param state     state of the table
+   */
+  public TableState(TableName tableName, State state) {
+    this(tableName, state, System.currentTimeMillis());
+  }
+
+  /**
+   * @return table state
+   */
+  public State getState() {
+    return state;
+  }
+
+  /**
+   * Timestamp of table state
+   *
+   * @return milliseconds
+   */
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  /**
+   * Table name for state
+   *
+   * @return milliseconds
+   */
+  public TableName getTableName() {
+    return tableName;
+  }
+
+  /**
+   * Check that table in given states
+   * @param state state
+   * @return true if satisfies
+   */
+  public boolean inStates(State state) {
+    return this.state.equals(state);
+  }
+
+  /**
+   * Check that table in given states
+   * @param states state list
+   * @return true if satisfies
+   */
+  public boolean inStates(State... states) {
+    for (State s : states) {
+      if (s.equals(this.state))
+        return true;
+    }
+    return false;
+  }
+
+
+  /**
+   * Covert to PB version of TableState
+   * @return PB
+   */
+  public HBaseProtos.TableState convert() {
+    return HBaseProtos.TableState.newBuilder()
+        .setState(this.state.convert())
+        .setTable(ProtobufUtil.toProtoTableName(this.tableName))
+        .setTimestamp(this.timestamp)
+            .build();
+  }
+
+  /**
+   * Covert from PB version of TableState
+   * @param tableState convert from
+   * @return POJO
+   */
+  public static TableState convert(HBaseProtos.TableState tableState) {
+    TableState.State state = State.convert(tableState.getState());
+    return new TableState(ProtobufUtil.toTableName(tableState.getTable()),
+        state, tableState.getTimestamp());
+  }
+
+  /**
+   * Static version of state checker
+   * @param state desired
+   * @param target equals to any of
+   * @return true if satisfies
+   */
+  public static boolean isInStates(State state, State... target) {
+    for (State tableState : target) {
+      if (state.equals(tableState))
+        return true;
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3cc5d190/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
index 9123d50..4d3cc3e 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
@@ -18,18 +18,17 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
-import java.io.InterruptedIOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
-import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.zookeeper.KeeperException;
 
@@ -98,24 +97,6 @@ class ZooKeeperRegistry implements Registry {
   }
 
   @Override
-  public boolean isTableOnlineState(TableName tableName, boolean enabled)
-  throws IOException {
-    ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher();
-    try {
-      if (enabled) {
-        return ZKTableStateClientSideReader.isEnabledTable(zkw, tableName);
-      }
-      return ZKTableStateClientSideReader.isDisabledTable(zkw, tableName);
-    } catch (KeeperException e) {
-      throw new IOException("Enable/Disable failed", e);
-    } catch (InterruptedException e) {
-      throw new InterruptedIOException();
-    } finally {
-       zkw.close();
-    }
-  }
-
-  @Override
   public int getCurrentNrHRS() throws IOException {
     ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher();
     try {
@@ -128,4 +109,4 @@ class ZooKeeperRegistry implements Registry {
         zkw.close();
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3cc5d190/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
index d6bcb29..ab764a3 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.protobuf;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.util.ByteStringer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -106,6 +107,8 @@ import org.apache.hadoop.hbase.util.Pair;
 
 import com.google.protobuf.ByteString;
 
+import static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.*;
+
 /**
  * Helper utility to build protocol buffer requests,
  * or build components for protocol buffer requests.
@@ -1177,6 +1180,19 @@ public final class RequestConverter {
   }
 
   /**
+   * Creates a protocol buffer GetTableStateRequest
+   *
+   * @param tableName table to get request for
+   * @return a GetTableStateRequest
+   */
+  public static GetTableStateRequest buildGetTableStateRequest(
+          final TableName tableName) {
+    return GetTableStateRequest.newBuilder()
+            .setTableName(ProtobufUtil.toProtoTableName(tableName))
+            .build();
+  }
+
+  /**
    * Creates a protocol buffer GetTableDescriptorsRequest for a single table
    *
    * @param tableName the table name

http://git-wip-us.apache.org/repos/asf/hbase/blob/3cc5d190/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableStateClientSideReader.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableStateClientSideReader.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableStateClientSideReader.java
deleted file mode 100644
index 94bd31e..0000000
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableStateClientSideReader.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/**
- * Copyright The Apache Software Foundation
- *
- * 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.zookeeper;
-
-import com.google.protobuf.InvalidProtocolBufferException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
-import org.apache.zookeeper.KeeperException;
-
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-/**
- * Non-instantiable class that provides helper functions to learn
- * about HBase table state for code running on client side (hence, not having
- * access to consensus context).
- *
- * Doesn't cache any table state, just goes directly to ZooKeeper.
- * TODO: decouple this class from ZooKeeper.
- */
-@InterfaceAudience.Private
-public class ZKTableStateClientSideReader {
-
-  private ZKTableStateClientSideReader() {}
-  
-  /**
-   * Go to zookeeper and see if state of table is {@code 
ZooKeeperProtos.Table.State#DISABLED}.
-   * This method does not use cache.
-   * This method is for clients other than AssignmentManager
-   * @param zkw ZooKeeperWatcher instance to use
-   * @param tableName table we're checking
-   * @return True if table is enabled.
-   * @throws KeeperException
-   */
-  public static boolean isDisabledTable(final ZooKeeperWatcher zkw,
-      final TableName tableName)
-      throws KeeperException, InterruptedException {
-    ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
-    return isTableState(ZooKeeperProtos.Table.State.DISABLED, state);
-  }
-
-  /**
-   * Go to zookeeper and see if state of table is {@code 
ZooKeeperProtos.Table.State#ENABLED}.
-   * This method does not use cache.
-   * This method is for clients other than AssignmentManager
-   * @param zkw ZooKeeperWatcher instance to use
-   * @param tableName table we're checking
-   * @return True if table is enabled.
-   * @throws KeeperException
-   */
-  public static boolean isEnabledTable(final ZooKeeperWatcher zkw,
-      final TableName tableName)
-      throws KeeperException, InterruptedException {
-    return getTableState(zkw, tableName) == 
ZooKeeperProtos.Table.State.ENABLED;
-  }
-
-  /**
-   * Go to zookeeper and see if state of table is {@code 
ZooKeeperProtos.Table.State#DISABLING}
-   * of {@code ZooKeeperProtos.Table.State#DISABLED}.
-   * This method does not use cache.
-   * This method is for clients other than AssignmentManager.
-   * @param zkw ZooKeeperWatcher instance to use
-   * @param tableName table we're checking
-   * @return True if table is enabled.
-   * @throws KeeperException
-   */
-  public static boolean isDisablingOrDisabledTable(final ZooKeeperWatcher zkw,
-      final TableName tableName)
-      throws KeeperException, InterruptedException {
-    ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
-    return isTableState(ZooKeeperProtos.Table.State.DISABLING, state) ||
-      isTableState(ZooKeeperProtos.Table.State.DISABLED, state);
-  }
-
-  /**
-   * Gets a list of all the tables set as disabled in zookeeper.
-   * @return Set of disabled tables, empty Set if none
-   * @throws KeeperException
-   */
-  public static Set<TableName> getDisabledTables(ZooKeeperWatcher zkw)
-      throws KeeperException, InterruptedException {
-    Set<TableName> disabledTables = new HashSet<TableName>();
-    List<String> children =
-      ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
-    for (String child: children) {
-      TableName tableName =
-          TableName.valueOf(child);
-      ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
-      if (state == ZooKeeperProtos.Table.State.DISABLED) 
disabledTables.add(tableName);
-    }
-    return disabledTables;
-  }
-
-  /**
-   * Gets a list of all the tables set as disabled in zookeeper.
-   * @return Set of disabled tables, empty Set if none
-   * @throws KeeperException
-   */
-  public static Set<TableName> getDisabledOrDisablingTables(ZooKeeperWatcher 
zkw)
-      throws KeeperException, InterruptedException {
-    Set<TableName> disabledTables = new HashSet<TableName>();
-    List<String> children =
-      ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
-    for (String child: children) {
-      TableName tableName =
-          TableName.valueOf(child);
-      ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
-      if (state == ZooKeeperProtos.Table.State.DISABLED ||
-          state == ZooKeeperProtos.Table.State.DISABLING)
-        disabledTables.add(tableName);
-    }
-    return disabledTables;
-  }
-
-  static boolean isTableState(final ZooKeeperProtos.Table.State expectedState,
-      final ZooKeeperProtos.Table.State currentState) {
-    return currentState != null && currentState.equals(expectedState);
-  }
-
-  /**
-   * @param zkw ZooKeeperWatcher instance to use
-   * @param tableName table we're checking
-   * @return Null or {@link ZooKeeperProtos.Table.State} found in znode.
-   * @throws KeeperException
-   */
-  static ZooKeeperProtos.Table.State getTableState(final ZooKeeperWatcher zkw,
-      final TableName tableName)
-      throws KeeperException, InterruptedException {
-    String znode = ZKUtil.joinZNode(zkw.tableZNode, 
tableName.getNameAsString());
-    byte [] data = ZKUtil.getData(zkw, znode);
-    if (data == null || data.length <= 0) return null;
-    try {
-      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);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3cc5d190/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
index 8af9555..2546f43 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
@@ -94,6 +94,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, 
Closeable {
   // znode containing the current cluster state
   public String clusterStateZNode;
   // znode used for table disabling/enabling
+  @Deprecated
   public String tableZNode;
   // znode containing the unique cluster ID
   public String clusterIdZNode;

http://git-wip-us.apache.org/repos/asf/hbase/blob/3cc5d190/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
 
b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
index 58b4604..c06dd99 100644
--- 
a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
+++ 
b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
@@ -130,12 +130,6 @@ public class TestClientNoCluster extends Configured 
implements Tool {
     }
 
     @Override
-    public boolean isTableOnlineState(TableName tableName, boolean enabled)
-    throws IOException {
-      return enabled;
-    }
-
-    @Override
     public int getCurrentNrHRS() throws IOException {
       return 1;
     }
@@ -814,4 +808,4 @@ public class TestClientNoCluster extends Configured 
implements Tool {
   public static void main(String[] args) throws Exception {
     System.exit(ToolRunner.run(HBaseConfiguration.create(), new 
TestClientNoCluster(), args));
   }
-}
\ No newline at end of file
+}

Reply via email to