public class HiveCatalog extends AbstractCatalog
Modifier and Type | Field and Description |
---|---|
static String |
DEFAULT_DB |
static String |
HIVE_SITE_FILE |
Constructor and Description |
---|
HiveCatalog(String catalogName,
String defaultDatabase,
HiveConf hiveConf,
String hiveVersion) |
HiveCatalog(String catalogName,
String defaultDatabase,
HiveConf hiveConf,
String hiveVersion,
boolean allowEmbedded) |
HiveCatalog(String catalogName,
String defaultDatabase,
String hiveConfDir) |
HiveCatalog(String catalogName,
String defaultDatabase,
String hiveConfDir,
String hiveVersion) |
HiveCatalog(String catalogName,
String defaultDatabase,
String hiveConfDir,
String hadoopConfDir,
String hiveVersion) |
Modifier and Type | Method and Description |
---|---|
void |
alterDatabase(String databaseName,
CatalogDatabase newDatabase,
boolean ignoreIfNotExists)
Modify an existing database.
|
void |
alterFunction(ObjectPath functionPath,
CatalogFunction newFunction,
boolean ignoreIfNotExists)
Modify an existing function.
|
void |
alterPartition(ObjectPath tablePath,
CatalogPartitionSpec partitionSpec,
CatalogPartition newPartition,
boolean ignoreIfNotExists)
Alter a partition.
|
void |
alterPartitionColumnStatistics(ObjectPath tablePath,
CatalogPartitionSpec partitionSpec,
CatalogColumnStatistics columnStatistics,
boolean ignoreIfNotExists)
Update the column statistics of a table partition.
|
void |
alterPartitionStatistics(ObjectPath tablePath,
CatalogPartitionSpec partitionSpec,
CatalogTableStatistics partitionStatistics,
boolean ignoreIfNotExists)
Update the statistics of a table partition.
|
void |
alterTable(ObjectPath tablePath,
CatalogBaseTable newCatalogTable,
boolean ignoreIfNotExists)
Modifies an existing table or view.
|
void |
alterTableColumnStatistics(ObjectPath tablePath,
CatalogColumnStatistics columnStatistics,
boolean ignoreIfNotExists)
Update the column statistics of a table.
|
void |
alterTableStatistics(ObjectPath tablePath,
CatalogTableStatistics tableStatistics,
boolean ignoreIfNotExists)
Update the statistics of a table.
|
List<CatalogColumnStatistics> |
bulkGetPartitionColumnStatistics(ObjectPath tablePath,
List<CatalogPartitionSpec> partitionSpecs)
Get a list of column statistics for given partitions.
|
List<CatalogTableStatistics> |
bulkGetPartitionStatistics(ObjectPath tablePath,
List<CatalogPartitionSpec> partitionSpecs)
Get a list of statistics of given partitions.
|
void |
close()
Close the catalog when it is no longer needed and release any resource that it might be
holding.
|
void |
createDatabase(String databaseName,
CatalogDatabase database,
boolean ignoreIfExists)
Create a database.
|
void |
createFunction(ObjectPath functionPath,
CatalogFunction function,
boolean ignoreIfExists)
Create a function.
|
static HiveConf |
createHiveConf(String hiveConfDir,
String hadoopConfDir) |
void |
createPartition(ObjectPath tablePath,
CatalogPartitionSpec partitionSpec,
CatalogPartition partition,
boolean ignoreIfExists)
Create a partition.
|
void |
createTable(ObjectPath tablePath,
CatalogBaseTable table,
boolean ignoreIfExists)
Creates a new table or view.
|
boolean |
databaseExists(String databaseName)
Check if a database exists in this catalog.
|
void |
dropDatabase(String name,
boolean ignoreIfNotExists,
boolean cascade)
Drop a database.
|
void |
dropFunction(ObjectPath functionPath,
boolean ignoreIfNotExists)
Drop a function.
|
void |
dropPartition(ObjectPath tablePath,
CatalogPartitionSpec partitionSpec,
boolean ignoreIfNotExists)
Drop a partition.
|
void |
dropTable(ObjectPath tablePath,
boolean ignoreIfNotExists)
Drop a table or view.
|
boolean |
functionExists(ObjectPath functionPath)
Check whether a function exists or not.
|
CatalogDatabase |
getDatabase(String databaseName)
Get a database from this catalog.
|
Optional<Factory> |
getFactory()
Returns a factory for creating instances from catalog objects.
|
static List<String> |
getFieldNames(List<org.apache.hadoop.hive.metastore.api.FieldSchema> fieldSchemas)
Get field names from field schemas.
|
CatalogFunction |
getFunction(ObjectPath functionPath)
Get the function.
|
Optional<FunctionDefinitionFactory> |
getFunctionDefinitionFactory()
Get an optional
FunctionDefinitionFactory instance that's responsible for
instantiating function definitions. |
HiveConf |
getHiveConf() |
org.apache.hadoop.hive.metastore.api.Database |
getHiveDatabase(String databaseName) |
org.apache.hadoop.hive.metastore.api.Partition |
getHivePartition(org.apache.hadoop.hive.metastore.api.Table hiveTable,
CatalogPartitionSpec partitionSpec) |
org.apache.hadoop.hive.metastore.api.Table |
getHiveTable(ObjectPath tablePath) |
String |
getHiveVersion() |
CatalogPartition |
getPartition(ObjectPath tablePath,
CatalogPartitionSpec partitionSpec)
Get a partition of the given table.
|
CatalogColumnStatistics |
getPartitionColumnStatistics(ObjectPath tablePath,
CatalogPartitionSpec partitionSpec)
Get the column statistics of a partition.
|
CatalogTableStatistics |
getPartitionStatistics(ObjectPath tablePath,
CatalogPartitionSpec partitionSpec)
Get the statistics of a partition.
|
CatalogBaseTable |
getTable(ObjectPath tablePath)
|
CatalogColumnStatistics |
getTableColumnStatistics(ObjectPath tablePath)
Get the column statistics of a table.
|
Optional<TableFactory> |
getTableFactory()
Get an optional
TableFactory instance that's responsible for generating table-related
instances stored in this catalog, instances such as source/sink. |
CatalogTableStatistics |
getTableStatistics(ObjectPath tablePath)
Get the statistics of a table.
|
static boolean |
isEmbeddedMetastore(HiveConf hiveConf) |
static boolean |
isHiveTable(Map<String,String> properties) |
static boolean |
isHiveTable(org.apache.hadoop.hive.metastore.api.Table table) |
List<String> |
listDatabases()
Get the names of all databases in this catalog.
|
List<String> |
listFunctions(String databaseName)
List the names of all functions in the given database.
|
List<CatalogPartitionSpec> |
listPartitions(ObjectPath tablePath)
Get CatalogPartitionSpec of all partitions of the table.
|
List<CatalogPartitionSpec> |
listPartitions(ObjectPath tablePath,
CatalogPartitionSpec partitionSpec)
Get CatalogPartitionSpec of all partitions that is under the given CatalogPartitionSpec in
the table.
|
List<CatalogPartitionSpec> |
listPartitionsByFilter(ObjectPath tablePath,
List<Expression> expressions)
Get CatalogPartitionSpec of partitions by expression filters in the table.
|
List<String> |
listTables(String databaseName)
Get names of all tables and views under this database.
|
List<String> |
listViews(String databaseName)
Get names of all views under this database.
|
void |
loadPartition(org.apache.hadoop.fs.Path loadPath,
ObjectPath tablePath,
Map<String,String> partSpec,
boolean isOverwrite,
boolean isSrcLocal) |
void |
loadTable(org.apache.hadoop.fs.Path loadPath,
ObjectPath tablePath,
boolean isOverwrite,
boolean isSrcLocal) |
void |
open()
Open the catalog.
|
boolean |
partitionExists(ObjectPath tablePath,
CatalogPartitionSpec partitionSpec)
Check whether a partition exists or not.
|
void |
renameTable(ObjectPath tablePath,
String newTableName,
boolean ignoreIfNotExists)
Rename an existing table or view.
|
boolean |
supportsManagedTable()
If true, tables which do not specify a connector will be translated to managed tables.
|
boolean |
tableExists(ObjectPath tablePath)
Check if a table or view exists in this catalog.
|
getDefaultDatabase, getName
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
alterTable, dropDatabase, getProcedure, getTable, listProcedures
public static final String DEFAULT_DB
public static final String HIVE_SITE_FILE
public HiveCatalog(String catalogName, @Nullable String defaultDatabase, @Nullable String hiveConfDir)
public HiveCatalog(String catalogName, @Nullable String defaultDatabase, @Nullable String hiveConfDir, String hiveVersion)
public HiveCatalog(String catalogName, @Nullable String defaultDatabase, @Nullable String hiveConfDir, @Nullable String hadoopConfDir, @Nullable String hiveVersion)
public HiveCatalog(String catalogName, @Nullable String defaultDatabase, @Nullable HiveConf hiveConf, @Nullable String hiveVersion)
public static HiveConf createHiveConf(@Nullable String hiveConfDir, @Nullable String hadoopConfDir)
public HiveConf getHiveConf()
public void open() throws CatalogException
Catalog
CatalogException
- in case of any runtime exceptionpublic void close() throws CatalogException
Catalog
CatalogException
- in case of any runtime exceptionpublic Optional<Factory> getFactory()
Catalog
This method enables bypassing the discovery process. Implementers can directly pass
internal catalog-specific objects to their own factory. For example, a custom CatalogTable
can be processed by a custom DynamicTableFactory
.
If this catalog support to create materialized table, you should also override this method
to provide DynamicTableFactory
which help planner to find DynamicTableSource
and DynamicTableSink
correctly during compile optimization phase. If you don't
override this method, you must specify the physical connector identifier that this catalog
represents storage when create materialized table. Otherwise, the planner can't find the
DynamicTableFactory
.
Because all factories are interfaces, the returned Factory
instance can implement
multiple supported extension points. An instanceof
check is performed by the caller
that checks whether a required factory is implemented; otherwise the discovery process is
used.
public Optional<TableFactory> getTableFactory()
Catalog
TableFactory
instance that's responsible for generating table-related
instances stored in this catalog, instances such as source/sink.public Optional<FunctionDefinitionFactory> getFunctionDefinitionFactory()
Catalog
FunctionDefinitionFactory
instance that's responsible for
instantiating function definitions.public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException
Catalog
databaseName
- Name of the databaseDatabaseNotExistException
- if the database does not existCatalogException
- in case of any runtime exceptionpublic void createDatabase(String databaseName, CatalogDatabase database, boolean ignoreIfExists) throws DatabaseAlreadyExistException, CatalogException
Catalog
databaseName
- Name of the database to be createddatabase
- The database definitionignoreIfExists
- Flag to specify behavior when a database with the given name already
exists: if set to false, throw a DatabaseAlreadyExistException, if set to true, do
nothing.DatabaseAlreadyExistException
- if the given database already exists and ignoreIfExists
is falseCatalogException
- in case of any runtime exceptionpublic void alterDatabase(String databaseName, CatalogDatabase newDatabase, boolean ignoreIfNotExists) throws DatabaseNotExistException, CatalogException
Catalog
databaseName
- Name of the database to be modifiednewDatabase
- The new database definitionignoreIfNotExists
- Flag to specify behavior when the given database does not exist: if
set to false, throw an exception, if set to true, do nothing.DatabaseNotExistException
- if the given database does not existCatalogException
- in case of any runtime exceptionpublic List<String> listDatabases() throws CatalogException
Catalog
CatalogException
- in case of any runtime exceptionpublic boolean databaseExists(String databaseName) throws CatalogException
Catalog
databaseName
- Name of the databaseCatalogException
- in case of any runtime exceptionpublic void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException
Catalog
name
- Name of the database to be dropped.ignoreIfNotExists
- Flag to specify behavior when the database does not exist: if set to
false, throw an exception, if set to true, do nothing.cascade
- Flag to specify behavior when the database contains table or function: if set
to true, delete all tables and functions in the database and then delete the database, if
set to false, throw an exception.DatabaseNotExistException
- if the given database does not existDatabaseNotEmptyException
- if the given database is not empty and isRestrict is trueCatalogException
- in case of any runtime exception@VisibleForTesting public org.apache.hadoop.hive.metastore.api.Database getHiveDatabase(String databaseName) throws DatabaseNotExistException
DatabaseNotExistException
public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException
Catalog
CatalogTable
or CatalogView
identified by the given ObjectPath
. The framework will resolve the metadata objects when necessary.tablePath
- Path of the table or viewTableNotExistException
- if the target does not existCatalogException
- in case of any runtime exceptionpublic void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) throws TableAlreadyExistException, DatabaseNotExistException, CatalogException
Catalog
The framework will make sure to call this method with fully validated ResolvedCatalogTable
or ResolvedCatalogView
. Those instances are easy to serialize
for a durable catalog implementation.
tablePath
- path of the table or view to be createdtable
- the table definitionignoreIfExists
- flag to specify behavior when a table or view already exists at the
given path: if set to false, it throws a TableAlreadyExistException, if set to true, do
nothing.TableAlreadyExistException
- if table already exists and ignoreIfExists is falseDatabaseNotExistException
- if the database in tablePath doesn't existCatalogException
- in case of any runtime exceptionpublic void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) throws TableNotExistException, TableAlreadyExistException, CatalogException
Catalog
tablePath
- Path of the table or view to be renamednewTableName
- the new name of the table or viewignoreIfNotExists
- Flag to specify behavior when the table or view does not exist: if
set to false, throw an exception, if set to true, do nothing.TableNotExistException
- if the table does not existCatalogException
- in case of any runtime exceptionTableAlreadyExistException
public void alterTable(ObjectPath tablePath, CatalogBaseTable newCatalogTable, boolean ignoreIfNotExists) throws TableNotExistException, CatalogException
Catalog
CatalogBaseTable
must
be of the same kind. For example, this doesn't allow altering a regular table to partitioned
table, or altering a view to a table, and vice versa.
The framework will make sure to call this method with fully validated ResolvedCatalogTable
or ResolvedCatalogView
. Those instances are easy to serialize
for a durable catalog implementation.
tablePath
- path of the table or view to be modifiednewCatalogTable
- the new table definitionignoreIfNotExists
- flag to specify behavior when the table or view does not exist: if
set to false, throw an exception, if set to true, do nothing.TableNotExistException
- if the table does not existCatalogException
- in case of any runtime exceptionpublic void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) throws TableNotExistException, CatalogException
Catalog
tablePath
- Path of the table or view to be droppedignoreIfNotExists
- Flag to specify behavior when the table or view does not exist: if
set to false, throw an exception, if set to true, do nothing.TableNotExistException
- if the table or view does not existCatalogException
- in case of any runtime exceptionpublic List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException
Catalog
DatabaseNotExistException
- if the database does not existCatalogException
- in case of any runtime exceptionpublic List<String> listViews(String databaseName) throws DatabaseNotExistException, CatalogException
Catalog
databaseName
- the name of the given databaseDatabaseNotExistException
- if the database does not existCatalogException
- in case of any runtime exceptionpublic boolean tableExists(ObjectPath tablePath) throws CatalogException
Catalog
tablePath
- Path of the table or viewCatalogException
- in case of any runtime exception@Internal public org.apache.hadoop.hive.metastore.api.Table getHiveTable(ObjectPath tablePath) throws TableNotExistException
TableNotExistException
public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException
Catalog
tablePath
- path of the tablepartitionSpec
- partition spec of the partition to checkCatalogException
- in case of any runtime exceptionpublic void createPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogPartition partition, boolean ignoreIfExists) throws TableNotExistException, TableNotPartitionedException, PartitionSpecInvalidException, PartitionAlreadyExistsException, CatalogException
Catalog
tablePath
- path of the table.partitionSpec
- partition spec of the partitionpartition
- the partition to add.ignoreIfExists
- flag to specify behavior if a partition with the given name already
exists: if set to false, it throws a PartitionAlreadyExistsException, if set to true,
nothing happens.TableNotExistException
- thrown if the target table does not existTableNotPartitionedException
- thrown if the target table is not partitionedPartitionSpecInvalidException
- thrown if the given partition spec is invalidPartitionAlreadyExistsException
- thrown if the target partition already existsCatalogException
- in case of any runtime exceptionpublic void dropPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) throws PartitionNotExistException, CatalogException
Catalog
tablePath
- path of the table.partitionSpec
- partition spec of the partition to dropignoreIfNotExists
- flag to specify behavior if the partition does not exist: if set to
false, throw an exception, if set to true, nothing happens.PartitionNotExistException
- thrown if the target partition does not existCatalogException
- in case of any runtime exceptionpublic List<CatalogPartitionSpec> listPartitions(ObjectPath tablePath) throws TableNotExistException, TableNotPartitionedException, CatalogException
Catalog
tablePath
- path of the tableTableNotExistException
- thrown if the table does not exist in the catalogTableNotPartitionedException
- thrown if the table is not partitionedCatalogException
- in case of any runtime exceptionpublic List<CatalogPartitionSpec> listPartitions(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws TableNotExistException, TableNotPartitionedException, PartitionSpecInvalidException, CatalogException
Catalog
tablePath
- path of the tablepartitionSpec
- the partition spec to listTableNotExistException
- thrown if the table does not exist in the catalogTableNotPartitionedException
- thrown if the table is not partitionedCatalogException
- in case of any runtime exceptionPartitionSpecInvalidException
public List<CatalogPartitionSpec> listPartitionsByFilter(ObjectPath tablePath, List<Expression> expressions) throws TableNotExistException, TableNotPartitionedException, CatalogException
Catalog
NOTE: For FieldReferenceExpression, the field index is based on schema of this table instead of partition columns only.
The passed in predicates have been translated in conjunctive form.
If catalog does not support this interface at present, throw an UnsupportedOperationException
directly. If the catalog does not have a valid filter, throw
the UnsupportedOperationException
directly. Planner will fallback to get all
partitions and filter by itself.
tablePath
- path of the tableexpressions
- filters to push down filter to catalogTableNotExistException
- thrown if the table does not exist in the catalogTableNotPartitionedException
- thrown if the table is not partitionedCatalogException
- in case of any runtime exceptionpublic CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws PartitionNotExistException, CatalogException
Catalog
tablePath
- path of the tablepartitionSpec
- partition spec of partition to getPartitionNotExistException
- thrown if the partition doesn't existCatalogException
- in case of any runtime exceptionpublic void alterPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogPartition newPartition, boolean ignoreIfNotExists) throws PartitionNotExistException, CatalogException
Catalog
tablePath
- path of the tablepartitionSpec
- partition spec of the partitionnewPartition
- new partition to replace the old oneignoreIfNotExists
- flag to specify behavior if the partition does not exist: if set to
false, throw an exception, if set to true, nothing happens.PartitionNotExistException
- thrown if the target partition does not existCatalogException
- in case of any runtime exceptionpublic static List<String> getFieldNames(List<org.apache.hadoop.hive.metastore.api.FieldSchema> fieldSchemas)
@VisibleForTesting public org.apache.hadoop.hive.metastore.api.Partition getHivePartition(org.apache.hadoop.hive.metastore.api.Table hiveTable, CatalogPartitionSpec partitionSpec) throws PartitionSpecInvalidException, org.apache.thrift.TException
PartitionSpecInvalidException
org.apache.thrift.TException
public void createFunction(ObjectPath functionPath, CatalogFunction function, boolean ignoreIfExists) throws FunctionAlreadyExistException, DatabaseNotExistException, CatalogException
Catalog
functionPath
- path of the functionfunction
- the function to be createdignoreIfExists
- flag to specify behavior if a function with the given name already
exists: if set to false, it throws a FunctionAlreadyExistException, if set to true,
nothing happens.FunctionAlreadyExistException
- if the function already existDatabaseNotExistException
- if the given database does not existCatalogException
- in case of any runtime exceptionpublic void alterFunction(ObjectPath functionPath, CatalogFunction newFunction, boolean ignoreIfNotExists) throws FunctionNotExistException, CatalogException
Catalog
functionPath
- path of the functionnewFunction
- the function to be modifiedignoreIfNotExists
- flag to specify behavior if the function does not exist: if set to
false, throw an exception if set to true, nothing happensFunctionNotExistException
- if the function does not existCatalogException
- in case of any runtime exceptionpublic void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) throws FunctionNotExistException, CatalogException
Catalog
functionPath
- path of the function to be droppedignoreIfNotExists
- flag to specify behavior if the function does not exist: if set to
false, throw an exception if set to true, nothing happensFunctionNotExistException
- if the function does not existCatalogException
- in case of any runtime exceptionpublic List<String> listFunctions(String databaseName) throws DatabaseNotExistException, CatalogException
Catalog
databaseName
- name of the database.DatabaseNotExistException
- if the database does not existCatalogException
- in case of any runtime exceptionpublic CatalogFunction getFunction(ObjectPath functionPath) throws FunctionNotExistException, CatalogException
Catalog
functionPath
- path of the functionFunctionNotExistException
- if the function does not exist in the catalogCatalogException
- in case of any runtime exceptionpublic boolean functionExists(ObjectPath functionPath) throws CatalogException
Catalog
functionPath
- path of the functionCatalogException
- in case of any runtime exceptionpublic void alterTableStatistics(ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) throws TableNotExistException, CatalogException
Catalog
tablePath
- path of the tabletableStatistics
- new statistics to updateignoreIfNotExists
- flag to specify behavior if the table does not exist: if set to
false, throw an exception, if set to true, nothing happens.TableNotExistException
- if the table does not exist in the catalogCatalogException
- in case of any runtime exceptionpublic void alterTableColumnStatistics(ObjectPath tablePath, CatalogColumnStatistics columnStatistics, boolean ignoreIfNotExists) throws TableNotExistException, CatalogException, TablePartitionedException
Catalog
tablePath
- path of the tablecolumnStatistics
- new column statistics to updateignoreIfNotExists
- flag to specify behavior if the table does not exist: if set to
false, throw an exception, if set to true, nothing happens.TableNotExistException
- if the table does not exist in the catalogCatalogException
- in case of any runtime exceptionTablePartitionedException
public void alterPartitionStatistics(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogTableStatistics partitionStatistics, boolean ignoreIfNotExists) throws PartitionNotExistException, CatalogException
Catalog
tablePath
- path of the tablepartitionSpec
- partition spec of the partitionpartitionStatistics
- new statistics to updateignoreIfNotExists
- flag to specify behavior if the partition does not exist: if set to
false, throw an exception, if set to true, nothing happens.PartitionNotExistException
- if the partition does not existCatalogException
- in case of any runtime exceptionpublic void alterPartitionColumnStatistics(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogColumnStatistics columnStatistics, boolean ignoreIfNotExists) throws PartitionNotExistException, CatalogException
Catalog
tablePath
- path of the tablepartitionSpec
- partition spec of the partition @@param columnStatistics new column
statistics to updateignoreIfNotExists
- flag to specify behavior if the partition does not exist: if set to
false, throw an exception, if set to true, nothing happens.PartitionNotExistException
- if the partition does not existCatalogException
- in case of any runtime exceptionpublic CatalogTableStatistics getTableStatistics(ObjectPath tablePath) throws TableNotExistException, CatalogException
Catalog
tablePath
- path of the tableTableNotExistException
- if the table does not exist in the catalogCatalogException
- in case of any runtime exceptionpublic CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) throws TableNotExistException, CatalogException
Catalog
tablePath
- path of the tableTableNotExistException
- if the table does not exist in the catalogCatalogException
- in case of any runtime exceptionpublic CatalogTableStatistics getPartitionStatistics(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws PartitionNotExistException, CatalogException
Catalog
tablePath
- path of the tablepartitionSpec
- partition spec of the partitionPartitionNotExistException
- if the partition does not existCatalogException
- in case of any runtime exceptionpublic List<CatalogTableStatistics> bulkGetPartitionStatistics(ObjectPath tablePath, List<CatalogPartitionSpec> partitionSpecs) throws PartitionNotExistException, CatalogException
Catalog
tablePath
- path of the tablepartitionSpecs
- partition specs of partitions that will be used to filter out all other
unrelated statistics, i.e. the statistics fetch will be limited within the given
partitionsPartitionNotExistException
- if one partition does not existCatalogException
- in case of any runtime exceptionpublic CatalogColumnStatistics getPartitionColumnStatistics(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws PartitionNotExistException, CatalogException
Catalog
tablePath
- path of the tablepartitionSpec
- partition spec of the partitionPartitionNotExistException
- if the partition does not existCatalogException
- in case of any runtime exceptionpublic List<CatalogColumnStatistics> bulkGetPartitionColumnStatistics(ObjectPath tablePath, List<CatalogPartitionSpec> partitionSpecs) throws PartitionNotExistException, CatalogException
Catalog
tablePath
- path of the tablepartitionSpecs
- partition specs of partitions that will be used to filter out all other
unrelated statistics, i.e. the statistics fetch will be limited within the given
partitionsPartitionNotExistException
- if one partition does not existCatalogException
- in case of any runtime exceptionpublic boolean supportsManagedTable()
Catalog
@Internal public static boolean isHiveTable(org.apache.hadoop.hive.metastore.api.Table table)
@Internal public void loadTable(org.apache.hadoop.fs.Path loadPath, ObjectPath tablePath, boolean isOverwrite, boolean isSrcLocal)
@Internal public void loadPartition(org.apache.hadoop.fs.Path loadPath, ObjectPath tablePath, Map<String,String> partSpec, boolean isOverwrite, boolean isSrcLocal)
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.