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 2018/03/31 01:25:28 UTC

[09/44] hive git commit: HIVE-18755 Modifications to the metastore for catalogs (Alan Gates, reviewed by Thejas Nair)

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index 357ac2d..773e4f8 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.metastore;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 
 import java.nio.ByteBuffer;
@@ -140,38 +141,64 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
+  public void createCatalog(Catalog cat) throws MetaException {
+
+  }
+
+  @Override
+  public Catalog getCatalog(String catalogName) throws NoSuchObjectException, MetaException {
+    return null;
+  }
+
+  @Override
+  public void alterCatalog(String catName, Catalog cat) throws MetaException,
+      InvalidOperationException {
+
+  }
+
+  @Override
+  public List<String> getCatalogs() throws MetaException {
+    return null;
+  }
+
+  @Override
+  public void dropCatalog(String catalogName) throws NoSuchObjectException, MetaException {
+
+  }
+
+  @Override
   public void createDatabase(Database db) throws InvalidObjectException, MetaException {
 
 
   }
 
   @Override
-  public Database getDatabase(String name) throws NoSuchObjectException {
+  public Database getDatabase(String catName, String name) throws NoSuchObjectException {
 
     return null;
   }
 
   @Override
-  public boolean dropDatabase(String dbname) throws NoSuchObjectException, MetaException {
+  public boolean dropDatabase(String catName, String dbname) throws NoSuchObjectException, MetaException {
 
     return false;
   }
 
   @Override
-  public boolean alterDatabase(String dbname, Database db) throws NoSuchObjectException,
+  public boolean alterDatabase(String catName, String dbname, Database db) throws NoSuchObjectException,
       MetaException {
 
     return false;
   }
 
   @Override
-  public List<String> getDatabases(String pattern) throws MetaException {
+  public List<String> getDatabases(String catName, String pattern) throws MetaException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> getAllDatabases() throws MetaException {
+  public List<String> getAllDatabases(String catName) throws MetaException {
 
     return Collections.emptyList();
   }
@@ -201,13 +228,13 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public boolean dropTable(String dbName, String tableName) throws MetaException {
+  public boolean dropTable(String catName, String dbName, String tableName) throws MetaException {
 
     return false;
   }
 
   @Override
-  public Table getTable(String dbName, String tableName) throws MetaException {
+  public Table getTable(String catName, String dbName, String tableName) throws MetaException {
 
     return null;
   }
@@ -219,144 +246,141 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public Partition getPartition(String dbName, String tableName, List<String> part_vals)
+  public Partition getPartition(String catName, String dbName, String tableName, List<String> part_vals)
       throws MetaException, NoSuchObjectException {
 
     return null;
   }
 
   @Override
-  public boolean dropPartition(String dbName, String tableName, List<String> part_vals)
+  public boolean dropPartition(String catName, String dbName, String tableName, List<String> part_vals)
       throws MetaException {
 
     return false;
   }
 
   @Override
-  public List<Partition> getPartitions(String dbName, String tableName, int max)
+  public List<Partition> getPartitions(String catName, String dbName, String tableName, int max)
       throws MetaException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public void alterTable(String dbname, String name, Table newTable) throws InvalidObjectException,
-      MetaException {
-
-
+  public void alterTable(String catName, String dbname, String name, Table newTable)
+      throws InvalidObjectException, MetaException {
   }
 
   @Override
-  public void updateCreationMetadata(String dbname, String tablename, CreationMetadata cm)
+  public void updateCreationMetadata(String catName, String dbname, String tablename, CreationMetadata cm)
       throws MetaException {
   }
 
-  @Override
-  public List<String> getTables(String dbName, String pattern) throws MetaException {
-
+  public List<String> getTables(String catName, String dbName, String pattern) throws MetaException {
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> getTables(String dbName, String pattern, TableType tableType) throws MetaException {
+  public List<String> getTables(String catName, String dbName, String pattern, TableType tableType) throws MetaException {
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> getMaterializedViewsForRewriting(String dbName)
+  public List<String> getMaterializedViewsForRewriting(String catName, String dbName)
       throws MetaException, NoSuchObjectException {
     return Collections.emptyList();
   }
 
   @Override
-  public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
+  public List<TableMeta> getTableMeta(String catName, String dbNames, String tableNames, List<String> tableTypes)
       throws MetaException {
     return Collections.emptyList();
   }
 
   @Override
-  public List<Table> getTableObjectsByName(String dbname, List<String> tableNames)
+  public List<Table> getTableObjectsByName(String catName, String dbname, List<String> tableNames)
       throws MetaException, UnknownDBException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> getAllTables(String dbName) throws MetaException {
+  public List<String> getAllTables(String catName, String dbName) throws MetaException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> listTableNamesByFilter(String dbName, String filter, short max_tables)
+  public List<String> listTableNamesByFilter(String catName, String dbName, String filter, short max_tables)
       throws MetaException, UnknownDBException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> listPartitionNames(String db_name, String tbl_name, short max_parts)
+  public List<String> listPartitionNames(String catName, String db_name, String tbl_name, short max_parts)
       throws MetaException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public PartitionValuesResponse listPartitionValues(String db_name, String tbl_name, List<FieldSchema> cols, boolean applyDistinct, String filter, boolean ascending, List<FieldSchema> order, long maxParts) throws MetaException {
+  public PartitionValuesResponse listPartitionValues(String catName, String db_name,
+                                                     String tbl_name, List<FieldSchema> cols,
+                                                     boolean applyDistinct, String filter,
+                                                     boolean ascending, List<FieldSchema> order,
+                                                     long maxParts) throws MetaException {
     return null;
   }
 
   @Override
-  public List<String> listPartitionNamesByFilter(String db_name, String tbl_name, String filter,
-      short max_parts) throws MetaException {
-
-    return Collections.emptyList();
-  }
-
-  @Override
-  public void alterPartition(String db_name, String tbl_name, List<String> part_vals,
+  public void alterPartition(String catName, String db_name, String tbl_name, List<String> part_vals,
       Partition new_part) throws InvalidObjectException, MetaException {
   }
 
   @Override
-  public void alterPartitions(String db_name, String tbl_name, List<List<String>> part_vals_list,
-      List<Partition> new_parts) throws InvalidObjectException, MetaException {
+  public void alterPartitions(String catName, String db_name, String tbl_name,
+                              List<List<String>> part_vals_list, List<Partition> new_parts)
+      throws InvalidObjectException, MetaException {
+
+
   }
 
   @Override
-  public List<Partition> getPartitionsByFilter(String dbName, String tblName, String filter,
-      short maxParts) throws MetaException, NoSuchObjectException {
+  public List<Partition> getPartitionsByFilter(String catName, String dbName, String tblName,
+                                               String filter, short maxParts)
+      throws MetaException, NoSuchObjectException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public List<Partition> getPartitionsByNames(String dbName, String tblName,
+  public List<Partition> getPartitionsByNames(String catName, String dbName, String tblName,
       List<String> partNames) throws MetaException, NoSuchObjectException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public boolean getPartitionsByExpr(String dbName, String tblName, byte[] expr,
+  public boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr,
       String defaultPartitionName, short maxParts, List<Partition> result) throws TException {
     return false;
   }
 
   @Override
-  public int getNumPartitionsByFilter(String dbName, String tblName, String filter)
+  public int getNumPartitionsByFilter(String catName, String dbName, String tblName, String filter)
     throws MetaException, NoSuchObjectException {
     return -1;
   }
 
   @Override
-  public int getNumPartitionsByExpr(String dbName, String tblName, byte[] expr)
+  public int getNumPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr)
       throws MetaException, NoSuchObjectException {
     return -1;
   }
 
   @Override
-  public Table markPartitionForEvent(String dbName, String tblName, Map<String, String> partVals,
+  public Table markPartitionForEvent(String catName, String dbName, String tblName, Map<String, String> partVals,
       PartitionEventType evtType) throws MetaException, UnknownTableException,
       InvalidPartitionException, UnknownPartitionException {
 
@@ -364,7 +388,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public boolean isPartitionMarkedForEvent(String dbName, String tblName,
+  public boolean isPartitionMarkedForEvent(String catName, String dbName, String tblName,
       Map<String, String> partName, PartitionEventType evtType) throws MetaException,
       UnknownTableException, InvalidPartitionException, UnknownPartitionException {
 
@@ -407,21 +431,21 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public PrincipalPrivilegeSet getDBPrivilegeSet(String dbName, String userName,
+  public PrincipalPrivilegeSet getDBPrivilegeSet(String catName, String dbName, String userName,
       List<String> groupNames) throws InvalidObjectException, MetaException {
 
     return null;
   }
 
   @Override
-  public PrincipalPrivilegeSet getTablePrivilegeSet(String dbName, String tableName,
+  public PrincipalPrivilegeSet getTablePrivilegeSet(String catName, String dbName, String tableName,
       String userName, List<String> groupNames) throws InvalidObjectException, MetaException {
 
     return null;
   }
 
   @Override
-  public PrincipalPrivilegeSet getPartitionPrivilegeSet(String dbName, String tableName,
+  public PrincipalPrivilegeSet getPartitionPrivilegeSet(String catName, String dbName, String tableName,
       String partition, String userName, List<String> groupNames) throws InvalidObjectException,
       MetaException {
 
@@ -429,7 +453,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public PrincipalPrivilegeSet getColumnPrivilegeSet(String dbName, String tableName,
+  public PrincipalPrivilegeSet getColumnPrivilegeSet(String catName, String dbName, String tableName,
       String partitionName, String columnName, String userName, List<String> groupNames)
       throws InvalidObjectException, MetaException {
 
@@ -445,21 +469,21 @@ public class DummyRawStoreForJdoConnection implements RawStore {
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalDBGrants(String principalName,
-      PrincipalType principalType, String dbName) {
+      PrincipalType principalType, String catName, String dbName) {
 
     return Collections.emptyList();
   }
 
   @Override
   public List<HiveObjectPrivilege> listAllTableGrants(String principalName,
-      PrincipalType principalType, String dbName, String tableName) {
+      PrincipalType principalType, String catName, String dbName, String tableName) {
 
     return Collections.emptyList();
   }
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalPartitionGrants(String principalName,
-      PrincipalType principalType, String dbName, String tableName, List<String> partValues,
+      PrincipalType principalType, String catName, String dbName, String tableName, List<String> partValues,
       String partName) {
 
     return Collections.emptyList();
@@ -467,14 +491,14 @@ public class DummyRawStoreForJdoConnection implements RawStore {
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalTableColumnGrants(String principalName,
-      PrincipalType principalType, String dbName, String tableName, String columnName) {
+      PrincipalType principalType, String catName, String dbName, String tableName, String columnName) {
 
     return Collections.emptyList();
   }
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrants(String principalName,
-      PrincipalType principalType, String dbName, String tableName, List<String> partVals,
+      PrincipalType principalType, String catName, String dbName, String tableName, List<String> partVals,
       String partName, String columnName) {
 
     return Collections.emptyList();
@@ -524,7 +548,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public Partition getPartitionWithAuth(String dbName, String tblName, List<String> partVals,
+  public Partition getPartitionWithAuth(String catName, String dbName, String tblName, List<String> partVals,
       String user_name, List<String> group_names) throws MetaException, NoSuchObjectException,
       InvalidObjectException {
 
@@ -532,7 +556,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public List<Partition> getPartitionsWithAuth(String dbName, String tblName, short maxParts,
+  public List<Partition> getPartitionsWithAuth(String catName, String dbName, String tblName, short maxParts,
       String userName, List<String> groupNames) throws MetaException, NoSuchObjectException,
       InvalidObjectException {
 
@@ -540,14 +564,14 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public List<String> listPartitionNamesPs(String db_name, String tbl_name, List<String> part_vals,
+  public List<String> listPartitionNamesPs(String catName, String db_name, String tbl_name, List<String> part_vals,
       short max_parts) throws MetaException, NoSuchObjectException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public List<Partition> listPartitionsPsWithAuth(String db_name, String tbl_name,
+  public List<Partition> listPartitionsPsWithAuth(String catName, String db_name, String tbl_name,
       List<String> part_vals, short max_parts, String userName, List<String> groupNames)
       throws MetaException, InvalidObjectException, NoSuchObjectException {
 
@@ -635,46 +659,46 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public List<HiveObjectPrivilege> listDBGrantsAll(String dbName) {
+  public List<HiveObjectPrivilege> listDBGrantsAll(String catName, String dbName) {
     return Collections.emptyList();
   }
 
   @Override
-  public List<HiveObjectPrivilege> listPartitionColumnGrantsAll(String dbName, String tableName, String partitionName, String columnName) {
+  public List<HiveObjectPrivilege> listPartitionColumnGrantsAll(String catName, String dbName, String tableName, String partitionName, String columnName) {
     return Collections.emptyList();
   }
 
   @Override
-  public List<HiveObjectPrivilege> listTableGrantsAll(String dbName, String tableName) {
+  public List<HiveObjectPrivilege> listTableGrantsAll(String catName, String dbName, String tableName) {
     return Collections.emptyList();
   }
 
   @Override
-  public List<HiveObjectPrivilege> listPartitionGrantsAll(String dbName, String tableName, String partitionName) {
+  public List<HiveObjectPrivilege> listPartitionGrantsAll(String catName, String dbName, String tableName, String partitionName) {
     return Collections.emptyList();
   }
 
   @Override
-  public List<HiveObjectPrivilege> listTableColumnGrantsAll(String dbName, String tableName, String columnName) {
+  public List<HiveObjectPrivilege> listTableColumnGrantsAll(String catName, String dbName, String tableName, String columnName) {
     return Collections.emptyList();
   }
 
   @Override
-  public  ColumnStatistics getTableColumnStatistics(String dbName, String tableName,
+  public  ColumnStatistics getTableColumnStatistics(String catName, String dbName, String tableName,
       List<String> colName) throws MetaException, NoSuchObjectException {
     return null;
   }
 
   @Override
-  public boolean deleteTableColumnStatistics(String dbName, String tableName,
-                                              String colName)
+  public boolean deleteTableColumnStatistics(String catName, String dbName, String tableName,
+                                             String colName)
       throws NoSuchObjectException, MetaException, InvalidObjectException {
     return false;
   }
 
 
   @Override
-  public boolean deletePartitionColumnStatistics(String dbName, String tableName,
+  public boolean deletePartitionColumnStatistics(String catName, String dbName, String tableName,
     String partName, List<String> partVals, String colName)
     throws NoSuchObjectException, MetaException, InvalidObjectException,
     InvalidInputException {
@@ -708,31 +732,31 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public List<ColumnStatistics> getPartitionColumnStatistics(String dbName,
+  public List<ColumnStatistics> getPartitionColumnStatistics(String catName, String dbName,
       String tblName, List<String> colNames, List<String> partNames)
       throws MetaException, NoSuchObjectException {
     return Collections.emptyList();
   }
 
   @Override
-  public boolean doesPartitionExist(String dbName, String tableName,
+  public boolean doesPartitionExist(String catName, String dbName, String tableName,
       List<String> partVals) throws MetaException, NoSuchObjectException {
     return false;
   }
 
   @Override
-  public boolean addPartitions(String dbName, String tblName, List<Partition> parts)
+  public boolean addPartitions(String catName, String dbName, String tblName, List<Partition> parts)
       throws InvalidObjectException, MetaException {
     return false;
   }
 
   @Override
-  public boolean addPartitions(String dbName, String tblName, PartitionSpecProxy partitionSpec, boolean ifNotExists) throws InvalidObjectException, MetaException {
+  public boolean addPartitions(String catName, String dbName, String tblName, PartitionSpecProxy partitionSpec, boolean ifNotExists) throws InvalidObjectException, MetaException {
     return false;
   }
 
   @Override
-  public void dropPartitions(String dbName, String tblName, List<String> partNames) {
+  public void dropPartitions(String catName, String dbName, String tblName, List<String> partNames) {
   }
 
   @Override
@@ -741,36 +765,36 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public void alterFunction(String dbName, String funcName, Function newFunction)
+  public void alterFunction(String catName, String dbName, String funcName, Function newFunction)
       throws InvalidObjectException, MetaException {
   }
 
   @Override
-  public void dropFunction(String dbName, String funcName)
+  public void dropFunction(String catName, String dbName, String funcName)
       throws MetaException, NoSuchObjectException, InvalidObjectException,
       InvalidInputException {
   }
 
   @Override
-  public Function getFunction(String dbName, String funcName)
+  public Function getFunction(String catName, String dbName, String funcName)
       throws MetaException {
     return null;
   }
 
   @Override
-  public List<Function> getAllFunctions()
+  public List<Function> getAllFunctions(String catName)
           throws MetaException {
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> getFunctions(String dbName, String pattern)
+  public List<String> getFunctions(String catName, String dbName, String pattern)
       throws MetaException {
     return Collections.emptyList();
   }
 
   @Override
-  public AggrStats get_aggr_stats_for(String dbName,
+  public AggrStats get_aggr_stats_for(String catName, String dbName,
       String tblName, List<String> partNames, List<String> colNames)
       throws MetaException {
     return null;
@@ -847,14 +871,14 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public List<SQLPrimaryKey> getPrimaryKeys(String db_name, String tbl_name)
+  public List<SQLPrimaryKey> getPrimaryKeys(String catName, String db_name, String tbl_name)
     throws MetaException {
     // TODO Auto-generated method stub
     return null;
   }
 
   @Override
-  public List<SQLForeignKey> getForeignKeys(String parent_db_name,
+  public List<SQLForeignKey> getForeignKeys(String catName, String parent_db_name,
     String parent_tbl_name, String foreign_db_name, String foreign_tbl_name)
     throws MetaException {
     // TODO Auto-generated method stub
@@ -862,28 +886,28 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public List<SQLUniqueConstraint> getUniqueConstraints(String db_name, String tbl_name)
+  public List<SQLUniqueConstraint> getUniqueConstraints(String catName, String db_name, String tbl_name)
     throws MetaException {
     // TODO Auto-generated method stub
     return null;
   }
 
   @Override
-  public List<SQLNotNullConstraint> getNotNullConstraints(String db_name, String tbl_name)
+  public List<SQLNotNullConstraint> getNotNullConstraints(String catName, String db_name, String tbl_name)
     throws MetaException {
     // TODO Auto-generated method stub
     return null;
   }
 
   @Override
-  public List<SQLDefaultConstraint> getDefaultConstraints(String db_name, String tbl_name)
+  public List<SQLDefaultConstraint> getDefaultConstraints(String catName, String db_name, String tbl_name)
       throws MetaException {
     // TODO Auto-generated method stub
     return null;
   }
 
   @Override
-  public List<SQLCheckConstraint> getCheckConstraints(String db_name, String tbl_name)
+  public List<SQLCheckConstraint> getCheckConstraints(String catName, String db_name, String tbl_name)
       throws MetaException {
     // TODO Auto-generated method stub
     return null;
@@ -902,8 +926,8 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public void dropConstraint(String dbName, String tableName,
-  String constraintName) throws NoSuchObjectException {
+  public void dropConstraint(String catName, String dbName, String tableName,
+  String constraintName, boolean missingOk) throws NoSuchObjectException {
     // TODO Auto-generated method stub
   }
 
@@ -1049,6 +1073,13 @@ public class DummyRawStoreForJdoConnection implements RawStore {
       String poolPath) throws NoSuchObjectException, InvalidOperationException, MetaException {
   }
 
+  @Override
+  public List<ColStatsObjWithSourceInfo> getPartitionColStatsForDatabase(String catName, String dbName)
+      throws MetaException, NoSuchObjectException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
   public void createISchema(ISchema schema) throws AlreadyExistsException, MetaException {
 
   }