You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ga...@apache.org on 2016/12/09 01:07:42 UTC

[1/3] hive git commit: HIVE-15342 Add support for primary/foreign keys in HBase metastore (Alan Gates, reviewed by Daniel Dai)

Repository: hive
Updated Branches:
  refs/heads/master d61c60ba7 -> 0ed01fdf8


http://git-wip-us.apache.org/repos/asf/hive/blob/0ed01fdf/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
index d503cff..e687a69 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
@@ -23,6 +23,8 @@ import com.google.common.collect.Iterators;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -143,6 +145,8 @@ public class HBaseReadWrite implements MetadataStore {
   private final static byte[] REF_COUNT_COL = "ref".getBytes(HBaseUtils.ENCODING);
   private final static byte[] DELEGATION_TOKEN_COL = "dt".getBytes(HBaseUtils.ENCODING);
   private final static byte[] MASTER_KEY_COL = "mk".getBytes(HBaseUtils.ENCODING);
+  private final static byte[] PRIMARY_KEY_COL = "pk".getBytes(HBaseUtils.ENCODING);
+  private final static byte[] FOREIGN_KEY_COL = "fk".getBytes(HBaseUtils.ENCODING);
   private final static byte[] GLOBAL_PRIVS_KEY = "gp".getBytes(HBaseUtils.ENCODING);
   private final static byte[] SEQUENCES_KEY = "seq".getBytes(HBaseUtils.ENCODING);
   private final static int TABLES_TO_CACHE = 10;
@@ -1716,6 +1720,22 @@ public class HBaseReadWrite implements MetadataStore {
       ColumnStatisticsObj cso = HBaseUtils.deserializeStatsForOneColumn(pcs, statsCol.getValue());
       builder.append(dumpThriftObject(cso));
     }
+    // Add the primary key
+    List<SQLPrimaryKey> pk = getPrimaryKey(sdParts.containingTable.getDbName(),
+        sdParts.containingTable.getTableName());
+    if (pk != null && pk.size() > 0) {
+      builder.append(" primary key: ");
+      for (SQLPrimaryKey pkcol : pk) builder.append(dumpThriftObject(pkcol));
+    }
+
+    // Add any foreign keys
+    List<SQLForeignKey> fks = getForeignKeys(sdParts.containingTable.getDbName(),
+        sdParts.containingTable.getTableName());
+    if (fks != null && fks.size() > 0) {
+      builder.append(" foreign keys: ");
+      for (SQLForeignKey fkcol : fks) builder.append(dumpThriftObject(fkcol));
+
+    }
     return builder.toString();
   }
 
@@ -2530,6 +2550,86 @@ public class HBaseReadWrite implements MetadataStore {
   }
 
   /**********************************************************************************************
+   * Constraints (pk/fk) related methods
+   *********************************************************************************************/
+
+  /**
+   * Fetch a primary key
+   * @param dbName database the table is in
+   * @param tableName table name
+   * @return List of primary key objects, which together make up one key
+   * @throws IOException if there's a read error
+   */
+  List<SQLPrimaryKey> getPrimaryKey(String dbName, String tableName) throws IOException {
+    byte[] key = HBaseUtils.buildKey(dbName, tableName);
+    byte[] serialized = read(TABLE_TABLE, key, CATALOG_CF, PRIMARY_KEY_COL);
+    if (serialized == null) return null;
+    return HBaseUtils.deserializePrimaryKey(dbName, tableName, serialized);
+  }
+
+  /**
+   * Fetch a the foreign keys for a table
+   * @param dbName database the table is in
+   * @param tableName table name
+   * @return All of the foreign key columns thrown together in one list.  Have fun sorting them out.
+   * @throws IOException if there's a read error
+   */
+  List<SQLForeignKey> getForeignKeys(String dbName, String tableName) throws IOException {
+    byte[] key = HBaseUtils.buildKey(dbName, tableName);
+    byte[] serialized = read(TABLE_TABLE, key, CATALOG_CF, FOREIGN_KEY_COL);
+    if (serialized == null) return null;
+    return HBaseUtils.deserializeForeignKeys(dbName, tableName, serialized);
+  }
+
+  /**
+   * Create a primary key on a table.
+   * @param pk Primary key for this table
+   * @throws IOException if unable to write the data to the store.
+   */
+  void putPrimaryKey(List<SQLPrimaryKey> pk) throws IOException {
+    byte[][] serialized = HBaseUtils.serializePrimaryKey(pk);
+    store(TABLE_TABLE, serialized[0], CATALOG_CF, PRIMARY_KEY_COL, serialized[1]);
+  }
+
+  /**
+   * Create one or more foreign keys on a table.  Note that this will not add a foreign key, it
+   * will overwrite whatever is there.  So if you wish to add a key to a table that may already
+   * foreign keys you need to first use {@link #getForeignKeys(String, String)} to fetch the
+   * existing keys, add to the list, and then call this.
+   * @param fks Foreign key(s) for this table
+   * @throws IOException if unable to write the data to the store.
+   */
+  void putForeignKeys(List<SQLForeignKey> fks) throws IOException {
+    byte[][] serialized = HBaseUtils.serializeForeignKeys(fks);
+    store(TABLE_TABLE, serialized[0], CATALOG_CF, FOREIGN_KEY_COL, serialized[1]);
+  }
+
+  /**
+   * Drop the primary key from a table.
+   * @param dbName database the table is in
+   * @param tableName table name
+   * @throws IOException if unable to delete from the store
+   */
+  void deletePrimaryKey(String dbName, String tableName) throws IOException {
+    byte[] key = HBaseUtils.buildKey(dbName, tableName);
+    delete(TABLE_TABLE, key, CATALOG_CF, PRIMARY_KEY_COL);
+  }
+
+  /**
+   * Drop all foreign keys from a table.  Note that this will drop all keys blindly.  You should
+   * only call this if you're sure you want to drop them all.  If you just want to drop one you
+   * should instead all {@link #getForeignKeys(String, String)}, modify the list it returns, and
+   * then call {@link #putForeignKeys(List)}.
+   * @param dbName database the table is in
+   * @param tableName table name
+   * @throws IOException if unable to delete from the store
+   */
+  void deleteForeignKeys(String dbName, String tableName) throws IOException {
+    byte[] key = HBaseUtils.buildKey(dbName, tableName);
+    delete(TABLE_TABLE, key, CATALOG_CF, FOREIGN_KEY_COL);
+  }
+
+  /**********************************************************************************************
    * Cache methods
    *********************************************************************************************/
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0ed01fdf/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index f9fad4c..07cc0da 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.cache.CacheLoader;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hive.common.ObjectPair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -2690,42 +2691,143 @@ public class HBaseStore implements RawStore {
   }
 
   @Override
-  public List<SQLPrimaryKey> getPrimaryKeys(String db_name, String tbl_name)
-    throws MetaException {
-    // TODO: WTF?
-    return null;
+  public List<SQLPrimaryKey> getPrimaryKeys(String db_name, String tbl_name) throws MetaException {
+    boolean commit = false;
+    openTransaction();
+    try {
+      List<SQLPrimaryKey> pk = getHBase().getPrimaryKey(db_name, tbl_name);
+      commit = true;
+      return pk;
+    } catch (IOException e) {
+      LOG.error("Unable to get primary key", e);
+      throw new MetaException("Error reading db " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
   }
 
   @Override
-  public List<SQLForeignKey> getForeignKeys(String parent_db_name,
-    String parent_tbl_name, String foreign_db_name, String foreign_tbl_name)
-    throws MetaException {
-    // TODO: WTF?
-    return null;
+  public List<SQLForeignKey> getForeignKeys(String parent_db_name, String parent_tbl_name,
+                                            String foreign_db_name, String foreign_tbl_name)
+      throws MetaException {
+    boolean commit = false;
+    openTransaction();
+    try {
+      List<SQLForeignKey> fks = getHBase().getForeignKeys(parent_db_name, parent_tbl_name);
+      if (fks == null || fks.size() == 0) return null;
+      List<SQLForeignKey> result = new ArrayList<>(fks.size());
+      for (SQLForeignKey fkcol : fks) {
+        if (fkcol.getFktable_db().equals(parent_db_name) &&
+            fkcol.getFktable_name().equals(parent_tbl_name)) {
+          result.add(fkcol);
+        }
+      }
+      commit = true;
+      return result;
+    } catch (IOException e) {
+      LOG.error("Unable to get foreign key", e);
+      throw new MetaException("Error reading db " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
   }
 
   @Override
-  public void createTableWithConstraints(Table tbl,
-    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
-    throws InvalidObjectException, MetaException {
-    // TODO: WTF?
+  public void createTableWithConstraints(Table tbl, List<SQLPrimaryKey> primaryKeys,
+                                         List<SQLForeignKey> foreignKeys)
+      throws InvalidObjectException, MetaException {
+    boolean commit = false;
+    openTransaction();
+    try {
+      createTable(tbl);
+      if (primaryKeys != null) addPrimaryKeys(primaryKeys);
+      if (foreignKeys != null) addForeignKeys(foreignKeys);
+      commit = true;
+    } finally {
+      commitOrRoleBack(commit);
+    }
   }
 
   @Override
-  public void dropConstraint(String dbName, String tableName,
-    String constraintName) throws NoSuchObjectException {
-    // TODO: WTF?
+  public void dropConstraint(String dbName, String tableName, String constraintName)
+      throws NoSuchObjectException {
+    // This is something of pain, since we have to search both primary key and foreign key to see
+    // which they want to drop.
+    boolean commit = false;
+    openTransaction();
+    try {
+      List<SQLPrimaryKey> pk = getHBase().getPrimaryKey(dbName, tableName);
+      if (pk != null && pk.size() > 0 && pk.get(0).getPk_name().equals(constraintName)) {
+        getHBase().deletePrimaryKey(dbName, tableName);
+        commit = true;
+        return;
+      }
+
+      List<SQLForeignKey> fks = getHBase().getForeignKeys(dbName, tableName);
+      if (fks != null && fks.size() > 0) {
+        List<SQLForeignKey> newKeyList = new ArrayList<>(fks.size());
+        // Make a new list of keys that excludes all columns from the constraint we're dropping.
+        for (SQLForeignKey fkcol : fks) {
+          if (!fkcol.getFk_name().equals(constraintName)) newKeyList.add(fkcol);
+        }
+        // If we've dropped only one foreign key out of many keys, than update so that we still
+        // have the existing keys.  Otherwise drop the foreign keys all together.
+        if (newKeyList.size() > 0) getHBase().putForeignKeys(newKeyList);
+        else getHBase().deleteForeignKeys(dbName, tableName);
+        commit = true;
+        return;
+      }
+
+      commit = true;
+      throw new NoSuchObjectException("Unable to find constraint named " + constraintName +
+        " on table " + tableNameForErrorMsg(dbName, tableName));
+    } catch (IOException e) {
+      LOG.error("Error fetching primary key for table " + tableNameForErrorMsg(dbName, tableName), e);
+      throw new NoSuchObjectException("Error fetching primary key for table " +
+          tableNameForErrorMsg(dbName, tableName) + " : " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
   }
 
   @Override
-  public void addPrimaryKeys(List<SQLPrimaryKey> pks)
-    throws InvalidObjectException, MetaException {
-    // TODO: WTF?
+  public void addPrimaryKeys(List<SQLPrimaryKey> pks) throws InvalidObjectException, MetaException {
+    boolean commit = false;
+    openTransaction();
+    try {
+      List<SQLPrimaryKey> currentPk =
+          getHBase().getPrimaryKey(pks.get(0).getTable_db(), pks.get(0).getTable_name());
+      if (currentPk != null) {
+        throw new MetaException(" Primary key already exists for: " +
+            tableNameForErrorMsg(pks.get(0).getTable_db(), pks.get(0).getTable_name()));
+      }
+      getHBase().putPrimaryKey(pks);
+      commit = true;
+    } catch (IOException e) {
+      LOG.error("Error writing primary key", e);
+      throw new MetaException("Error writing primary key: " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
   }
 
   @Override
-  public void addForeignKeys(List<SQLForeignKey> fks)
-    throws InvalidObjectException, MetaException {
-    // TODO: WTF?
+  public void addForeignKeys(List<SQLForeignKey> fks) throws InvalidObjectException, MetaException {
+    boolean commit = false;
+    openTransaction();
+    try {
+      // Fetch the existing keys (if any) and add in these new ones
+      List<SQLForeignKey> existing =
+          getHBase().getForeignKeys(fks.get(0).getFktable_db(), fks.get(0).getFktable_name());
+      if (existing == null) existing = new ArrayList<>(fks.size());
+      existing.addAll(fks);
+      getHBase().putForeignKeys(existing);
+      commit = true;
+    } catch (IOException e) {
+      LOG.error("Error writing foreign keys", e);
+      throw new MetaException("Error writing foreign keys: " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/0ed01fdf/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
index 54daa4a..4546d43 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
@@ -37,6 +37,7 @@ import java.util.TreeSet;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
@@ -61,6 +62,8 @@ import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
 import org.apache.hadoop.hive.metastore.api.ResourceType;
 import org.apache.hadoop.hive.metastore.api.ResourceUri;
 import org.apache.hadoop.hive.metastore.api.Role;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -360,6 +363,7 @@ public class HBaseUtils {
     return result;
   }
 
+
   /**
    * Deserialize a database.  This method should be used when the db anme is already known as it
    * doesn't have to re-deserialize it.
@@ -1495,6 +1499,123 @@ public class HBaseUtils {
   }
 
   /**
+   * Serialize the primary key for a table.
+   * @param pk Primary key columns.  It is expected that all of these match to one pk, since
+   *           anything else is meaningless.
+   * @return two byte arrays, first containts the hbase key, the second the serialized value.
+   */
+  static byte[][] serializePrimaryKey(List<SQLPrimaryKey> pk) {
+    // First, figure out the dbName and tableName.  We expect this to match for all list entries.
+    byte[][] result = new byte[2][];
+    String dbName = pk.get(0).getTable_db();
+    String tableName = pk.get(0).getTable_name();
+    result[0] = buildKey(HiveStringUtils.normalizeIdentifier(dbName),
+        HiveStringUtils.normalizeIdentifier(tableName));
+
+    HbaseMetastoreProto.PrimaryKey.Builder builder = HbaseMetastoreProto.PrimaryKey.newBuilder();
+    // Encode the primary key, if present
+    builder.setPkName(pk.get(0).getPk_name());
+    builder.setEnableConstraint(pk.get(0).isEnable_cstr());
+    builder.setValidateConstraint(pk.get(0).isValidate_cstr());
+    builder.setRelyConstraint(pk.get(0).isRely_cstr());
+
+    for (SQLPrimaryKey pkcol : pk) {
+      HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.Builder pkColBuilder =
+          HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.newBuilder();
+      pkColBuilder.setColumnName(pkcol.getColumn_name());
+      pkColBuilder.setKeySeq(pkcol.getKey_seq());
+      builder.addCols(pkColBuilder);
+    }
+
+    result[1] = builder.build().toByteArray();
+    return result;
+  }
+
+  /**
+   * Serialize the foreign key(s) for a table.
+   * @param fks Foreign key columns.  These may belong to multiple foreign keys.
+   * @return two byte arrays, first containts the key, the second the serialized value.
+   */
+  static byte[][] serializeForeignKeys(List<SQLForeignKey> fks) {
+    // First, figure out the dbName and tableName.  We expect this to match for all list entries.
+    byte[][] result = new byte[2][];
+    String dbName = fks.get(0).getFktable_db();
+    String tableName = fks.get(0).getFktable_name();
+    result[0] = buildKey(HiveStringUtils.normalizeIdentifier(dbName),
+        HiveStringUtils.normalizeIdentifier(tableName));
+
+    HbaseMetastoreProto.ForeignKeys.Builder builder = HbaseMetastoreProto.ForeignKeys.newBuilder();
+
+    // Encode any foreign keys we find.  This can be complex because there may be more than
+    // one foreign key in here, so we need to detect that.
+    Map<String, HbaseMetastoreProto.ForeignKeys.ForeignKey.Builder> fkBuilders = new HashMap<>();
+
+    for (SQLForeignKey fkcol : fks) {
+      HbaseMetastoreProto.ForeignKeys.ForeignKey.Builder fkBuilder =
+          fkBuilders.get(fkcol.getFk_name());
+      if (fkBuilder == null) {
+        // We haven't seen this key before, so add it
+        fkBuilder = HbaseMetastoreProto.ForeignKeys.ForeignKey.newBuilder();
+        fkBuilder.setFkName(fkcol.getFk_name());
+        fkBuilder.setReferencedDbName(fkcol.getPktable_db());
+        assert dbName.equals(fkcol.getFktable_db()) : "You switched databases on me!";
+        fkBuilder.setReferencedTableName(fkcol.getPktable_name());
+        assert tableName.equals(fkcol.getFktable_name()) : "You switched tables on me!";
+        fkBuilder.setReferencedPkName(fkcol.getPk_name());
+        fkBuilder.setUpdateRule(fkcol.getUpdate_rule());
+        fkBuilder.setDeleteRule(fkcol.getDelete_rule());
+        fkBuilder.setEnableConstraint(fkcol.isEnable_cstr());
+        fkBuilder.setValidateConstraint(fkcol.isValidate_cstr());
+        fkBuilder.setRelyConstraint(fkcol.isRely_cstr());
+        fkBuilders.put(fkcol.getFk_name(), fkBuilder);
+      }
+      HbaseMetastoreProto.ForeignKeys.ForeignKey.ForeignKeyColumn.Builder fkColBuilder =
+          HbaseMetastoreProto.ForeignKeys.ForeignKey.ForeignKeyColumn.newBuilder();
+      fkColBuilder.setColumnName(fkcol.getFkcolumn_name());
+      fkColBuilder.setReferencedColumnName(fkcol.getPkcolumn_name());
+      fkColBuilder.setKeySeq(fkcol.getKey_seq());
+      fkBuilder.addCols(fkColBuilder);
+    }
+    for (HbaseMetastoreProto.ForeignKeys.ForeignKey.Builder fkBuilder : fkBuilders.values()) {
+      builder.addFks(fkBuilder);
+    }
+    result[1] = builder.build().toByteArray();
+    return result;
+  }
+
+  static List<SQLPrimaryKey> deserializePrimaryKey(String dbName, String tableName, byte[] value)
+      throws InvalidProtocolBufferException {
+    HbaseMetastoreProto.PrimaryKey proto = HbaseMetastoreProto.PrimaryKey.parseFrom(value);
+    List<SQLPrimaryKey> result = new ArrayList<>();
+    for (HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn protoPkCol : proto.getColsList()) {
+      result.add(new SQLPrimaryKey(dbName, tableName, protoPkCol.getColumnName(),
+          protoPkCol.getKeySeq(), proto.getPkName(), proto.getEnableConstraint(),
+          proto.getValidateConstraint(), proto.getRelyConstraint()));
+    }
+
+    return result;
+  }
+
+  static List<SQLForeignKey> deserializeForeignKeys(String dbName, String tableName, byte[] value)
+      throws InvalidProtocolBufferException {
+    List<SQLForeignKey> result = new ArrayList<>();
+    HbaseMetastoreProto.ForeignKeys protoConstraints =
+        HbaseMetastoreProto.ForeignKeys.parseFrom(value);
+
+    for (HbaseMetastoreProto.ForeignKeys.ForeignKey protoFk : protoConstraints.getFksList()) {
+      for (HbaseMetastoreProto.ForeignKeys.ForeignKey.ForeignKeyColumn protoFkCol :
+          protoFk.getColsList()) {
+        result.add(new SQLForeignKey(protoFk.getReferencedDbName(), protoFk.getReferencedTableName(),
+            protoFkCol.getReferencedColumnName(), dbName, tableName, protoFkCol.getColumnName(),
+            protoFkCol.getKeySeq(), protoFk.getUpdateRule(), protoFk.getDeleteRule(),
+            protoFk.getFkName(), protoFk.getReferencedPkName(), protoFk.getEnableConstraint(),
+            protoFk.getValidateConstraint(), protoFk.getRelyConstraint()));
+      }
+    }
+    return result;
+  }
+
+  /**
    * @param keyStart byte array representing the start prefix
    * @return byte array corresponding to the next possible prefix
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/0ed01fdf/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
----------------------------------------------------------------------
diff --git a/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto b/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
index 6fbe36c..3f9e4c5 100644
--- a/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
+++ b/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
@@ -295,3 +295,39 @@ message PartitionKeyComparator {
   repeated Operator op = 3;
   repeated Range range = 4;
 }
+
+message PrimaryKey {
+  message PrimaryKeyColumn {
+    required string column_name = 1;
+    required sint32 key_seq = 2;
+  }
+
+  required string pk_name = 1;
+  repeated PrimaryKeyColumn cols = 2;
+  optional bool enable_constraint = 3;
+  optional bool validate_constraint = 4;
+  optional bool rely_constraint = 5;
+}
+
+message ForeignKeys {
+  message ForeignKey {
+    message ForeignKeyColumn {
+      required string column_name = 1;
+      required string referenced_column_name = 2;
+      required sint32 key_seq = 3;
+    }
+
+    required string fk_name = 1;
+    required string referenced_db_name = 2;
+    required string referenced_table_name = 3;
+    optional string referenced_pk_name = 4;
+    optional int32 update_rule = 5;
+    optional int32 delete_rule = 6;
+    repeated ForeignKeyColumn cols = 7;
+    optional bool enable_constraint = 8;
+    optional bool validate_constraint = 9;
+    optional bool rely_constraint = 10;
+  }
+
+  repeated ForeignKey fks = 1;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/0ed01fdf/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
index 4894ed3..fb0a8e7 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
@@ -22,10 +22,12 @@ import java.io.IOException;
 import java.security.MessageDigest;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedMap;
+import java.util.SortedSet;
 import java.util.TreeMap;
 
 import org.apache.hadoop.hbase.Cell;
@@ -46,6 +48,7 @@ import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.FunctionType;
 import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.Partition;
@@ -53,6 +56,8 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.ResourceType;
 import org.apache.hadoop.hive.metastore.api.ResourceUri;
 import org.apache.hadoop.hive.metastore.api.Role;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -62,6 +67,7 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -1391,6 +1397,366 @@ public class TestHBaseStore {
     Assert.assertEquals(decimalData.getNumDVs(), decimalDataFromDB.getNumDVs());
   }
 
+  @Test
+  public void createTableWithPrimaryKey() throws Exception {
+    String tableName = "pktable";
+    String pkName = "test_pk";
+    String pkColNames[] = { "col0" };
+    Table table = createMultiColumnTable(tableName, "int");
+
+    List<SQLPrimaryKey> pk = Arrays.asList(
+        new SQLPrimaryKey(DB, tableName, pkColNames[0], 0, pkName, true, false, true));
+
+    store.createTableWithConstraints(table, pk, null);
+
+    pk = store.getPrimaryKeys(DB, tableName);
+
+    Assert.assertNotNull(pk);
+    Assert.assertEquals(1, pk.size());
+    Assert.assertEquals(DB, pk.get(0).getTable_db());
+    Assert.assertEquals(tableName, pk.get(0).getTable_name());
+    Assert.assertEquals(pkColNames[0], pk.get(0).getColumn_name());
+    Assert.assertEquals(0, pk.get(0).getKey_seq());
+    Assert.assertEquals(pkName, pk.get(0).getPk_name());
+    Assert.assertTrue(pk.get(0).isEnable_cstr());
+    Assert.assertFalse(pk.get(0).isValidate_cstr());
+    Assert.assertTrue(pk.get(0).isRely_cstr());
+
+    // Drop the primary key
+    store.dropConstraint(DB, tableName, pkName);
+
+    pk = store.getPrimaryKeys(DB, tableName);
+    Assert.assertNull(pk);
+  }
+
+  @Test
+  public void createTableWithForeignKey() throws Exception {
+    String tableName = "fktable";
+    String pkTable = "pktable";
+    String pkName = "test_pk";
+    String fkName = "test_fk";
+    String fkColNames[] = { "col0" };
+    String pkColNames[] = { "pcol0" };
+    Table table = createMultiColumnTable(tableName, "int");
+
+    List<SQLForeignKey> fk = Arrays.asList(
+        new SQLForeignKey(DB, pkTable, pkColNames[0], DB, tableName, fkColNames[0], 0, 1, 2,
+            fkName, pkName, true, false, false));
+
+    store.createTableWithConstraints(table, null, fk);
+
+    fk = store.getForeignKeys(DB, tableName, DB, pkTable);
+
+    Assert.assertNotNull(fk);
+    Assert.assertEquals(1, fk.size());
+    Assert.assertEquals(DB, fk.get(0).getPktable_db());
+    Assert.assertEquals(pkTable, fk.get(0).getPktable_name());
+    Assert.assertEquals(pkColNames[0], fk.get(0).getPkcolumn_name());
+    Assert.assertEquals(DB, fk.get(0).getFktable_db());
+    Assert.assertEquals(tableName, fk.get(0).getFktable_name());
+    Assert.assertEquals(fkColNames[0], fk.get(0).getFkcolumn_name());
+    Assert.assertEquals(0, fk.get(0).getKey_seq());
+    Assert.assertEquals(1, fk.get(0).getUpdate_rule());
+    Assert.assertEquals(2, fk.get(0).getDelete_rule());
+    Assert.assertEquals(fkName, fk.get(0).getFk_name());
+    Assert.assertEquals(pkName, fk.get(0).getPk_name());
+    Assert.assertTrue(fk.get(0).isEnable_cstr());
+    Assert.assertFalse(fk.get(0).isValidate_cstr());
+    Assert.assertFalse(fk.get(0).isRely_cstr());
+  }
+
+  // Test that we can add a primary key with multiple columns
+  @Test
+  public void addMultiColPrimaryKey() throws Exception {
+    String tableName = "mcpktable";
+    String pkName = "test_pk";
+    String pkColNames[] = { "col0", "col1", "col2" };
+    Table table = createMultiColumnTable(tableName, "int", "varchar(32)", "decimal(10,2)");
+
+    List<SQLPrimaryKey> pk = Arrays.asList(
+        new SQLPrimaryKey(DB, tableName, pkColNames[1], 0, pkName, false, true, true),
+        new SQLPrimaryKey(DB, tableName, pkColNames[2], 1, pkName, false, true, true)
+    );
+
+    store.createTable(table);
+    store.addPrimaryKeys(pk);
+
+    Assert.assertNotNull(pk);
+    Assert.assertEquals(2, pk.size());
+    SQLPrimaryKey[] sorted = pk.toArray(new SQLPrimaryKey[2]);
+    Arrays.sort(sorted, new Comparator<SQLPrimaryKey>() {
+      @Override
+      public int compare(SQLPrimaryKey o1, SQLPrimaryKey o2) {
+        return o1.getColumn_name().compareTo(o2.getColumn_name());
+      }
+    });
+    for (int i = 0; i < 2; i++) {
+      Assert.assertEquals(DB, sorted[i].getTable_db());
+      Assert.assertEquals(tableName, sorted[i].getTable_name());
+      Assert.assertEquals(pkColNames[i+1], sorted[i].getColumn_name());
+      Assert.assertEquals(i, sorted[i].getKey_seq());
+      Assert.assertEquals(pkName, sorted[i].getPk_name());
+      Assert.assertFalse(sorted[i].isEnable_cstr());
+      Assert.assertTrue(sorted[i].isValidate_cstr());
+      Assert.assertTrue(sorted[i].isRely_cstr());
+    }
+
+  }
+
+  // Test that we can create a foreign key with multiple columns
+  @Test
+  public void addMultiColForeignKey() throws Exception {
+    String tableName = "mcfktable";
+    String pkTable = "pktable";
+    String pkName = "test_pk";
+    String fkName = "test_fk";
+    String fkColNames[] = { "col0", "col1", "col2" };
+    String pkColNames[] = { "pcol0", "pcol1" };
+    Table table = createMultiColumnTable(tableName, "int", "double", "timestamp");
+
+    List<SQLForeignKey> fk = Arrays.asList(
+        new SQLForeignKey(DB, pkTable, pkColNames[0], DB, tableName, fkColNames[1], 0, 1, 2,
+            fkName, pkName, true, false, false),
+        new SQLForeignKey(DB, pkTable, pkColNames[1], DB, tableName, fkColNames[2], 1, 1, 2,
+            fkName, pkName, true, false, false)
+        );
+
+    store.createTable(table);
+    store.addForeignKeys(fk);
+
+    fk = store.getForeignKeys(DB, tableName, DB, pkTable);
+
+    Assert.assertNotNull(fk);
+    Assert.assertEquals(2, fk.size());
+    SQLForeignKey[] sorted = fk.toArray(new SQLForeignKey[2]);
+    Arrays.sort(sorted, new Comparator<SQLForeignKey>() {
+      @Override
+      public int compare(SQLForeignKey o1, SQLForeignKey o2) {
+        if (o1.getFk_name().equals(o2.getFk_name())) {
+          return o1.getFkcolumn_name().compareTo(o2.getFkcolumn_name());
+        } else {
+          return o1.getFk_name().compareTo(o2.getFk_name());
+        }
+      }
+    });
+
+    for (int i = 0; i < 2; i++) {
+      Assert.assertEquals(DB, sorted[i].getPktable_db());
+      Assert.assertEquals(pkTable, sorted[i].getPktable_name());
+      Assert.assertEquals(pkColNames[i], sorted[i].getPkcolumn_name());
+      Assert.assertEquals(DB, sorted[i].getFktable_db());
+      Assert.assertEquals(tableName, sorted[i].getFktable_name());
+      Assert.assertEquals(fkColNames[i+1], sorted[i].getFkcolumn_name());
+      Assert.assertEquals(i, sorted[i].getKey_seq());
+      Assert.assertEquals(1, sorted[i].getUpdate_rule());
+      Assert.assertEquals(2, sorted[i].getDelete_rule());
+      Assert.assertEquals(fkName, sorted[i].getFk_name());
+      Assert.assertEquals(pkName, sorted[i].getPk_name());
+      Assert.assertTrue(sorted[i].isEnable_cstr());
+      Assert.assertFalse(sorted[i].isValidate_cstr());
+      Assert.assertFalse(sorted[i].isRely_cstr());
+    }
+
+  }
+
+  // Test that we can add 2 foreign keys at once
+  @Test
+  public void addMultiForeignKeys() throws Exception {
+    String tableName = "mcfktable";
+    String pkTable = "pktable";
+    String pkTable2 = "pktable2";
+    String pkName = "test_pk";
+    String pkName2 = "test_pk2";
+    String fkName = "test_fk";
+    String fkName2 = "test_fk2";
+    String fkColNames[] = { "col0", "col1", "col2" };
+    String pkColNames[] = { "pcol0", "pcol1" };
+    String pkColNames2[] = { "p2col0" };
+    Table table = createMultiColumnTable(tableName, "int", "double", "timestamp");
+
+    List<SQLForeignKey> fk = Arrays.asList(
+        new SQLForeignKey(DB, pkTable, pkColNames[0], DB, tableName, fkColNames[1], 0, 1, 2,
+            fkName, pkName, true, false, true),
+        new SQLForeignKey(DB, pkTable, pkColNames[1], DB, tableName, fkColNames[2], 1, 1, 2,
+            fkName, pkName, true, false, true),
+        new SQLForeignKey(DB, pkTable2, pkColNames2[0], DB, tableName, fkColNames[0], 0, 1, 2,
+            fkName2, pkName2, true, false, true)
+    );
+
+    store.createTable(table);
+    store.addForeignKeys(fk);
+
+    fk = store.getForeignKeys(DB, tableName, DB, pkTable);
+
+    Assert.assertNotNull(fk);
+    Assert.assertEquals(3, fk.size());
+    SQLForeignKey[] sorted = fk.toArray(new SQLForeignKey[2]);
+    Arrays.sort(sorted, new Comparator<SQLForeignKey>() {
+      @Override
+      public int compare(SQLForeignKey o1, SQLForeignKey o2) {
+        if (o1.getFk_name().equals(o2.getFk_name())) {
+          return o1.getFkcolumn_name().compareTo(o2.getFkcolumn_name());
+        } else {
+          return o1.getFk_name().compareTo(o2.getFk_name());
+        }
+      }
+    });
+
+    for (int i = 0; i < 2; i++) {
+      Assert.assertEquals(DB, sorted[i].getPktable_db());
+      Assert.assertEquals(pkTable, sorted[i].getPktable_name());
+      Assert.assertEquals(pkColNames[i], sorted[i].getPkcolumn_name());
+      Assert.assertEquals(DB, sorted[i].getFktable_db());
+      Assert.assertEquals(tableName, sorted[i].getFktable_name());
+      Assert.assertEquals(fkColNames[i+1], sorted[i].getFkcolumn_name());
+      Assert.assertEquals(i, sorted[i].getKey_seq());
+      Assert.assertEquals(1, sorted[i].getUpdate_rule());
+      Assert.assertEquals(2, sorted[i].getDelete_rule());
+      Assert.assertEquals(fkName, sorted[i].getFk_name());
+      Assert.assertEquals(pkName, sorted[i].getPk_name());
+      Assert.assertTrue(sorted[i].isEnable_cstr());
+      Assert.assertFalse(sorted[i].isValidate_cstr());
+      Assert.assertTrue(sorted[i].isRely_cstr());
+    }
+    Assert.assertEquals(DB, sorted[2].getPktable_db());
+    Assert.assertEquals(pkTable2, sorted[2].getPktable_name());
+    Assert.assertEquals(pkColNames2[0], sorted[2].getPkcolumn_name());
+    Assert.assertEquals(DB, sorted[2].getFktable_db());
+    Assert.assertEquals(tableName, sorted[2].getFktable_name());
+    Assert.assertEquals(fkColNames[0], sorted[2].getFkcolumn_name());
+    Assert.assertEquals(0, sorted[2].getKey_seq());
+    Assert.assertEquals(1, sorted[2].getUpdate_rule());
+    Assert.assertEquals(2, sorted[2].getDelete_rule());
+    Assert.assertEquals(fkName2, sorted[2].getFk_name());
+    Assert.assertEquals(pkName2, sorted[2].getPk_name());
+    Assert.assertTrue(sorted[2].isEnable_cstr());
+    Assert.assertFalse(sorted[2].isValidate_cstr());
+    Assert.assertTrue(sorted[2].isRely_cstr());
+
+  }
+
+  // Test that we can add a foreign key when one already exists
+  @Test
+  public void addSecondForeignKeys() throws Exception {
+    String tableName = "mcfktable";
+    String pkTable = "pktable";
+    String pkTable2 = "pktable2";
+    String pkName = "test_pk";
+    String pkName2 = "test_pk2";
+    String fkName = "test_fk";
+    String fkName2 = "test_fk2";
+    String fkColNames[] = { "col0", "col1", "col2" };
+    String pkColNames[] = { "pcol0", "pcol1" };
+    String pkColNames2[] = { "p2col0" };
+    Table table = createMultiColumnTable(tableName, "int", "double", "timestamp");
+
+    List<SQLForeignKey> fk = Arrays.asList(
+        new SQLForeignKey(DB, pkTable, pkColNames[0], DB, tableName, fkColNames[1], 0, 1, 2,
+            fkName, pkName, true, false, true),
+        new SQLForeignKey(DB, pkTable, pkColNames[1], DB, tableName, fkColNames[2], 1, 1, 2,
+            fkName, pkName, true, false, true)
+    );
+
+    store.createTable(table);
+    store.addForeignKeys(fk);
+
+    fk = Arrays.asList(
+        new SQLForeignKey(DB, pkTable2, pkColNames2[0], DB, tableName, fkColNames[0], 0, 1, 2,
+            fkName2, pkName2, true, false, true)
+    );
+    store.addForeignKeys(fk);
+
+    fk = store.getForeignKeys(DB, tableName, DB, pkTable);
+
+    Assert.assertNotNull(fk);
+    Assert.assertEquals(3, fk.size());
+    SQLForeignKey[] sorted = fk.toArray(new SQLForeignKey[2]);
+    Arrays.sort(sorted, new Comparator<SQLForeignKey>() {
+      @Override
+      public int compare(SQLForeignKey o1, SQLForeignKey o2) {
+        if (o1.getFk_name().equals(o2.getFk_name())) {
+          return o1.getFkcolumn_name().compareTo(o2.getFkcolumn_name());
+        } else {
+          return o1.getFk_name().compareTo(o2.getFk_name());
+        }
+      }
+    });
+
+    for (int i = 0; i < 2; i++) {
+      Assert.assertEquals(DB, sorted[i].getPktable_db());
+      Assert.assertEquals(pkTable, sorted[i].getPktable_name());
+      Assert.assertEquals(pkColNames[i], sorted[i].getPkcolumn_name());
+      Assert.assertEquals(DB, sorted[i].getFktable_db());
+      Assert.assertEquals(tableName, sorted[i].getFktable_name());
+      Assert.assertEquals(fkColNames[i+1], sorted[i].getFkcolumn_name());
+      Assert.assertEquals(i, sorted[i].getKey_seq());
+      Assert.assertEquals(1, sorted[i].getUpdate_rule());
+      Assert.assertEquals(2, sorted[i].getDelete_rule());
+      Assert.assertEquals(fkName, sorted[i].getFk_name());
+      Assert.assertEquals(pkName, sorted[i].getPk_name());
+      Assert.assertTrue(sorted[i].isEnable_cstr());
+      Assert.assertFalse(sorted[i].isValidate_cstr());
+      Assert.assertTrue(sorted[i].isRely_cstr());
+    }
+    Assert.assertEquals(DB, sorted[2].getPktable_db());
+    Assert.assertEquals(pkTable2, sorted[2].getPktable_name());
+    Assert.assertEquals(pkColNames2[0], sorted[2].getPkcolumn_name());
+    Assert.assertEquals(DB, sorted[2].getFktable_db());
+    Assert.assertEquals(tableName, sorted[2].getFktable_name());
+    Assert.assertEquals(fkColNames[0], sorted[2].getFkcolumn_name());
+    Assert.assertEquals(0, sorted[2].getKey_seq());
+    Assert.assertEquals(1, sorted[2].getUpdate_rule());
+    Assert.assertEquals(2, sorted[2].getDelete_rule());
+    Assert.assertEquals(fkName2, sorted[2].getFk_name());
+    Assert.assertEquals(pkName2, sorted[2].getPk_name());
+    Assert.assertTrue(sorted[2].isEnable_cstr());
+    Assert.assertFalse(sorted[2].isValidate_cstr());
+    Assert.assertTrue(sorted[2].isRely_cstr());
+
+    store.dropConstraint(DB, tableName, fkName);
+
+    fk = store.getForeignKeys(DB, tableName, DB, pkTable);
+    Assert.assertNotNull(fk);
+    Assert.assertEquals(1, fk.size());
+    Assert.assertEquals(DB, fk.get(0).getPktable_db());
+    Assert.assertEquals(pkTable2, fk.get(0).getPktable_name());
+    Assert.assertEquals(pkColNames2[0], fk.get(0).getPkcolumn_name());
+    Assert.assertEquals(DB, fk.get(0).getFktable_db());
+    Assert.assertEquals(tableName, fk.get(0).getFktable_name());
+    Assert.assertEquals(fkColNames[0], fk.get(0).getFkcolumn_name());
+    Assert.assertEquals(0, fk.get(0).getKey_seq());
+    Assert.assertEquals(1, fk.get(0).getUpdate_rule());
+    Assert.assertEquals(2, fk.get(0).getDelete_rule());
+    Assert.assertEquals(fkName2, fk.get(0).getFk_name());
+    Assert.assertEquals(pkName2, fk.get(0).getPk_name());
+    Assert.assertTrue(fk.get(0).isEnable_cstr());
+    Assert.assertFalse(fk.get(0).isValidate_cstr());
+    Assert.assertTrue(fk.get(0).isRely_cstr());
+
+    store.dropConstraint(DB, tableName, fkName2);
+
+    fk = store.getForeignKeys(DB, tableName, DB, pkTable);
+    Assert.assertNull(fk);
+  }
+
+  // Try adding a primary key when one already exists
+  @Test(expected= MetaException.class)
+  public void doublePrimaryKey() throws Exception {
+    String tableName = "pktable";
+    String pkName = "test_pk";
+    String pkColNames[] = { "col0" };
+    Table table = createMultiColumnTable(tableName, "int");
+
+    List<SQLPrimaryKey> pk = Arrays.asList(
+        new SQLPrimaryKey(DB, tableName, pkColNames[0], 0, pkName, true, false, true));
+
+    store.createTableWithConstraints(table, pk, null);
+
+    store.addPrimaryKeys(pk);
+  }
+
+
+
   private Table createMockTableAndPartition(String partType, String partVal) throws Exception {
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
     cols.add(new FieldSchema("col1", partType, ""));
@@ -1425,6 +1791,22 @@ public class TestHBaseStore {
     store.createTable(table);
     return table;
   }
+
+  private Table createMultiColumnTable(String tblName, String... types) throws Exception {
+    List<FieldSchema> cols = new ArrayList<FieldSchema>();
+    for (int i = 0; i < types.length; i++) cols.add(new FieldSchema("col" + i, types[i], ""));
+    SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
+    Map<String, String> params = new HashMap<String, String>();
+    params.put("key", "value");
+    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17,
+        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
+    int currentTime = (int)(System.currentTimeMillis() / 1000);
+    Table table = new Table(tblName, DB, "me", currentTime, currentTime, 0, sd, cols,
+        emptyParameters, null, null, null);
+    store.createTable(table);
+    return table;
+  }
+
   /**
    * Returns a dummy table level ColumnStatisticsDesc with default values
    */


[3/3] hive git commit: HIVE-15342 Add support for primary/foreign keys in HBase metastore (Alan Gates, reviewed by Daniel Dai)

Posted by ga...@apache.org.
HIVE-15342 Add support for primary/foreign keys in HBase metastore (Alan Gates, reviewed by Daniel Dai)


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

Branch: refs/heads/master
Commit: 0ed01fdf85fc7b4ecee4090eae67e42d2a84421c
Parents: d61c60b
Author: Alan Gates <ga...@hortonworks.com>
Authored: Thu Dec 8 17:06:51 2016 -0800
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Thu Dec 8 17:06:51 2016 -0800

----------------------------------------------------------------------
 .../metastore/hbase/HbaseMetastoreProto.java    | 4903 +++++++++++++++++-
 .../hive/metastore/hbase/HBaseReadWrite.java    |  100 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |  146 +-
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |  121 +
 .../metastore/hbase/hbase_metastore_proto.proto |   36 +
 .../hive/metastore/hbase/TestHBaseStore.java    |  382 ++
 6 files changed, 5664 insertions(+), 24 deletions(-)
----------------------------------------------------------------------



[2/3] hive git commit: HIVE-15342 Add support for primary/foreign keys in HBase metastore (Alan Gates, reviewed by Daniel Dai)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0ed01fdf/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
index 9f2a88c..b15b0de 100644
--- a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
+++ b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
@@ -36396,6 +36396,4830 @@ public final class HbaseMetastoreProto {
     // @@protoc_insertion_point(class_scope:org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator)
   }
 
+  public interface PrimaryKeyOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string pk_name = 1;
+    /**
+     * <code>required string pk_name = 1;</code>
+     */
+    boolean hasPkName();
+    /**
+     * <code>required string pk_name = 1;</code>
+     */
+    java.lang.String getPkName();
+    /**
+     * <code>required string pk_name = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getPkNameBytes();
+
+    // repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+     */
+    java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn> 
+        getColsList();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+     */
+    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn getCols(int index);
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+     */
+    int getColsCount();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumnOrBuilder> 
+        getColsOrBuilderList();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+     */
+    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumnOrBuilder getColsOrBuilder(
+        int index);
+
+    // optional bool enable_constraint = 3;
+    /**
+     * <code>optional bool enable_constraint = 3;</code>
+     */
+    boolean hasEnableConstraint();
+    /**
+     * <code>optional bool enable_constraint = 3;</code>
+     */
+    boolean getEnableConstraint();
+
+    // optional bool validate_constraint = 4;
+    /**
+     * <code>optional bool validate_constraint = 4;</code>
+     */
+    boolean hasValidateConstraint();
+    /**
+     * <code>optional bool validate_constraint = 4;</code>
+     */
+    boolean getValidateConstraint();
+
+    // optional bool rely_constraint = 5;
+    /**
+     * <code>optional bool rely_constraint = 5;</code>
+     */
+    boolean hasRelyConstraint();
+    /**
+     * <code>optional bool rely_constraint = 5;</code>
+     */
+    boolean getRelyConstraint();
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.PrimaryKey}
+   */
+  public static final class PrimaryKey extends
+      com.google.protobuf.GeneratedMessage
+      implements PrimaryKeyOrBuilder {
+    // Use PrimaryKey.newBuilder() to construct.
+    private PrimaryKey(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private PrimaryKey(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final PrimaryKey defaultInstance;
+    public static PrimaryKey getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public PrimaryKey getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private PrimaryKey(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              pkName_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                cols_ = new java.util.ArrayList<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn>();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              cols_.add(input.readMessage(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.PARSER, extensionRegistry));
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000002;
+              enableConstraint_ = input.readBool();
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000004;
+              validateConstraint_ = input.readBool();
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000008;
+              relyConstraint_ = input.readBool();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          cols_ = java.util.Collections.unmodifiableList(cols_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PrimaryKey_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PrimaryKey_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<PrimaryKey> PARSER =
+        new com.google.protobuf.AbstractParser<PrimaryKey>() {
+      public PrimaryKey parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new PrimaryKey(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<PrimaryKey> getParserForType() {
+      return PARSER;
+    }
+
+    public interface PrimaryKeyColumnOrBuilder
+        extends com.google.protobuf.MessageOrBuilder {
+
+      // required string column_name = 1;
+      /**
+       * <code>required string column_name = 1;</code>
+       */
+      boolean hasColumnName();
+      /**
+       * <code>required string column_name = 1;</code>
+       */
+      java.lang.String getColumnName();
+      /**
+       * <code>required string column_name = 1;</code>
+       */
+      com.google.protobuf.ByteString
+          getColumnNameBytes();
+
+      // required sint32 key_seq = 2;
+      /**
+       * <code>required sint32 key_seq = 2;</code>
+       */
+      boolean hasKeySeq();
+      /**
+       * <code>required sint32 key_seq = 2;</code>
+       */
+      int getKeySeq();
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn}
+     */
+    public static final class PrimaryKeyColumn extends
+        com.google.protobuf.GeneratedMessage
+        implements PrimaryKeyColumnOrBuilder {
+      // Use PrimaryKeyColumn.newBuilder() to construct.
+      private PrimaryKeyColumn(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+        super(builder);
+        this.unknownFields = builder.getUnknownFields();
+      }
+      private PrimaryKeyColumn(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+      private static final PrimaryKeyColumn defaultInstance;
+      public static PrimaryKeyColumn getDefaultInstance() {
+        return defaultInstance;
+      }
+
+      public PrimaryKeyColumn getDefaultInstanceForType() {
+        return defaultInstance;
+      }
+
+      private final com.google.protobuf.UnknownFieldSet unknownFields;
+      @java.lang.Override
+      public final com.google.protobuf.UnknownFieldSet
+          getUnknownFields() {
+        return this.unknownFields;
+      }
+      private PrimaryKeyColumn(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        initFields();
+        int mutable_bitField0_ = 0;
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            com.google.protobuf.UnknownFieldSet.newBuilder();
+        try {
+          boolean done = false;
+          while (!done) {
+            int tag = input.readTag();
+            switch (tag) {
+              case 0:
+                done = true;
+                break;
+              default: {
+                if (!parseUnknownField(input, unknownFields,
+                                       extensionRegistry, tag)) {
+                  done = true;
+                }
+                break;
+              }
+              case 10: {
+                bitField0_ |= 0x00000001;
+                columnName_ = input.readBytes();
+                break;
+              }
+              case 16: {
+                bitField0_ |= 0x00000002;
+                keySeq_ = input.readSInt32();
+                break;
+              }
+            }
+          }
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          throw e.setUnfinishedMessage(this);
+        } catch (java.io.IOException e) {
+          throw new com.google.protobuf.InvalidProtocolBufferException(
+              e.getMessage()).setUnfinishedMessage(this);
+        } finally {
+          this.unknownFields = unknownFields.build();
+          makeExtensionsImmutable();
+        }
+      }
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PrimaryKey_PrimaryKeyColumn_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PrimaryKey_PrimaryKeyColumn_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.Builder.class);
+      }
+
+      public static com.google.protobuf.Parser<PrimaryKeyColumn> PARSER =
+          new com.google.protobuf.AbstractParser<PrimaryKeyColumn>() {
+        public PrimaryKeyColumn parsePartialFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return new PrimaryKeyColumn(input, extensionRegistry);
+        }
+      };
+
+      @java.lang.Override
+      public com.google.protobuf.Parser<PrimaryKeyColumn> getParserForType() {
+        return PARSER;
+      }
+
+      private int bitField0_;
+      // required string column_name = 1;
+      public static final int COLUMN_NAME_FIELD_NUMBER = 1;
+      private java.lang.Object columnName_;
+      /**
+       * <code>required string column_name = 1;</code>
+       */
+      public boolean hasColumnName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string column_name = 1;</code>
+       */
+      public java.lang.String getColumnName() {
+        java.lang.Object ref = columnName_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            columnName_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string column_name = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getColumnNameBytes() {
+        java.lang.Object ref = columnName_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          columnName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      // required sint32 key_seq = 2;
+      public static final int KEY_SEQ_FIELD_NUMBER = 2;
+      private int keySeq_;
+      /**
+       * <code>required sint32 key_seq = 2;</code>
+       */
+      public boolean hasKeySeq() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required sint32 key_seq = 2;</code>
+       */
+      public int getKeySeq() {
+        return keySeq_;
+      }
+
+      private void initFields() {
+        columnName_ = "";
+        keySeq_ = 0;
+      }
+      private byte memoizedIsInitialized = -1;
+      public final boolean isInitialized() {
+        byte isInitialized = memoizedIsInitialized;
+        if (isInitialized != -1) return isInitialized == 1;
+
+        if (!hasColumnName()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        if (!hasKeySeq()) {
+          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, getColumnNameBytes());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          output.writeSInt32(2, keySeq_);
+        }
+        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, getColumnNameBytes());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeSInt32Size(2, keySeq_);
+        }
+        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();
+      }
+
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn parseFrom(
+          com.google.protobuf.ByteString data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn parseFrom(
+          com.google.protobuf.ByteString data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn parseFrom(byte[] data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn parseFrom(
+          byte[] data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn parseFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn parseFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn parseDelimitedFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn parseDelimitedFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn parseFrom(
+          com.google.protobuf.CodedInputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn parseFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+
+      public static Builder newBuilder() { return Builder.create(); }
+      public Builder newBuilderForType() { return newBuilder(); }
+      public static Builder newBuilder(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn 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;
+      }
+      /**
+       * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn}
+       */
+      public static final class Builder extends
+          com.google.protobuf.GeneratedMessage.Builder<Builder>
+         implements org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumnOrBuilder {
+        public static final com.google.protobuf.Descriptors.Descriptor
+            getDescriptor() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PrimaryKey_PrimaryKeyColumn_descriptor;
+        }
+
+        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+            internalGetFieldAccessorTable() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PrimaryKey_PrimaryKeyColumn_fieldAccessorTable
+              .ensureFieldAccessorsInitialized(
+                  org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.Builder.class);
+        }
+
+        // Construct using org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.newBuilder()
+        private Builder() {
+          maybeForceBuilderInitialization();
+        }
+
+        private Builder(
+            com.google.protobuf.GeneratedMessage.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();
+          columnName_ = "";
+          bitField0_ = (bitField0_ & ~0x00000001);
+          keySeq_ = 0;
+          bitField0_ = (bitField0_ & ~0x00000002);
+          return this;
+        }
+
+        public Builder clone() {
+          return create().mergeFrom(buildPartial());
+        }
+
+        public com.google.protobuf.Descriptors.Descriptor
+            getDescriptorForType() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PrimaryKey_PrimaryKeyColumn_descriptor;
+        }
+
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn getDefaultInstanceForType() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.getDefaultInstance();
+        }
+
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn build() {
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn result = buildPartial();
+          if (!result.isInitialized()) {
+            throw newUninitializedMessageException(result);
+          }
+          return result;
+        }
+
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn buildPartial() {
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn result = new org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn(this);
+          int from_bitField0_ = bitField0_;
+          int to_bitField0_ = 0;
+          if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+            to_bitField0_ |= 0x00000001;
+          }
+          result.columnName_ = columnName_;
+          if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+            to_bitField0_ |= 0x00000002;
+          }
+          result.keySeq_ = keySeq_;
+          result.bitField0_ = to_bitField0_;
+          onBuilt();
+          return result;
+        }
+
+        public Builder mergeFrom(com.google.protobuf.Message other) {
+          if (other instanceof org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn) {
+            return mergeFrom((org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn)other);
+          } else {
+            super.mergeFrom(other);
+            return this;
+          }
+        }
+
+        public Builder mergeFrom(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn other) {
+          if (other == org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.getDefaultInstance()) return this;
+          if (other.hasColumnName()) {
+            bitField0_ |= 0x00000001;
+            columnName_ = other.columnName_;
+            onChanged();
+          }
+          if (other.hasKeySeq()) {
+            setKeySeq(other.getKeySeq());
+          }
+          this.mergeUnknownFields(other.getUnknownFields());
+          return this;
+        }
+
+        public final boolean isInitialized() {
+          if (!hasColumnName()) {
+            
+            return false;
+          }
+          if (!hasKeySeq()) {
+            
+            return false;
+          }
+          return true;
+        }
+
+        public Builder mergeFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn parsedMessage = null;
+          try {
+            parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+            parsedMessage = (org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn) e.getUnfinishedMessage();
+            throw e;
+          } finally {
+            if (parsedMessage != null) {
+              mergeFrom(parsedMessage);
+            }
+          }
+          return this;
+        }
+        private int bitField0_;
+
+        // required string column_name = 1;
+        private java.lang.Object columnName_ = "";
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        public boolean hasColumnName() {
+          return ((bitField0_ & 0x00000001) == 0x00000001);
+        }
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        public java.lang.String getColumnName() {
+          java.lang.Object ref = columnName_;
+          if (!(ref instanceof java.lang.String)) {
+            java.lang.String s = ((com.google.protobuf.ByteString) ref)
+                .toStringUtf8();
+            columnName_ = s;
+            return s;
+          } else {
+            return (java.lang.String) ref;
+          }
+        }
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        public com.google.protobuf.ByteString
+            getColumnNameBytes() {
+          java.lang.Object ref = columnName_;
+          if (ref instanceof String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            columnName_ = b;
+            return b;
+          } else {
+            return (com.google.protobuf.ByteString) ref;
+          }
+        }
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        public Builder setColumnName(
+            java.lang.String value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          columnName_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        public Builder clearColumnName() {
+          bitField0_ = (bitField0_ & ~0x00000001);
+          columnName_ = getDefaultInstance().getColumnName();
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        public Builder setColumnNameBytes(
+            com.google.protobuf.ByteString value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          columnName_ = value;
+          onChanged();
+          return this;
+        }
+
+        // required sint32 key_seq = 2;
+        private int keySeq_ ;
+        /**
+         * <code>required sint32 key_seq = 2;</code>
+         */
+        public boolean hasKeySeq() {
+          return ((bitField0_ & 0x00000002) == 0x00000002);
+        }
+        /**
+         * <code>required sint32 key_seq = 2;</code>
+         */
+        public int getKeySeq() {
+          return keySeq_;
+        }
+        /**
+         * <code>required sint32 key_seq = 2;</code>
+         */
+        public Builder setKeySeq(int value) {
+          bitField0_ |= 0x00000002;
+          keySeq_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required sint32 key_seq = 2;</code>
+         */
+        public Builder clearKeySeq() {
+          bitField0_ = (bitField0_ & ~0x00000002);
+          keySeq_ = 0;
+          onChanged();
+          return this;
+        }
+
+        // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn)
+      }
+
+      static {
+        defaultInstance = new PrimaryKeyColumn(true);
+        defaultInstance.initFields();
+      }
+
+      // @@protoc_insertion_point(class_scope:org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn)
+    }
+
+    private int bitField0_;
+    // required string pk_name = 1;
+    public static final int PK_NAME_FIELD_NUMBER = 1;
+    private java.lang.Object pkName_;
+    /**
+     * <code>required string pk_name = 1;</code>
+     */
+    public boolean hasPkName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string pk_name = 1;</code>
+     */
+    public java.lang.String getPkName() {
+      java.lang.Object ref = pkName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          pkName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string pk_name = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getPkNameBytes() {
+      java.lang.Object ref = pkName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        pkName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;
+    public static final int COLS_FIELD_NUMBER = 2;
+    private java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn> cols_;
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+     */
+    public java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn> getColsList() {
+      return cols_;
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumnOrBuilder> 
+        getColsOrBuilderList() {
+      return cols_;
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+     */
+    public int getColsCount() {
+      return cols_.size();
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+     */
+    public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn getCols(int index) {
+      return cols_.get(index);
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+     */
+    public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumnOrBuilder getColsOrBuilder(
+        int index) {
+      return cols_.get(index);
+    }
+
+    // optional bool enable_constraint = 3;
+    public static final int ENABLE_CONSTRAINT_FIELD_NUMBER = 3;
+    private boolean enableConstraint_;
+    /**
+     * <code>optional bool enable_constraint = 3;</code>
+     */
+    public boolean hasEnableConstraint() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional bool enable_constraint = 3;</code>
+     */
+    public boolean getEnableConstraint() {
+      return enableConstraint_;
+    }
+
+    // optional bool validate_constraint = 4;
+    public static final int VALIDATE_CONSTRAINT_FIELD_NUMBER = 4;
+    private boolean validateConstraint_;
+    /**
+     * <code>optional bool validate_constraint = 4;</code>
+     */
+    public boolean hasValidateConstraint() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional bool validate_constraint = 4;</code>
+     */
+    public boolean getValidateConstraint() {
+      return validateConstraint_;
+    }
+
+    // optional bool rely_constraint = 5;
+    public static final int RELY_CONSTRAINT_FIELD_NUMBER = 5;
+    private boolean relyConstraint_;
+    /**
+     * <code>optional bool rely_constraint = 5;</code>
+     */
+    public boolean hasRelyConstraint() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional bool rely_constraint = 5;</code>
+     */
+    public boolean getRelyConstraint() {
+      return relyConstraint_;
+    }
+
+    private void initFields() {
+      pkName_ = "";
+      cols_ = java.util.Collections.emptyList();
+      enableConstraint_ = false;
+      validateConstraint_ = false;
+      relyConstraint_ = false;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasPkName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getColsCount(); i++) {
+        if (!getCols(i).isInitialized()) {
+          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, getPkNameBytes());
+      }
+      for (int i = 0; i < cols_.size(); i++) {
+        output.writeMessage(2, cols_.get(i));
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBool(3, enableConstraint_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBool(4, validateConstraint_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBool(5, relyConstraint_);
+      }
+      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, getPkNameBytes());
+      }
+      for (int i = 0; i < cols_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, cols_.get(i));
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(3, enableConstraint_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(4, validateConstraint_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(5, relyConstraint_);
+      }
+      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();
+    }
+
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey 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;
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.PrimaryKey}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKeyOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PrimaryKey_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PrimaryKey_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getColsFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        pkName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (colsBuilder_ == null) {
+          cols_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+        } else {
+          colsBuilder_.clear();
+        }
+        enableConstraint_ = false;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        validateConstraint_ = false;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        relyConstraint_ = false;
+        bitField0_ = (bitField0_ & ~0x00000010);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_PrimaryKey_descriptor;
+      }
+
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey getDefaultInstanceForType() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey build() {
+        org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey buildPartial() {
+        org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey result = new org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.pkName_ = pkName_;
+        if (colsBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+            cols_ = java.util.Collections.unmodifiableList(cols_);
+            bitField0_ = (bitField0_ & ~0x00000002);
+          }
+          result.cols_ = cols_;
+        } else {
+          result.cols_ = colsBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.enableConstraint_ = enableConstraint_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.validateConstraint_ = validateConstraint_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.relyConstraint_ = relyConstraint_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey) {
+          return mergeFrom((org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey other) {
+        if (other == org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.getDefaultInstance()) return this;
+        if (other.hasPkName()) {
+          bitField0_ |= 0x00000001;
+          pkName_ = other.pkName_;
+          onChanged();
+        }
+        if (colsBuilder_ == null) {
+          if (!other.cols_.isEmpty()) {
+            if (cols_.isEmpty()) {
+              cols_ = other.cols_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+            } else {
+              ensureColsIsMutable();
+              cols_.addAll(other.cols_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.cols_.isEmpty()) {
+            if (colsBuilder_.isEmpty()) {
+              colsBuilder_.dispose();
+              colsBuilder_ = null;
+              cols_ = other.cols_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+              colsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getColsFieldBuilder() : null;
+            } else {
+              colsBuilder_.addAllMessages(other.cols_);
+            }
+          }
+        }
+        if (other.hasEnableConstraint()) {
+          setEnableConstraint(other.getEnableConstraint());
+        }
+        if (other.hasValidateConstraint()) {
+          setValidateConstraint(other.getValidateConstraint());
+        }
+        if (other.hasRelyConstraint()) {
+          setRelyConstraint(other.getRelyConstraint());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasPkName()) {
+          
+          return false;
+        }
+        for (int i = 0; i < getColsCount(); i++) {
+          if (!getCols(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string pk_name = 1;
+      private java.lang.Object pkName_ = "";
+      /**
+       * <code>required string pk_name = 1;</code>
+       */
+      public boolean hasPkName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string pk_name = 1;</code>
+       */
+      public java.lang.String getPkName() {
+        java.lang.Object ref = pkName_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          pkName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string pk_name = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getPkNameBytes() {
+        java.lang.Object ref = pkName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          pkName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string pk_name = 1;</code>
+       */
+      public Builder setPkName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        pkName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string pk_name = 1;</code>
+       */
+      public Builder clearPkName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        pkName_ = getDefaultInstance().getPkName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string pk_name = 1;</code>
+       */
+      public Builder setPkNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        pkName_ = value;
+        onChanged();
+        return this;
+      }
+
+      // repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;
+      private java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn> cols_ =
+        java.util.Collections.emptyList();
+      private void ensureColsIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          cols_ = new java.util.ArrayList<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn>(cols_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.Builder, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumnOrBuilder> colsBuilder_;
+
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn> getColsList() {
+        if (colsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(cols_);
+        } else {
+          return colsBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public int getColsCount() {
+        if (colsBuilder_ == null) {
+          return cols_.size();
+        } else {
+          return colsBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn getCols(int index) {
+        if (colsBuilder_ == null) {
+          return cols_.get(index);
+        } else {
+          return colsBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public Builder setCols(
+          int index, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn value) {
+        if (colsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureColsIsMutable();
+          cols_.set(index, value);
+          onChanged();
+        } else {
+          colsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public Builder setCols(
+          int index, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.Builder builderForValue) {
+        if (colsBuilder_ == null) {
+          ensureColsIsMutable();
+          cols_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          colsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public Builder addCols(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn value) {
+        if (colsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureColsIsMutable();
+          cols_.add(value);
+          onChanged();
+        } else {
+          colsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public Builder addCols(
+          int index, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn value) {
+        if (colsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureColsIsMutable();
+          cols_.add(index, value);
+          onChanged();
+        } else {
+          colsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public Builder addCols(
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.Builder builderForValue) {
+        if (colsBuilder_ == null) {
+          ensureColsIsMutable();
+          cols_.add(builderForValue.build());
+          onChanged();
+        } else {
+          colsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public Builder addCols(
+          int index, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.Builder builderForValue) {
+        if (colsBuilder_ == null) {
+          ensureColsIsMutable();
+          cols_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          colsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public Builder addAllCols(
+          java.lang.Iterable<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn> values) {
+        if (colsBuilder_ == null) {
+          ensureColsIsMutable();
+          super.addAll(values, cols_);
+          onChanged();
+        } else {
+          colsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public Builder clearCols() {
+        if (colsBuilder_ == null) {
+          cols_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+          onChanged();
+        } else {
+          colsBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public Builder removeCols(int index) {
+        if (colsBuilder_ == null) {
+          ensureColsIsMutable();
+          cols_.remove(index);
+          onChanged();
+        } else {
+          colsBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.Builder getColsBuilder(
+          int index) {
+        return getColsFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumnOrBuilder getColsOrBuilder(
+          int index) {
+        if (colsBuilder_ == null) {
+          return cols_.get(index);  } else {
+          return colsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumnOrBuilder> 
+           getColsOrBuilderList() {
+        if (colsBuilder_ != null) {
+          return colsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(cols_);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.Builder addColsBuilder() {
+        return getColsFieldBuilder().addBuilder(
+            org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.Builder addColsBuilder(
+          int index) {
+        return getColsFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.PrimaryKey.PrimaryKeyColumn cols = 2;</code>
+       */
+      public java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.Builder> 
+           getColsBuilderList() {
+        return getColsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.Builder, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumnOrBuilder> 
+          getColsFieldBuilder() {
+        if (colsBuilder_ == null) {
+          colsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumn.Builder, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrimaryKey.PrimaryKeyColumnOrBuilder>(
+                  cols_,
+                  ((bitField0_ & 0x00000002) == 0x00000002),
+                  getParentForChildren(),
+                  isClean());
+          cols_ = null;
+        }
+        return colsBuilder_;
+      }
+
+      // optional bool enable_constraint = 3;
+      private boolean enableConstraint_ ;
+      /**
+       * <code>optional bool enable_constraint = 3;</code>
+       */
+      public boolean hasEnableConstraint() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional bool enable_constraint = 3;</code>
+       */
+      public boolean getEnableConstraint() {
+        return enableConstraint_;
+      }
+      /**
+       * <code>optional bool enable_constraint = 3;</code>
+       */
+      public Builder setEnableConstraint(boolean value) {
+        bitField0_ |= 0x00000004;
+        enableConstraint_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool enable_constraint = 3;</code>
+       */
+      public Builder clearEnableConstraint() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        enableConstraint_ = false;
+        onChanged();
+        return this;
+      }
+
+      // optional bool validate_constraint = 4;
+      private boolean validateConstraint_ ;
+      /**
+       * <code>optional bool validate_constraint = 4;</code>
+       */
+      public boolean hasValidateConstraint() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional bool validate_constraint = 4;</code>
+       */
+      public boolean getValidateConstraint() {
+        return validateConstraint_;
+      }
+      /**
+       * <code>optional bool validate_constraint = 4;</code>
+       */
+      public Builder setValidateConstraint(boolean value) {
+        bitField0_ |= 0x00000008;
+        validateConstraint_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool validate_constraint = 4;</code>
+       */
+      public Builder clearValidateConstraint() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        validateConstraint_ = false;
+        onChanged();
+        return this;
+      }
+
+      // optional bool rely_constraint = 5;
+      private boolean relyConstraint_ ;
+      /**
+       * <code>optional bool rely_constraint = 5;</code>
+       */
+      public boolean hasRelyConstraint() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional bool rely_constraint = 5;</code>
+       */
+      public boolean getRelyConstraint() {
+        return relyConstraint_;
+      }
+      /**
+       * <code>optional bool rely_constraint = 5;</code>
+       */
+      public Builder setRelyConstraint(boolean value) {
+        bitField0_ |= 0x00000010;
+        relyConstraint_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool rely_constraint = 5;</code>
+       */
+      public Builder clearRelyConstraint() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        relyConstraint_ = false;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hive.metastore.hbase.PrimaryKey)
+    }
+
+    static {
+      defaultInstance = new PrimaryKey(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:org.apache.hadoop.hive.metastore.hbase.PrimaryKey)
+  }
+
+  public interface ForeignKeysOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated .org.apache.hadoop.hive.metastore.hbase.ForeignKeys.ForeignKey fks = 1;
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.ForeignKeys.ForeignKey fks = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKey> 
+        getFksList();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.ForeignKeys.ForeignKey fks = 1;</code>
+     */
+    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKey getFks(int index);
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.ForeignKeys.ForeignKey fks = 1;</code>
+     */
+    int getFksCount();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.ForeignKeys.ForeignKey fks = 1;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKeyOrBuilder> 
+        getFksOrBuilderList();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.ForeignKeys.ForeignKey fks = 1;</code>
+     */
+    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKeyOrBuilder getFksOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.ForeignKeys}
+   */
+  public static final class ForeignKeys extends
+      com.google.protobuf.GeneratedMessage
+      implements ForeignKeysOrBuilder {
+    // Use ForeignKeys.newBuilder() to construct.
+    private ForeignKeys(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ForeignKeys(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ForeignKeys defaultInstance;
+    public static ForeignKeys getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ForeignKeys getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ForeignKeys(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                fks_ = new java.util.ArrayList<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKey>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              fks_.add(input.readMessage(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKey.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          fks_ = java.util.Collections.unmodifiableList(fks_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_ForeignKeys_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_ForeignKeys_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<ForeignKeys> PARSER =
+        new com.google.protobuf.AbstractParser<ForeignKeys>() {
+      public ForeignKeys parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ForeignKeys(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ForeignKeys> getParserForType() {
+      return PARSER;
+    }
+
+    public interface ForeignKeyOrBuilder
+        extends com.google.protobuf.MessageOrBuilder {
+
+      // required string fk_name = 1;
+      /**
+       * <code>required string fk_name = 1;</code>
+       */
+      boolean hasFkName();
+      /**
+       * <code>required string fk_name = 1;</code>
+       */
+      java.lang.String getFkName();
+      /**
+       * <code>required string fk_name = 1;</code>
+       */
+      com.google.protobuf.ByteString
+          getFkNameBytes();
+
+      // required string referenced_db_name = 2;
+      /**
+       * <code>required string referenced_db_name = 2;</code>
+       */
+      boolean hasReferencedDbName();
+      /**
+       * <code>required string referenced_db_name = 2;</code>
+       */
+      java.lang.String getReferencedDbName();
+      /**
+       * <code>required string referenced_db_name = 2;</code>
+       */
+      com.google.protobuf.ByteString
+          getReferencedDbNameBytes();
+
+      // required string referenced_table_name = 3;
+      /**
+       * <code>required string referenced_table_name = 3;</code>
+       */
+      boolean hasReferencedTableName();
+      /**
+       * <code>required string referenced_table_name = 3;</code>
+       */
+      java.lang.String getReferencedTableName();
+      /**
+       * <code>required string referenced_table_name = 3;</code>
+       */
+      com.google.protobuf.ByteString
+          getReferencedTableNameBytes();
+
+      // optional string referenced_pk_name = 4;
+      /**
+       * <code>optional string referenced_pk_name = 4;</code>
+       */
+      boolean hasReferencedPkName();
+      /**
+       * <code>optional string referenced_pk_name = 4;</code>
+       */
+      java.lang.String getReferencedPkName();
+      /**
+       * <code>optional string referenced_pk_name = 4;</code>
+       */
+      com.google.protobuf.ByteString
+          getReferencedPkNameBytes();
+
+      // optional int32 update_rule = 5;
+      /**
+       * <code>optional int32 update_rule = 5;</code>
+       */
+      boolean hasUpdateRule();
+      /**
+       * <code>optional int32 update_rule = 5;</code>
+       */
+      int getUpdateRule();
+
+      // optional int32 delete_rule = 6;
+      /**
+       * <code>optional int32 delete_rule = 6;</code>
+       */
+      boolean hasDeleteRule();
+      /**
+       * <code>optional int32 delete_rule = 6;</code>
+       */
+      int getDeleteRule();
+
+      // repeated .org.apache.hadoop.hive.metastore.hbase.ForeignKeys.ForeignKey.ForeignKeyColumn cols = 7;
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.ForeignKeys.ForeignKey.ForeignKeyColumn cols = 7;</code>
+       */
+      java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKey.ForeignKeyColumn> 
+          getColsList();
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.ForeignKeys.ForeignKey.ForeignKeyColumn cols = 7;</code>
+       */
+      org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKey.ForeignKeyColumn getCols(int index);
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.ForeignKeys.ForeignKey.ForeignKeyColumn cols = 7;</code>
+       */
+      int getColsCount();
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.ForeignKeys.ForeignKey.ForeignKeyColumn cols = 7;</code>
+       */
+      java.util.List<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKey.ForeignKeyColumnOrBuilder> 
+          getColsOrBuilderList();
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.ForeignKeys.ForeignKey.ForeignKeyColumn cols = 7;</code>
+       */
+      org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKey.ForeignKeyColumnOrBuilder getColsOrBuilder(
+          int index);
+
+      // optional bool enable_constraint = 8;
+      /**
+       * <code>optional bool enable_constraint = 8;</code>
+       */
+      boolean hasEnableConstraint();
+      /**
+       * <code>optional bool enable_constraint = 8;</code>
+       */
+      boolean getEnableConstraint();
+
+      // optional bool validate_constraint = 9;
+      /**
+       * <code>optional bool validate_constraint = 9;</code>
+       */
+      boolean hasValidateConstraint();
+      /**
+       * <code>optional bool validate_constraint = 9;</code>
+       */
+      boolean getValidateConstraint();
+
+      // optional bool rely_constraint = 10;
+      /**
+       * <code>optional bool rely_constraint = 10;</code>
+       */
+      boolean hasRelyConstraint();
+      /**
+       * <code>optional bool rely_constraint = 10;</code>
+       */
+      boolean getRelyConstraint();
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.ForeignKeys.ForeignKey}
+     */
+    public static final class ForeignKey extends
+        com.google.protobuf.GeneratedMessage
+        implements ForeignKeyOrBuilder {
+      // Use ForeignKey.newBuilder() to construct.
+      private ForeignKey(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+        super(builder);
+        this.unknownFields = builder.getUnknownFields();
+      }
+      private ForeignKey(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+      private static final ForeignKey defaultInstance;
+      public static ForeignKey getDefaultInstance() {
+        return defaultInstance;
+      }
+
+      public ForeignKey getDefaultInstanceForType() {
+        return defaultInstance;
+      }
+
+      private final com.google.protobuf.UnknownFieldSet unknownFields;
+      @java.lang.Override
+      public final com.google.protobuf.UnknownFieldSet
+          getUnknownFields() {
+        return this.unknownFields;
+      }
+      private ForeignKey(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        initFields();
+        int mutable_bitField0_ = 0;
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            com.google.protobuf.UnknownFieldSet.newBuilder();
+        try {
+          boolean done = false;
+          while (!done) {
+            int tag = input.readTag();
+            switch (tag) {
+              case 0:
+                done = true;
+                break;
+              default: {
+                if (!parseUnknownField(input, unknownFields,
+                                       extensionRegistry, tag)) {
+                  done = true;
+                }
+                break;
+              }
+              case 10: {
+                bitField0_ |= 0x00000001;
+                fkName_ = input.readBytes();
+                break;
+              }
+              case 18: {
+                bitField0_ |= 0x00000002;
+                referencedDbName_ = input.readBytes();
+                break;
+              }
+              case 26: {
+                bitField0_ |= 0x00000004;
+                referencedTableName_ = input.readBytes();
+                break;
+              }
+              case 34: {
+                bitField0_ |= 0x00000008;
+                referencedPkName_ = input.readBytes();
+                break;
+              }
+              case 40: {
+                bitField0_ |= 0x00000010;
+                updateRule_ = input.readInt32();
+                break;
+              }
+              case 48: {
+                bitField0_ |= 0x00000020;
+                deleteRule_ = input.readInt32();
+                break;
+              }
+              case 58: {
+                if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+                  cols_ = new java.util.ArrayList<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKey.ForeignKeyColumn>();
+                  mutable_bitField0_ |= 0x00000040;
+                }
+                cols_.add(input.readMessage(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKey.ForeignKeyColumn.PARSER, extensionRegistry));
+                break;
+              }
+              case 64: {
+                bitField0_ |= 0x00000040;
+                enableConstraint_ = input.readBool();
+                break;
+              }
+              case 72: {
+                bitField0_ |= 0x00000080;
+                validateConstraint_ = input.readBool();
+                break;
+              }
+              case 80: {
+                bitField0_ |= 0x00000100;
+                relyConstraint_ = input.readBool();
+                break;
+              }
+            }
+          }
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          throw e.setUnfinishedMessage(this);
+        } catch (java.io.IOException e) {
+          throw new com.google.protobuf.InvalidProtocolBufferException(
+              e.getMessage()).setUnfinishedMessage(this);
+        } finally {
+          if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+            cols_ = java.util.Collections.unmodifiableList(cols_);
+          }
+          this.unknownFields = unknownFields.build();
+          makeExtensionsImmutable();
+        }
+      }
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_ForeignKeys_ForeignKey_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_ForeignKeys_ForeignKey_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKey.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKey.Builder.class);
+      }
+
+      public static com.google.protobuf.Parser<ForeignKey> PARSER =
+          new com.google.protobuf.AbstractParser<ForeignKey>() {
+        public ForeignKey parsePartialFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return new ForeignKey(input, extensionRegistry);
+        }
+      };
+
+      @java.lang.Override
+      public com.google.protobuf.Parser<ForeignKey> getParserForType() {
+        return PARSER;
+      }
+
+      public interface ForeignKeyColumnOrBuilder
+          extends com.google.protobuf.MessageOrBuilder {
+
+        // required string column_name = 1;
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        boolean hasColumnName();
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        java.lang.String getColumnName();
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        com.google.protobuf.ByteString
+            getColumnNameBytes();
+
+        // required string referenced_column_name = 2;
+        /**
+         * <code>required string referenced_column_name = 2;</code>
+         */
+        boolean hasReferencedColumnName();
+        /**
+         * <code>required string referenced_column_name = 2;</code>
+         */
+        java.lang.String getReferencedColumnName();
+        /**
+         * <code>required string referenced_column_name = 2;</code>
+         */
+        com.google.protobuf.ByteString
+            getReferencedColumnNameBytes();
+
+        // required sint32 key_seq = 3;
+        /**
+         * <code>required sint32 key_seq = 3;</code>
+         */
+        boolean hasKeySeq();
+        /**
+         * <code>required sint32 key_seq = 3;</code>
+         */
+        int getKeySeq();
+      }
+      /**
+       * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.ForeignKeys.ForeignKey.ForeignKeyColumn}
+       */
+      public static final class ForeignKeyColumn extends
+          com.google.protobuf.GeneratedMessage
+          implements ForeignKeyColumnOrBuilder {
+        // Use ForeignKeyColumn.newBuilder() to construct.
+        private ForeignKeyColumn(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+          super(builder);
+          this.unknownFields = builder.getUnknownFields();
+        }
+        private ForeignKeyColumn(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+        private static final ForeignKeyColumn defaultInstance;
+        public static ForeignKeyColumn getDefaultInstance() {
+          return defaultInstance;
+        }
+
+        public ForeignKeyColumn getDefaultInstanceForType() {
+          return defaultInstance;
+        }
+
+        private final com.google.protobuf.UnknownFieldSet unknownFields;
+        @java.lang.Override
+        public final com.google.protobuf.UnknownFieldSet
+            getUnknownFields() {
+          return this.unknownFields;
+        }
+        private ForeignKeyColumn(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          initFields();
+          int mutable_bitField0_ = 0;
+          com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+              com.google.protobuf.UnknownFieldSet.newBuilder();
+          try {
+            boolean done = false;
+            while (!done) {
+              int tag = input.readTag();
+              switch (tag) {
+                case 0:
+                  done = true;
+                  break;
+                default: {
+                  if (!parseUnknownField(input, unknownFields,
+                                         extensionRegistry, tag)) {
+                    done = true;
+                  }
+                  break;
+                }
+                case 10: {
+                  bitField0_ |= 0x00000001;
+                  columnName_ = input.readBytes();
+                  break;
+                }
+                case 18: {
+                  bitField0_ |= 0x00000002;
+                  referencedColumnName_ = input.readBytes();
+                  break;
+                }
+                case 24: {
+                  bitField0_ |= 0x00000004;
+                  keySeq_ = input.readSInt32();
+                  break;
+                }
+              }
+            }
+          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+            throw e.setUnfinishedMessage(this);
+          } catch (java.io.IOException e) {
+            throw new com.google.protobuf.InvalidProtocolBufferException(
+                e.getMessage()).setUnfinishedMessage(this);
+          } finally {
+            this.unknownFields = unknownFields.build();
+            makeExtensionsImmutable();
+          }
+        }
+        public static final com.google.protobuf.Descriptors.Descriptor
+            getDescriptor() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_ForeignKeys_ForeignKey_ForeignKeyColumn_descriptor;
+        }
+
+        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+            internalGetFieldAccessorTable() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_ForeignKeys_ForeignKey_ForeignKeyColumn_fieldAccessorTable
+              .ensureFieldAccessorsInitialized(
+                  org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKey.ForeignKeyColumn.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.ForeignKeys.ForeignKey.ForeignKeyColumn.Builder.class);
+        }
+
+        public static com.google.protobuf.Parser<ForeignKeyColumn> PARSER =
+            new com.google.protobuf.AbstractParser<ForeignKeyColumn>() {
+          public ForeignKeyColumn parsePartialFrom(
+              com.google.protobuf.CodedInputStream input,
+              com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+              throws com.google.protobuf.InvalidProtocolBufferException {
+            return new ForeignKeyColumn(input, extensionRegistry);
+          }
+        };
+
+        @java.lang.Override
+        public com.google.protobuf.Parser<ForeignKeyColumn> getParserForType() {
+          return PARSER;
+        }
+
+        private int bitField0_;
+        // required string column_name = 1;
+        public static final int COLUMN_NAME_FIELD_NUMBER = 1;
+        private java.lang.Object columnName_;
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        public boolean hasColumnName() {
+          return ((bitField0_ & 0x00000001) == 0x00000001);
+        }
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        public java.lang.String getColumnName() {
+          java.lang.Object ref = columnName_;
+          if (ref instanceof java.lang.String) {
+            return (java.lang.String) ref;
+          } else {
+            com.google.protobuf.ByteString bs = 
+                (com.google.protobuf.ByteString) ref;
+            java.lang.String s = bs.toStringUtf8();
+            if (bs.isValidUtf8()) {
+              columnName_ = s;
+            }
+            return s;
+          }
+        }
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        public com.google.protobuf.ByteString
+            getColumnNameBytes() {
+          java.lang.Object ref = columnName_;
+          if (ref instanceof java.lang.String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            columnName

<TRUNCATED>