You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2017/12/18 12:46:47 UTC

[15/26] hbase git commit: HBASE-19530 New regions should always be added with state CLOSED

HBASE-19530 New regions should always be added with state CLOSED


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

Branch: refs/heads/HBASE-19397
Commit: a651ab93e3303eb5c60747c0a6bf6cc0122eee2f
Parents: d276c32
Author: Apekshit Sharma <ap...@apache.org>
Authored: Fri Dec 15 17:30:21 2017 -0800
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Fri Dec 15 21:20:58 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/MetaTableAccessor.java  | 36 ++++++++++++++++----
 1 file changed, 30 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a651ab93/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index fd339b3..df2102a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
@@ -872,11 +873,20 @@ public class MetaTableAccessor {
    *
    * @return HConstants.TABLE_STATE_QUALIFIER
    */
-  protected static byte[] getStateColumn() {
+  protected static byte[] getTableStateColumn() {
     return HConstants.TABLE_STATE_QUALIFIER;
   }
 
   /**
+   * Returns the column qualifier for serialized region state
+   *
+   * @return HConstants.TABLE_STATE_QUALIFIER
+   */
+  protected static byte[] getRegionStateColumn() {
+    return HConstants.STATE_QUALIFIER;
+  }
+
+  /**
    * Returns the column qualifier for server column for replicaId
    * @param replicaId the replicaId of the region
    * @return a byte[] for server column qualifier
@@ -1101,7 +1111,7 @@ public class MetaTableAccessor {
   public static TableState getTableState(Connection conn, TableName tableName)
       throws IOException {
     Table metaHTable = getMetaHTable(conn);
-    Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getStateColumn());
+    Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getTableStateColumn());
     long time = EnvironmentEdgeManager.currentTime();
     get.setTimeRange(0, time);
     Result result =
@@ -1152,7 +1162,7 @@ public class MetaTableAccessor {
   @Nullable
   public static TableState getTableState(Result r)
       throws IOException {
-    Cell cell = r.getColumnLatestCell(getTableFamily(), getStateColumn());
+    Cell cell = r.getColumnLatestCell(getTableFamily(), getTableStateColumn());
     if (cell == null) return null;
     try {
       return TableState.parseFrom(TableName.valueOf(r.getRow()),
@@ -1556,7 +1566,8 @@ public class MetaTableAccessor {
    * @param regionInfo region information
    * @throws IOException if problem connecting or updating meta
    */
-  public static void addRegionToMeta(Connection connection,
+  @VisibleForTesting
+  static void addRegionToMeta(Connection connection,
                                      RegionInfo regionInfo)
     throws IOException {
     putToMetaTable(connection, makePutFromRegionInfo(regionInfo));
@@ -1574,6 +1585,18 @@ public class MetaTableAccessor {
     addRegionToMeta(meta, regionInfo, null, null);
   }
 
+  static void addRegionStateToPut(Put put, RegionState.State  state)
+      throws IOException {
+    put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
+        .setRow(put.getRow())
+        .setFamily(HConstants.CATALOG_FAMILY)
+        .setQualifier(getRegionStateColumn())
+        .setTimestamp(put.getTimeStamp())
+        .setType(CellBuilder.DataType.Put)
+        .setValue(Bytes.toBytes(state.name()))
+        .build());
+  }
+
   /**
    * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
    * does not add its daughter's as different rows, but adds information about the daughters
@@ -1589,6 +1612,7 @@ public class MetaTableAccessor {
   public static void addRegionToMeta(Table meta, RegionInfo regionInfo,
                                      RegionInfo splitA, RegionInfo splitB) throws IOException {
     Put put = makePutFromRegionInfo(regionInfo);
+    addRegionStateToPut(put, RegionState.State.CLOSED);
     addDaughtersToPut(put, splitA, splitB);
     meta.put(put);
     if (METALOG.isDebugEnabled()) {
@@ -1835,7 +1859,7 @@ public class MetaTableAccessor {
   public static Put makePutFromTableState(TableState state) {
     long time = EnvironmentEdgeManager.currentTime();
     Put put = new Put(state.getTableName().getName(), time);
-    put.addColumn(getTableFamily(), getStateColumn(), state.convert().toByteArray());
+    put.addColumn(getTableFamily(), getTableStateColumn(), state.convert().toByteArray());
     return put;
   }
 
@@ -1848,7 +1872,7 @@ public class MetaTableAccessor {
       throws IOException {
     long time = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(table.getName());
-    delete.addColumns(getTableFamily(), getStateColumn(), time);
+    delete.addColumns(getTableFamily(), getTableStateColumn(), time);
     deleteFromMetaTable(connection, delete);
     LOG.info("Deleted table " + table + " state from META");
   }