@InterfaceAudience.Public @InterfaceStability.Unstable public class HiveMetaStoreClient extends Object implements IMetaStoreClient
IMetaStoreClient.IncompatibleMetastoreException, IMetaStoreClient.NotificationFilter
Modifier and Type | Field and Description |
---|---|
protected HiveConf |
conf |
protected static org.apache.commons.logging.Log |
LOG |
Constructor and Description |
---|
HiveMetaStoreClient(HiveConf conf) |
HiveMetaStoreClient(HiveConf conf,
HiveMetaHookLoader hookLoader) |
Modifier and Type | Method and Description |
---|---|
Partition |
add_partition(Partition new_part)
Add a partition to the table.
|
Partition |
add_partition(Partition new_part,
EnvironmentContext envContext) |
int |
add_partitions_pspec(PartitionSpecProxy partitionSpec) |
int |
add_partitions(List<Partition> new_parts)
Add partitions to the table.
|
List<Partition> |
add_partitions(List<Partition> parts,
boolean ifNotExists,
boolean needResults)
Add partitions to the table.
|
void |
addDynamicPartitions(long txnId,
String dbName,
String tableName,
List<String> partNames)
Send a list of partitions to the metastore to indicate which partitions were loaded
dynamically.
|
void |
alter_index(String dbname,
String base_tbl_name,
String idx_name,
Index new_idx) |
void |
alter_partition(String dbName,
String tblName,
Partition newPart)
updates a partition to new partition
|
void |
alter_partitions(String dbName,
String tblName,
List<Partition> newParts)
updates a list of partitions
|
void |
alter_table(String dbname,
String tbl_name,
Table new_tbl) |
void |
alter_table(String dbname,
String tbl_name,
Table new_tbl,
boolean cascade) |
void |
alter_table(String dbname,
String tbl_name,
Table new_tbl,
EnvironmentContext envContext) |
void |
alterDatabase(String dbName,
Database db) |
void |
alterFunction(String dbName,
String funcName,
Function newFunction) |
Partition |
appendPartition(String db_name,
String table_name,
List<String> part_vals) |
Partition |
appendPartition(String db_name,
String table_name,
List<String> part_vals,
EnvironmentContext envContext) |
Partition |
appendPartition(String dbName,
String tableName,
String partName) |
Partition |
appendPartition(String dbName,
String tableName,
String partName,
EnvironmentContext envContext) |
Partition |
appendPartitionByName(String dbName,
String tableName,
String partName) |
Partition |
appendPartitionByName(String dbName,
String tableName,
String partName,
EnvironmentContext envContext) |
void |
cancelDelegationToken(String tokenStrForm) |
LockResponse |
checkLock(long lockid)
Check the status of a set of locks requested via a
IMetaStoreClient.lock(org.apache.hadoop.hive.metastore.api.LockRequest) call. |
void |
close()
close connection to meta store
|
void |
commitTxn(long txnid)
Commit a transaction.
|
void |
compact(String dbname,
String tableName,
String partitionName,
CompactionType type)
Send a request to compact a table or partition.
|
boolean |
create_role(Role role) |
protected void |
create_table_with_environment_context(Table tbl,
EnvironmentContext envContext) |
void |
createDatabase(Database db)
Create a new Database
|
void |
createFunction(Function func) |
void |
createIndex(Index index,
Table indexTable)
create an index
|
void |
createTable(Table tbl) |
void |
createTable(Table tbl,
EnvironmentContext envContext) |
boolean |
createType(Type type) |
protected PrincipalPrivilegeSet |
deepCopy(PrincipalPrivilegeSet pps) |
protected Table |
deepCopy(Table table) |
protected List<FieldSchema> |
deepCopyFieldSchemas(List<FieldSchema> schemas) |
boolean |
deletePartitionColumnStatistics(String dbName,
String tableName,
String partName,
String colName)
Delete partition level column statistics given dbName, tableName, partName and colName
|
boolean |
deleteTableColumnStatistics(String dbName,
String tableName,
String colName)
Delete table level column statistics given dbName, tableName and colName
|
boolean |
drop_role(String roleName) |
protected void |
drop_table_with_environment_context(String dbname,
String name,
boolean deleteData,
EnvironmentContext envContext) |
void |
dropDatabase(String name) |
void |
dropDatabase(String name,
boolean deleteData,
boolean ignoreUnknownDb) |
void |
dropDatabase(String name,
boolean deleteData,
boolean ignoreUnknownDb,
boolean cascade) |
void |
dropFunction(String dbName,
String funcName) |
boolean |
dropIndex(String dbName,
String tblName,
String name,
boolean deleteData) |
boolean |
dropPartition(String db_name,
String tbl_name,
List<String> part_vals) |
boolean |
dropPartition(String db_name,
String tbl_name,
List<String> part_vals,
boolean deleteData) |
boolean |
dropPartition(String db_name,
String tbl_name,
List<String> part_vals,
boolean deleteData,
EnvironmentContext envContext) |
boolean |
dropPartition(String db_name,
String tbl_name,
List<String> part_vals,
EnvironmentContext env_context) |
boolean |
dropPartition(String db_name,
String tbl_name,
List<String> part_vals,
PartitionDropOptions options)
Method to dropPartitions() with the option to purge the partition data directly,
rather than to move data to trash.
|
boolean |
dropPartition(String dbName,
String tableName,
String partName,
boolean deleteData) |
boolean |
dropPartition(String dbName,
String tableName,
String partName,
boolean deleteData,
EnvironmentContext envContext) |
boolean |
dropPartitionByName(String dbName,
String tableName,
String partName,
boolean deleteData) |
boolean |
dropPartitionByName(String dbName,
String tableName,
String partName,
boolean deleteData,
EnvironmentContext envContext) |
List<Partition> |
dropPartitions(String dbName,
String tblName,
List<ObjectPair<Integer,byte[]>> partExprs,
boolean deleteData,
boolean ignoreProtection,
boolean ifExists) |
List<Partition> |
dropPartitions(String dbName,
String tblName,
List<ObjectPair<Integer,byte[]>> partExprs,
boolean deleteData,
boolean ignoreProtection,
boolean ifExists,
boolean needResult) |
List<Partition> |
dropPartitions(String dbName,
String tblName,
List<ObjectPair<Integer,byte[]>> partExprs,
PartitionDropOptions options)
Generalization of dropPartitions(),
|
void |
dropTable(String tableName,
boolean deleteData)
Deprecated.
|
void |
dropTable(String dbname,
String name) |
void |
dropTable(String dbname,
String name,
boolean deleteData,
boolean ignoreUnknownTab)
Drop the table.
|
void |
dropTable(String dbname,
String name,
boolean deleteData,
boolean ignoreUnknownTab,
boolean ifPurge)
Drop the table and choose whether to save the data in the trash.
|
void |
dropTable(String dbname,
String name,
boolean deleteData,
boolean ignoreUnknownTab,
EnvironmentContext envContext)
Drop the table and choose whether to: delete the underlying table data;
throw if the table doesn't exist; save the data in the trash.
|
boolean |
dropType(String type) |
Partition |
exchange_partition(Map<String,String> partitionSpecs,
String sourceDb,
String sourceTable,
String destDb,
String destinationTableName)
Exchange the partition between two tables
|
FireEventResponse |
fireListenerEvent(FireEventRequest rqst)
Request that the metastore fire an event.
|
GetPrincipalsInRoleResponse |
get_principals_in_role(GetPrincipalsInRoleRequest req)
get all role-grants for users/roles that have been granted the given role
Note that in the returned list of RolePrincipalGrants, the roleName is
redundant as it would match the role_name argument of this function
|
PrincipalPrivilegeSet |
get_privilege_set(HiveObjectRef hiveObject,
String userName,
List<String> groupNames)
Return the privileges that the user, group have directly and indirectly through roles
on the given hiveObject
|
GetRoleGrantsForPrincipalResponse |
get_role_grants_for_principal(GetRoleGrantsForPrincipalRequest getRolePrincReq)
get all role-grants for roles that have been granted to given principal
Note that in the returned list of RolePrincipalGrants, the principal information
redundant as it would match the principal information in request
|
AggrStats |
getAggrColStatsFor(String dbName,
String tblName,
List<String> colNames,
List<String> partNames) |
List<String> |
getAllDatabases()
Get the names of all databases in the MetaStore.
|
List<String> |
getAllTables(String dbname)
Get the names of all tables in the specified database.
|
String |
getConfigValue(String name,
String defaultValue) |
CurrentNotificationEventId |
getCurrentNotificationEventId()
Get the last used notification event id.
|
Database |
getDatabase(String name)
Get a Database Object
|
List<String> |
getDatabases(String databasePattern)
Get the names of all databases in the MetaStore that match the given pattern.
|
String |
getDelegationToken(String renewerKerberosPrincipalName) |
String |
getDelegationToken(String owner,
String renewerKerberosPrincipalName)
This is expected to be a no-op when in local mode,
which means that the implementation will return null.
|
List<FieldSchema> |
getFields(String db,
String tableName) |
Function |
getFunction(String dbName,
String funcName) |
List<String> |
getFunctions(String dbName,
String pattern) |
Index |
getIndex(String dbName,
String tblName,
String indexName) |
String |
getMetaConf(String key)
get current meta variable
|
NotificationEventResponse |
getNextNotification(long lastEventId,
int maxEvents,
IMetaStoreClient.NotificationFilter filter)
Get the next set of notifications from the database.
|
Partition |
getPartition(String db_name,
String tbl_name,
List<String> part_vals) |
Partition |
getPartition(String db,
String tableName,
String partName) |
Map<String,List<ColumnStatisticsObj>> |
getPartitionColumnStatistics(String dbName,
String tableName,
List<String> partNames,
List<String> colNames)
Get partitions column statistics given dbName, tableName, multiple partitions and colName-s
|
List<Partition> |
getPartitionsByNames(String db_name,
String tbl_name,
List<String> part_names)
Get partitions by a list of partition names.
|
Partition |
getPartitionWithAuthInfo(String db_name,
String tbl_name,
List<String> part_vals,
String user_name,
List<String> group_names) |
List<FieldSchema> |
getSchema(String db,
String tableName) |
Table |
getTable(String tableName)
Deprecated.
|
Table |
getTable(String dbname,
String name)
Get a table object.
|
List<ColumnStatisticsObj> |
getTableColumnStatistics(String dbName,
String tableName,
List<String> colNames)
Get table column statistics given dbName, tableName and multiple colName-s
|
List<Table> |
getTableObjectsByName(String dbName,
List<String> tableNames) |
List<String> |
getTables(String dbname,
String tablePattern)
Get the names of all tables in the specified database that satisfy the supplied
table name pattern.
|
String |
getTokenStrForm() |
Type |
getType(String name) |
Map<String,Type> |
getTypeAll(String name) |
ValidTxnList |
getValidTxns()
Get a structure that details valid transactions.
|
ValidTxnList |
getValidTxns(long currentTxn)
Get a structure that details valid transactions.
|
boolean |
grant_privileges(PrivilegeBag privileges) |
boolean |
grant_role(String roleName,
String userName,
PrincipalType principalType,
String grantor,
PrincipalType grantorType,
boolean grantOption) |
void |
heartbeat(long txnid,
long lockid)
Send a heartbeat to indicate that the client holding these locks (if
any) and that opened this transaction (if one exists) is still alive.
|
HeartbeatTxnRangeResponse |
heartbeatTxnRange(long min,
long max)
Send heartbeats for a range of transactions.
|
boolean |
isCompatibleWith(HiveConf conf)
Returns whether current client is compatible with conf argument or not
|
boolean |
isPartitionMarkedForEvent(String db_name,
String tbl_name,
Map<String,String> partKVs,
PartitionEventType eventType) |
List<HiveObjectPrivilege> |
list_privileges(String principalName,
PrincipalType principalType,
HiveObjectRef hiveObject)
Return the privileges that this principal has directly over the object (not through roles).
|
List<Role> |
list_roles(String principalName,
PrincipalType principalType) |
List<Index> |
listIndexes(String dbName,
String tblName,
short max)
list all the index names of the give base table.
|
List<String> |
listIndexNames(String dbName,
String tblName,
short max)
list indexes of the give base table
|
List<String> |
listPartitionNames(String db_name,
String tbl_name,
List<String> part_vals,
short max_parts) |
List<String> |
listPartitionNames(String dbName,
String tblName,
short max) |
List<Partition> |
listPartitions(String db_name,
String tbl_name,
List<String> part_vals,
short max_parts) |
List<Partition> |
listPartitions(String db_name,
String tbl_name,
short max_parts) |
boolean |
listPartitionsByExpr(String db_name,
String tbl_name,
byte[] expr,
String default_partition_name,
short max_parts,
List<Partition> result)
Get list of partitions matching specified serialized expression
|
List<Partition> |
listPartitionsByFilter(String db_name,
String tbl_name,
String filter,
short max_parts)
Get list of partitions matching specified filter
|
PartitionSpecProxy |
listPartitionSpecs(String dbName,
String tableName,
int maxParts) |
PartitionSpecProxy |
listPartitionSpecsByFilter(String db_name,
String tbl_name,
String filter,
int max_parts) |
List<Partition> |
listPartitionsWithAuthInfo(String db_name,
String tbl_name,
List<String> part_vals,
short max_parts,
String user_name,
List<String> group_names) |
List<Partition> |
listPartitionsWithAuthInfo(String db_name,
String tbl_name,
short max_parts,
String user_name,
List<String> group_names) |
List<String> |
listRoleNames()
list all role names
|
List<String> |
listTableNamesByFilter(String dbName,
String filter,
short maxTables)
Get a list of table names that match a filter.
|
LockResponse |
lock(LockRequest request)
Request a set of locks.
|
void |
markPartitionForEvent(String db_name,
String tbl_name,
Map<String,String> partKVs,
PartitionEventType eventType) |
static IMetaStoreClient |
newSynchronizedClient(IMetaStoreClient client)
Creates a synchronized wrapper for any
IMetaStoreClient . |
long |
openTxn(String user)
Initiate a transaction.
|
OpenTxnsResponse |
openTxns(String user,
int numTxns)
Initiate a batch of transactions.
|
Map<String,String> |
partitionNameToSpec(String name) |
List<String> |
partitionNameToVals(String name) |
void |
reconnect()
Tries to reconnect this MetaStoreClient to the MetaStore.
|
void |
renamePartition(String dbname,
String name,
List<String> part_vals,
Partition newPart)
rename a partition to a new partition
|
long |
renewDelegationToken(String tokenStrForm) |
boolean |
revoke_privileges(PrivilegeBag privileges,
boolean grantOption) |
boolean |
revoke_role(String roleName,
String userName,
PrincipalType principalType,
boolean grantOption) |
void |
rollbackTxn(long txnid)
Rollback a transaction.
|
void |
setHiveAddedJars(String addedJars)
Set added jars path info to MetaStoreClient.
|
void |
setMetaConf(String key,
String value)
set meta variable which is open to end users
|
boolean |
setPartitionColumnStatistics(SetPartitionsStatsRequest request) |
ShowCompactResponse |
showCompactions()
Get a list of all current compactions.
|
ShowLocksResponse |
showLocks()
Show all currently held and waiting locks.
|
GetOpenTxnsInfoResponse |
showTxns()
Show the list of currently open transactions.
|
boolean |
tableExists(String tableName)
Deprecated.
|
boolean |
tableExists(String databaseName,
String tableName) |
void |
unlock(long lockid)
Unlock a set of locks.
|
boolean |
updatePartitionColumnStatistics(ColumnStatistics statsObj)
Write partition level column statistics to persistent store
|
boolean |
updateTableColumnStatistics(ColumnStatistics statsObj)
Write table level column statistics to persistent store
|
void |
validatePartitionNameCharacters(List<String> partVals) |
protected final HiveConf conf
protected static final org.apache.commons.logging.Log LOG
public HiveMetaStoreClient(HiveConf conf) throws MetaException
MetaException
public HiveMetaStoreClient(HiveConf conf, HiveMetaHookLoader hookLoader) throws MetaException
MetaException
public boolean isCompatibleWith(HiveConf conf)
IMetaStoreClient
isCompatibleWith
in interface IMetaStoreClient
public void setHiveAddedJars(String addedJars)
IMetaStoreClient
setHiveAddedJars
in interface IMetaStoreClient
addedJars
- the hive.added.jars.path. It is qualified paths separated by commas.public void reconnect() throws MetaException
IMetaStoreClient
reconnect
in interface IMetaStoreClient
MetaException
public void alter_table(String dbname, String tbl_name, Table new_tbl) throws InvalidOperationException, MetaException, org.apache.thrift.TException
alter_table
in interface IMetaStoreClient
dbname
- tbl_name
- new_tbl
- InvalidOperationException
MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.alter_table(
java.lang.String, java.lang.String,
org.apache.hadoop.hive.metastore.api.Table)
public void alter_table(String dbname, String tbl_name, Table new_tbl, boolean cascade) throws InvalidOperationException, MetaException, org.apache.thrift.TException
alter_table
in interface IMetaStoreClient
InvalidOperationException
MetaException
org.apache.thrift.TException
public void alter_table(String dbname, String tbl_name, Table new_tbl, EnvironmentContext envContext) throws InvalidOperationException, MetaException, org.apache.thrift.TException
InvalidOperationException
MetaException
org.apache.thrift.TException
public void renamePartition(String dbname, String name, List<String> part_vals, Partition newPart) throws InvalidOperationException, MetaException, org.apache.thrift.TException
IMetaStoreClient
renamePartition
in interface IMetaStoreClient
dbname
- name
- part_vals
- newPart
- InvalidOperationException
MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.rename_partition(
java.lang.String, java.lang.String, java.util.List, org.apache.hadoop.hive.metastore.api.Partition)
public String getTokenStrForm() throws IOException
getTokenStrForm
in interface IMetaStoreClient
IOException
public void close()
IMetaStoreClient
close
in interface IMetaStoreClient
public void setMetaConf(String key, String value) throws org.apache.thrift.TException
IMetaStoreClient
setMetaConf
in interface IMetaStoreClient
org.apache.thrift.TException
public String getMetaConf(String key) throws org.apache.thrift.TException
IMetaStoreClient
getMetaConf
in interface IMetaStoreClient
org.apache.thrift.TException
public Partition add_partition(Partition new_part) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
IMetaStoreClient
add_partition
in interface IMetaStoreClient
new_part
- InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.add_partition(org.apache.hadoop.hive.metastore.api.Partition)
public Partition add_partition(Partition new_part, EnvironmentContext envContext) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
public int add_partitions(List<Partition> new_parts) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
IMetaStoreClient
add_partitions
in interface IMetaStoreClient
new_parts
- InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.add_partitions(List)
public List<Partition> add_partitions(List<Partition> parts, boolean ifNotExists, boolean needResults) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
IMetaStoreClient
add_partitions
in interface IMetaStoreClient
parts
- The partitions to addifNotExists
- only add partitions if they don't existneedResults
- Whether the results are neededInvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
public int add_partitions_pspec(PartitionSpecProxy partitionSpec) throws org.apache.thrift.TException
add_partitions_pspec
in interface IMetaStoreClient
org.apache.thrift.TException
public Partition appendPartition(String db_name, String table_name, List<String> part_vals) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
appendPartition
in interface IMetaStoreClient
table_name
- db_name
- part_vals
- InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.append_partition(java.lang.String,
java.lang.String, java.util.List)
public Partition appendPartition(String db_name, String table_name, List<String> part_vals, EnvironmentContext envContext) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
public Partition appendPartition(String dbName, String tableName, String partName) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
appendPartition
in interface IMetaStoreClient
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
public Partition appendPartition(String dbName, String tableName, String partName, EnvironmentContext envContext) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
public Partition exchange_partition(Map<String,String> partitionSpecs, String sourceDb, String sourceTable, String destDb, String destinationTableName) throws MetaException, NoSuchObjectException, InvalidObjectException, org.apache.thrift.TException
exchange_partition
in interface IMetaStoreClient
partitionSpecs
- partitions specs of the parent partition to be exchangeddestDb
- the db of the destination tabledestinationTableName
- the destination table nameMetaException
NoSuchObjectException
InvalidObjectException
org.apache.thrift.TException
public void validatePartitionNameCharacters(List<String> partVals) throws org.apache.thrift.TException, MetaException
validatePartitionNameCharacters
in interface IMetaStoreClient
org.apache.thrift.TException
MetaException
public void createDatabase(Database db) throws AlreadyExistsException, InvalidObjectException, MetaException, org.apache.thrift.TException
createDatabase
in interface IMetaStoreClient
db
- AlreadyExistsException
InvalidObjectException
MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.create_database(Database)
public void createTable(Table tbl) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException
createTable
in interface IMetaStoreClient
tbl
- MetaException
NoSuchObjectException
org.apache.thrift.TException
AlreadyExistsException
InvalidObjectException
ThriftHiveMetastore.Iface.create_table(org.apache.hadoop.hive.metastore.api.Table)
public void createTable(Table tbl, EnvironmentContext envContext) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException
AlreadyExistsException
InvalidObjectException
MetaException
NoSuchObjectException
org.apache.thrift.TException
public boolean createType(Type type) throws AlreadyExistsException, InvalidObjectException, MetaException, org.apache.thrift.TException
type
- AlreadyExistsException
InvalidObjectException
MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.create_type(org.apache.hadoop.hive.metastore.api.Type)
public void dropDatabase(String name) throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException
dropDatabase
in interface IMetaStoreClient
name
- NoSuchObjectException
InvalidOperationException
MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.drop_database(java.lang.String, boolean, boolean)
public void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb) throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException
dropDatabase
in interface IMetaStoreClient
NoSuchObjectException
InvalidOperationException
MetaException
org.apache.thrift.TException
public void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb, boolean cascade) throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException
dropDatabase
in interface IMetaStoreClient
NoSuchObjectException
InvalidOperationException
MetaException
org.apache.thrift.TException
public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
tbl_name
- db_name
- part_vals
- NoSuchObjectException
MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.drop_partition(java.lang.String,
java.lang.String, java.util.List, boolean)
public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals, EnvironmentContext env_context) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
public boolean dropPartition(String dbName, String tableName, String partName, boolean deleteData) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
dropPartition
in interface IMetaStoreClient
NoSuchObjectException
MetaException
org.apache.thrift.TException
public boolean dropPartition(String dbName, String tableName, String partName, boolean deleteData, EnvironmentContext envContext) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals, boolean deleteData) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
dropPartition
in interface IMetaStoreClient
db_name
- tbl_name
- part_vals
- deleteData
- delete the underlying data or just delete the table in metadataNoSuchObjectException
MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.drop_partition(java.lang.String,
java.lang.String, java.util.List, boolean)
public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals, PartitionDropOptions options) throws org.apache.thrift.TException
IMetaStoreClient
dropPartition
in interface IMetaStoreClient
db_name
- Name of the database.tbl_name
- Name of the table.part_vals
- Specification of the partitions being dropped.options
- PartitionDropOptions for the operation.org.apache.thrift.TException
public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals, boolean deleteData, EnvironmentContext envContext) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
public List<Partition> dropPartitions(String dbName, String tblName, List<ObjectPair<Integer,byte[]>> partExprs, PartitionDropOptions options) throws org.apache.thrift.TException
IMetaStoreClient
dropPartitions
in interface IMetaStoreClient
dbName
- Name of the databasetblName
- Name of the tablepartExprs
- Partition-specificationoptions
- Boolean options for dropping partitionsorg.apache.thrift.TException
- On failurepublic List<Partition> dropPartitions(String dbName, String tblName, List<ObjectPair<Integer,byte[]>> partExprs, boolean deleteData, boolean ignoreProtection, boolean ifExists, boolean needResult) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
dropPartitions
in interface IMetaStoreClient
NoSuchObjectException
MetaException
org.apache.thrift.TException
public List<Partition> dropPartitions(String dbName, String tblName, List<ObjectPair<Integer,byte[]>> partExprs, boolean deleteData, boolean ignoreProtection, boolean ifExists) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
dropPartitions
in interface IMetaStoreClient
NoSuchObjectException
MetaException
org.apache.thrift.TException
public void dropTable(String dbname, String name, boolean deleteData, boolean ignoreUnknownTab) throws MetaException, org.apache.thrift.TException, NoSuchObjectException, UnsupportedOperationException
dropTable
in interface IMetaStoreClient
dbname
- The database for this tablename
- The table to dropdeleteData
- Should we delete the underlying dataignoreUnknownTab
- don't throw if the requested table doesn't existMetaException
- Could not drop table properly.NoSuchObjectException
- The table wasn't found.org.apache.thrift.TException
- A thrift communication error occurredUnsupportedOperationException
dropTable(String, String, boolean, boolean, EnvironmentContext)
public void dropTable(String dbname, String name, boolean deleteData, boolean ignoreUnknownTab, boolean ifPurge) throws MetaException, org.apache.thrift.TException, NoSuchObjectException, UnsupportedOperationException
dropTable
in interface IMetaStoreClient
ifPurge
- completely purge the table (skipping trash) while removing
data from warehouseMetaException
org.apache.thrift.TException
NoSuchObjectException
UnsupportedOperationException
dropTable(String, String, boolean, boolean, EnvironmentContext)
@Deprecated public void dropTable(String tableName, boolean deleteData) throws MetaException, UnknownTableException, org.apache.thrift.TException, NoSuchObjectException
dropTable
in interface IMetaStoreClient
tableName
- The table to dropdeleteData
- Should we delete the underlying dataMetaException
- Could not drop table properly.UnknownTableException
- The table wasn't found.org.apache.thrift.TException
- A thrift communication error occurredNoSuchObjectException
- The table wasn't found.public void dropTable(String dbname, String name) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
dropTable
in interface IMetaStoreClient
NoSuchObjectException
MetaException
org.apache.thrift.TException
dropTable(String, String, boolean, boolean, EnvironmentContext)
public void dropTable(String dbname, String name, boolean deleteData, boolean ignoreUnknownTab, EnvironmentContext envContext) throws MetaException, org.apache.thrift.TException, NoSuchObjectException, UnsupportedOperationException
dbname
- name
- deleteData
- delete the underlying data or just delete the table in metadataignoreUnknownTab
- don't throw if the requested table doesn't existenvContext
- for communicating with thriftMetaException
- could not drop table properlyNoSuchObjectException
- the table wasn't foundorg.apache.thrift.TException
- a thrift communication error occurredUnsupportedOperationException
- dropping an index table is not allowedThriftHiveMetastore.Iface.drop_table(java.lang.String,
java.lang.String, boolean)
public boolean dropType(String type) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
type
- MetaException
org.apache.thrift.TException
NoSuchObjectException
ThriftHiveMetastore.Iface.drop_type(java.lang.String)
public Map<String,Type> getTypeAll(String name) throws MetaException, org.apache.thrift.TException
name
- MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.get_type_all(java.lang.String)
public List<String> getDatabases(String databasePattern) throws MetaException
getDatabases
in interface IMetaStoreClient
MetaException
public List<String> getAllDatabases() throws MetaException
getAllDatabases
in interface IMetaStoreClient
MetaException
public List<Partition> listPartitions(String db_name, String tbl_name, short max_parts) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
listPartitions
in interface IMetaStoreClient
tbl_name
- db_name
- max_parts
- NoSuchObjectException
MetaException
org.apache.thrift.TException
public PartitionSpecProxy listPartitionSpecs(String dbName, String tableName, int maxParts) throws org.apache.thrift.TException
listPartitionSpecs
in interface IMetaStoreClient
org.apache.thrift.TException
public List<Partition> listPartitions(String db_name, String tbl_name, List<String> part_vals, short max_parts) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
listPartitions
in interface IMetaStoreClient
NoSuchObjectException
MetaException
org.apache.thrift.TException
public List<Partition> listPartitionsWithAuthInfo(String db_name, String tbl_name, short max_parts, String user_name, List<String> group_names) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
listPartitionsWithAuthInfo
in interface IMetaStoreClient
NoSuchObjectException
MetaException
org.apache.thrift.TException
public List<Partition> listPartitionsWithAuthInfo(String db_name, String tbl_name, List<String> part_vals, short max_parts, String user_name, List<String> group_names) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
listPartitionsWithAuthInfo
in interface IMetaStoreClient
NoSuchObjectException
MetaException
org.apache.thrift.TException
public List<Partition> listPartitionsByFilter(String db_name, String tbl_name, String filter, short max_parts) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
listPartitionsByFilter
in interface IMetaStoreClient
db_name
- the database nametbl_name
- the table namefilter
- the filter string,
for example "part1 = \"p1_abc\" and part2 <= "\p2_test\"". Filtering can
be done only on string partition keys.max_parts
- the maximum number of partitions to return,
all partitions are returned if -1 is passedMetaException
NoSuchObjectException
org.apache.thrift.TException
public PartitionSpecProxy listPartitionSpecsByFilter(String db_name, String tbl_name, String filter, int max_parts) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
listPartitionSpecsByFilter
in interface IMetaStoreClient
MetaException
NoSuchObjectException
org.apache.thrift.TException
public boolean listPartitionsByExpr(String db_name, String tbl_name, byte[] expr, String default_partition_name, short max_parts, List<Partition> result) throws org.apache.thrift.TException
IMetaStoreClient
listPartitionsByExpr
in interface IMetaStoreClient
db_name
- the database nametbl_name
- the table nameexpr
- expression, serialized from ExprNodeDescdefault_partition_name
- Default partition name from configuration. If blank, the
metastore server-side configuration is used.max_parts
- the maximum number of partitions to return,
all partitions are returned if -1 is passedresult
- the resulting list of partitionsorg.apache.thrift.TException
public Database getDatabase(String name) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
IMetaStoreClient
getDatabase
in interface IMetaStoreClient
name
- NoSuchObjectException
MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.get_database(java.lang.String)
public Partition getPartition(String db_name, String tbl_name, List<String> part_vals) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
getPartition
in interface IMetaStoreClient
tbl_name
- db_name
- part_vals
- MetaException
org.apache.thrift.TException
NoSuchObjectException
ThriftHiveMetastore.Iface.get_partition(java.lang.String,
java.lang.String, java.util.List)
public List<Partition> getPartitionsByNames(String db_name, String tbl_name, List<String> part_names) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
IMetaStoreClient
getPartitionsByNames
in interface IMetaStoreClient
db_name
- database nametbl_name
- table namepart_names
- list of partition namesNoSuchObjectException
MetaException
org.apache.thrift.TException
public Partition getPartitionWithAuthInfo(String db_name, String tbl_name, List<String> part_vals, String user_name, List<String> group_names) throws MetaException, UnknownTableException, NoSuchObjectException, org.apache.thrift.TException
getPartitionWithAuthInfo
in interface IMetaStoreClient
MetaException
UnknownTableException
NoSuchObjectException
org.apache.thrift.TException
public Table getTable(String dbname, String name) throws MetaException, org.apache.thrift.TException, NoSuchObjectException
IMetaStoreClient
getTable
in interface IMetaStoreClient
name
- dbname
- NoSuchObjectException
MetaException
org.apache.thrift.TException
NoSuchObjectException
ThriftHiveMetastore.Iface.get_table(java.lang.String,
java.lang.String)
@Deprecated public Table getTable(String tableName) throws MetaException, org.apache.thrift.TException, NoSuchObjectException
getTable
in interface IMetaStoreClient
tableName
- Name of the table to fetch.MetaException
- Could not fetch the tableorg.apache.thrift.TException
- A thrift communication error occurredNoSuchObjectException
- In case the table wasn't found.public List<Table> getTableObjectsByName(String dbName, List<String> tableNames) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException
getTableObjectsByName
in interface IMetaStoreClient
dbName
- The database the tables are located in.tableNames
- The names of the tables to fetchMetaException
- Any other errorsInvalidOperationException
- The input to this operation is invalid (e.g., the list of tables names is null)UnknownDBException
- The requested database could not be fetched.org.apache.thrift.TException
- A thrift communication error occurredpublic List<String> listTableNamesByFilter(String dbName, String filter, short maxTables) throws MetaException, org.apache.thrift.TException, InvalidOperationException, UnknownDBException
listTableNamesByFilter
in interface IMetaStoreClient
dbName
- The name of the database from which you will retrieve the table namesfilter
- The filter stringmaxTables
- The maximum number of tables returnedMetaException
org.apache.thrift.TException
InvalidOperationException
UnknownDBException
public Type getType(String name) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
name
- MetaException
org.apache.thrift.TException
NoSuchObjectException
ThriftHiveMetastore.Iface.get_type(java.lang.String)
public List<String> getTables(String dbname, String tablePattern) throws MetaException
getTables
in interface IMetaStoreClient
MetaException
public List<String> getAllTables(String dbname) throws MetaException
getAllTables
in interface IMetaStoreClient
MetaException
public boolean tableExists(String databaseName, String tableName) throws MetaException, org.apache.thrift.TException, UnknownDBException
tableExists
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
UnknownDBException
@Deprecated public boolean tableExists(String tableName) throws MetaException, org.apache.thrift.TException, UnknownDBException
tableExists
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
UnknownDBException
public List<String> listPartitionNames(String dbName, String tblName, short max) throws MetaException, org.apache.thrift.TException
listPartitionNames
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
public List<String> listPartitionNames(String db_name, String tbl_name, List<String> part_vals, short max_parts) throws MetaException, org.apache.thrift.TException, NoSuchObjectException
listPartitionNames
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
NoSuchObjectException
public void alter_partition(String dbName, String tblName, Partition newPart) throws InvalidOperationException, MetaException, org.apache.thrift.TException
IMetaStoreClient
alter_partition
in interface IMetaStoreClient
dbName
- database of the old partitiontblName
- table name of the old partitionnewPart
- new partitionInvalidOperationException
- if the old partition does not existMetaException
- if error in updating metadataorg.apache.thrift.TException
- if error in communicating with metastore serverpublic void alter_partitions(String dbName, String tblName, List<Partition> newParts) throws InvalidOperationException, MetaException, org.apache.thrift.TException
IMetaStoreClient
alter_partitions
in interface IMetaStoreClient
dbName
- database of the old partitiontblName
- table name of the old partitionnewParts
- list of partitionsInvalidOperationException
- if the old partition does not existMetaException
- if error in updating metadataorg.apache.thrift.TException
- if error in communicating with metastore serverpublic void alterDatabase(String dbName, Database db) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
alterDatabase
in interface IMetaStoreClient
MetaException
NoSuchObjectException
org.apache.thrift.TException
public List<FieldSchema> getFields(String db, String tableName) throws MetaException, org.apache.thrift.TException, UnknownTableException, UnknownDBException
getFields
in interface IMetaStoreClient
db
- tableName
- UnknownTableException
UnknownDBException
MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.get_fields(java.lang.String,
java.lang.String)
public void createIndex(Index index, Table indexTable) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException
createIndex
in interface IMetaStoreClient
index
- the index objectindexTable
- which stores the index dataInvalidObjectException
MetaException
NoSuchObjectException
org.apache.thrift.TException
AlreadyExistsException
public void alter_index(String dbname, String base_tbl_name, String idx_name, Index new_idx) throws InvalidOperationException, MetaException, org.apache.thrift.TException
alter_index
in interface IMetaStoreClient
dbname
- base_tbl_name
- idx_name
- new_idx
- InvalidOperationException
MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.alter_index(java.lang.String,
java.lang.String, java.lang.String, org.apache.hadoop.hive.metastore.api.Index)
public Index getIndex(String dbName, String tblName, String indexName) throws MetaException, UnknownTableException, NoSuchObjectException, org.apache.thrift.TException
getIndex
in interface IMetaStoreClient
dbName
- tblName
- indexName
- MetaException
UnknownTableException
NoSuchObjectException
org.apache.thrift.TException
public List<String> listIndexNames(String dbName, String tblName, short max) throws MetaException, org.apache.thrift.TException
listIndexNames
in interface IMetaStoreClient
dbName
- tblName
- max
- NoSuchObjectException
MetaException
org.apache.thrift.TException
public List<Index> listIndexes(String dbName, String tblName, short max) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
listIndexes
in interface IMetaStoreClient
dbName
- tblName
- max
- MetaException
org.apache.thrift.TException
NoSuchObjectException
public boolean updateTableColumnStatistics(ColumnStatistics statsObj) throws NoSuchObjectException, InvalidObjectException, MetaException, org.apache.thrift.TException, InvalidInputException
updateTableColumnStatistics
in interface IMetaStoreClient
NoSuchObjectException
InvalidObjectException
MetaException
org.apache.thrift.TException
InvalidInputException
public boolean updatePartitionColumnStatistics(ColumnStatistics statsObj) throws NoSuchObjectException, InvalidObjectException, MetaException, org.apache.thrift.TException, InvalidInputException
updatePartitionColumnStatistics
in interface IMetaStoreClient
NoSuchObjectException
InvalidObjectException
MetaException
org.apache.thrift.TException
InvalidInputException
public boolean setPartitionColumnStatistics(SetPartitionsStatsRequest request) throws NoSuchObjectException, InvalidObjectException, MetaException, org.apache.thrift.TException, InvalidInputException
setPartitionColumnStatistics
in interface IMetaStoreClient
NoSuchObjectException
InvalidObjectException
MetaException
org.apache.thrift.TException
InvalidInputException
public List<ColumnStatisticsObj> getTableColumnStatistics(String dbName, String tableName, List<String> colNames) throws NoSuchObjectException, MetaException, org.apache.thrift.TException, InvalidInputException, InvalidObjectException
getTableColumnStatistics
in interface IMetaStoreClient
NoSuchObjectException
MetaException
org.apache.thrift.TException
InvalidInputException
InvalidObjectException
public Map<String,List<ColumnStatisticsObj>> getPartitionColumnStatistics(String dbName, String tableName, List<String> partNames, List<String> colNames) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
getPartitionColumnStatistics
in interface IMetaStoreClient
NoSuchObjectException
MetaException
org.apache.thrift.TException
public boolean deletePartitionColumnStatistics(String dbName, String tableName, String partName, String colName) throws NoSuchObjectException, InvalidObjectException, MetaException, org.apache.thrift.TException, InvalidInputException
deletePartitionColumnStatistics
in interface IMetaStoreClient
NoSuchObjectException
InvalidObjectException
MetaException
org.apache.thrift.TException
InvalidInputException
public boolean deleteTableColumnStatistics(String dbName, String tableName, String colName) throws NoSuchObjectException, InvalidObjectException, MetaException, org.apache.thrift.TException, InvalidInputException
deleteTableColumnStatistics
in interface IMetaStoreClient
NoSuchObjectException
InvalidObjectException
MetaException
org.apache.thrift.TException
InvalidInputException
public List<FieldSchema> getSchema(String db, String tableName) throws MetaException, org.apache.thrift.TException, UnknownTableException, UnknownDBException
getSchema
in interface IMetaStoreClient
db
- tableName
- UnknownTableException
UnknownDBException
MetaException
org.apache.thrift.TException
ThriftHiveMetastore.Iface.get_schema(java.lang.String,
java.lang.String)
public String getConfigValue(String name, String defaultValue) throws org.apache.thrift.TException, ConfigValSecurityException
getConfigValue
in interface IMetaStoreClient
name
- name of the configuration property to get the value ofdefaultValue
- the value to return if property with the given name doesn't existorg.apache.thrift.TException
ConfigValSecurityException
public Partition getPartition(String db, String tableName, String partName) throws MetaException, org.apache.thrift.TException, UnknownTableException, NoSuchObjectException
getPartition
in interface IMetaStoreClient
partName
- - partition name i.e. 'ds=2010-02-03/ts=2010-02-03 18%3A16%3A01'MetaException
org.apache.thrift.TException
UnknownTableException
NoSuchObjectException
ThriftHiveMetastore.Iface.get_partition(java.lang.String,
java.lang.String, java.util.List)
public Partition appendPartitionByName(String dbName, String tableName, String partName) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
public Partition appendPartitionByName(String dbName, String tableName, String partName, EnvironmentContext envContext) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
public boolean dropPartitionByName(String dbName, String tableName, String partName, boolean deleteData) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
public boolean dropPartitionByName(String dbName, String tableName, String partName, boolean deleteData, EnvironmentContext envContext) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
public List<String> partitionNameToVals(String name) throws MetaException, org.apache.thrift.TException
partitionNameToVals
in interface IMetaStoreClient
name
- the partition name e.g. ("ds=2010-03-03/hr=12")MetaException
org.apache.thrift.TException
public Map<String,String> partitionNameToSpec(String name) throws MetaException, org.apache.thrift.TException
partitionNameToSpec
in interface IMetaStoreClient
name
- the partition name e.g. ("ds=2010-03-03/hr=12")MetaException
org.apache.thrift.TException
protected PrincipalPrivilegeSet deepCopy(PrincipalPrivilegeSet pps)
protected List<FieldSchema> deepCopyFieldSchemas(List<FieldSchema> schemas)
public boolean dropIndex(String dbName, String tblName, String name, boolean deleteData) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
dropIndex
in interface IMetaStoreClient
name
- index nameNoSuchObjectException
MetaException
org.apache.thrift.TException
public boolean grant_role(String roleName, String userName, PrincipalType principalType, String grantor, PrincipalType grantorType, boolean grantOption) throws MetaException, org.apache.thrift.TException
grant_role
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
public boolean create_role(Role role) throws MetaException, org.apache.thrift.TException
create_role
in interface IMetaStoreClient
role
- role objectMetaException
org.apache.thrift.TException
public boolean drop_role(String roleName) throws MetaException, org.apache.thrift.TException
drop_role
in interface IMetaStoreClient
roleName
- role nameMetaException
org.apache.thrift.TException
public List<Role> list_roles(String principalName, PrincipalType principalType) throws MetaException, org.apache.thrift.TException
list_roles
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
public List<String> listRoleNames() throws MetaException, org.apache.thrift.TException
IMetaStoreClient
listRoleNames
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
public GetPrincipalsInRoleResponse get_principals_in_role(GetPrincipalsInRoleRequest req) throws MetaException, org.apache.thrift.TException
IMetaStoreClient
get_principals_in_role
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
public GetRoleGrantsForPrincipalResponse get_role_grants_for_principal(GetRoleGrantsForPrincipalRequest getRolePrincReq) throws MetaException, org.apache.thrift.TException
IMetaStoreClient
get_role_grants_for_principal
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
public boolean grant_privileges(PrivilegeBag privileges) throws MetaException, org.apache.thrift.TException
grant_privileges
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
public boolean revoke_role(String roleName, String userName, PrincipalType principalType, boolean grantOption) throws MetaException, org.apache.thrift.TException
revoke_role
in interface IMetaStoreClient
roleName
- role nameuserName
- user nameMetaException
org.apache.thrift.TException
public boolean revoke_privileges(PrivilegeBag privileges, boolean grantOption) throws MetaException, org.apache.thrift.TException
revoke_privileges
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
public PrincipalPrivilegeSet get_privilege_set(HiveObjectRef hiveObject, String userName, List<String> groupNames) throws MetaException, org.apache.thrift.TException
IMetaStoreClient
get_privilege_set
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
public List<HiveObjectPrivilege> list_privileges(String principalName, PrincipalType principalType, HiveObjectRef hiveObject) throws MetaException, org.apache.thrift.TException
IMetaStoreClient
list_privileges
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
public String getDelegationToken(String renewerKerberosPrincipalName) throws MetaException, org.apache.thrift.TException, IOException
MetaException
org.apache.thrift.TException
IOException
public String getDelegationToken(String owner, String renewerKerberosPrincipalName) throws MetaException, org.apache.thrift.TException
IMetaStoreClient
getDelegationToken
in interface IMetaStoreClient
owner
- the intended owner for the tokenMetaException
org.apache.thrift.TException
public long renewDelegationToken(String tokenStrForm) throws MetaException, org.apache.thrift.TException
renewDelegationToken
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
public void cancelDelegationToken(String tokenStrForm) throws MetaException, org.apache.thrift.TException
cancelDelegationToken
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
public ValidTxnList getValidTxns() throws org.apache.thrift.TException
IMetaStoreClient
getValidTxns
in interface IMetaStoreClient
org.apache.thrift.TException
public ValidTxnList getValidTxns(long currentTxn) throws org.apache.thrift.TException
IMetaStoreClient
getValidTxns
in interface IMetaStoreClient
currentTxn
- The current transaction of the caller. This will be removed from the
exceptions list so that the caller sees records from his own transaction.org.apache.thrift.TException
public long openTxn(String user) throws org.apache.thrift.TException
IMetaStoreClient
openTxn
in interface IMetaStoreClient
user
- User who is opening this transaction. This is the Hive user,
not necessarily the OS user. It is assumed that this user has already been
authenticated and authorized at this point.org.apache.thrift.TException
public OpenTxnsResponse openTxns(String user, int numTxns) throws org.apache.thrift.TException
IMetaStoreClient
Increasing the number of transactions requested in the batch will allow applications that stream data into Hive to place more commits in a single file, thus reducing load on the namenode and making reads of the data more efficient. However, opening more transactions in a batch will also result in readers needing to keep a larger list of open transactions to ignore, potentially slowing their reads. Users will need to test in their system to understand the optimal number of transactions to request in a batch.
openTxns
in interface IMetaStoreClient
user
- User who is opening this transaction. This is the Hive user,
not necessarily the OS user. It is assumed that this user has already been
authenticated and authorized at this point.numTxns
- number of requested transactions to openorg.apache.thrift.TException
public void rollbackTxn(long txnid) throws NoSuchTxnException, org.apache.thrift.TException
IMetaStoreClient
rollbackTxn
in interface IMetaStoreClient
txnid
- id of transaction to be rolled back.NoSuchTxnException
- if the requested transaction does not exist.
Note that this can result from the transaction having timed out and been
deleted.org.apache.thrift.TException
public void commitTxn(long txnid) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException
IMetaStoreClient
commitTxn
in interface IMetaStoreClient
txnid
- id of transaction to be committed.NoSuchTxnException
- if the requested transaction does not exist.
This can result fro the transaction having timed out and been deleted by
the compactor.TxnAbortedException
- if the requested transaction has been
aborted. This can result from the transaction timing out.org.apache.thrift.TException
public GetOpenTxnsInfoResponse showTxns() throws org.apache.thrift.TException
IMetaStoreClient
IMetaStoreClient.getValidTxns()
.showTxns
in interface IMetaStoreClient
org.apache.thrift.TException
public LockResponse lock(LockRequest request) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException
IMetaStoreClient
If the operation requires a transaction (INSERT, UPDATE, or DELETE) that transaction id must be provided as part this lock request. All locks associated with a transaction will be released when that transaction is committed or rolled back.
*Once a lock is acquired, IMetaStoreClient.heartbeat(long, long)
must be called
on a regular basis to avoid the lock being timed out by the system.
lock
in interface IMetaStoreClient
request
- The lock request. LockRequestBuilder
can be used
construct this request.IMetaStoreClient.checkLock(long)
before proceeding. All components of the lock
will have the same state.NoSuchTxnException
- if the requested transaction does not exist.
This can result fro the transaction having timed out and been deleted by
the compactor.TxnAbortedException
- if the requested transaction has been
aborted. This can result from the transaction timing out.org.apache.thrift.TException
public LockResponse checkLock(long lockid) throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, org.apache.thrift.TException
IMetaStoreClient
IMetaStoreClient.lock(org.apache.hadoop.hive.metastore.api.LockRequest)
call.
Once a lock is acquired, IMetaStoreClient.heartbeat(long, long)
must be called
on a regular basis to avoid the lock being timed out by the system.checkLock
in interface IMetaStoreClient
lockid
- lock id returned by lock().NoSuchTxnException
- if the requested transaction does not exist.
This can result fro the transaction having timed out and been deleted by
the compactor.TxnAbortedException
- if the requested transaction has been
aborted. This can result from the transaction timing out.NoSuchLockException
- if the requested lockid does not exist.
This can result from the lock timing out and being unlocked by the system.org.apache.thrift.TException
public void unlock(long lockid) throws NoSuchLockException, TxnOpenException, org.apache.thrift.TException
IMetaStoreClient
unlock
in interface IMetaStoreClient
lockid
- lock id returned by
IMetaStoreClient.lock(org.apache.hadoop.hive.metastore.api.LockRequest)
NoSuchLockException
- if the requested lockid does not exist.
This can result from the lock timing out and being unlocked by the system.TxnOpenException
- if the locks are are associated with a
transaction.org.apache.thrift.TException
public ShowLocksResponse showLocks() throws org.apache.thrift.TException
IMetaStoreClient
showLocks
in interface IMetaStoreClient
org.apache.thrift.TException
public void heartbeat(long txnid, long lockid) throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException
IMetaStoreClient
heartbeat
in interface IMetaStoreClient
txnid
- the id of the open transaction. If no transaction is open
(it is a DDL or query) then this can be set to 0.lockid
- the id of the locks obtained. If no locks have been
obtained then this can be set to 0.NoSuchLockException
- if the requested lockid does not exist.
This can result from the lock timing out and being unlocked by the system.NoSuchTxnException
- if the requested transaction does not exist.
This can result fro the transaction having timed out and been deleted by
the compactor.TxnAbortedException
- if the requested transaction has been
aborted. This can result from the transaction timing out.org.apache.thrift.TException
public HeartbeatTxnRangeResponse heartbeatTxnRange(long min, long max) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException
IMetaStoreClient
IMetaStoreClient.heartbeat(long, long)
.heartbeatTxnRange
in interface IMetaStoreClient
min
- minimum transaction id to heartbeat, inclusivemax
- maximum transaction id to heartbeat, inclusiveorg.apache.thrift.TException
NoSuchTxnException
TxnAbortedException
public void compact(String dbname, String tableName, String partitionName, CompactionType type) throws org.apache.thrift.TException
IMetaStoreClient
compact
in interface IMetaStoreClient
dbname
- Name of the database the table is in. If null, this will be assumed to be
'default'.tableName
- Name of the table to be compacted. This cannot be null. If partitionName
is null, this must be a non-partitioned table.partitionName
- Name of the partition to be compactedtype
- Whether this is a major or minor compaction.org.apache.thrift.TException
public ShowCompactResponse showCompactions() throws org.apache.thrift.TException
IMetaStoreClient
showCompactions
in interface IMetaStoreClient
org.apache.thrift.TException
public void addDynamicPartitions(long txnId, String dbName, String tableName, List<String> partNames) throws org.apache.thrift.TException
IMetaStoreClient
addDynamicPartitions
in interface IMetaStoreClient
txnId
- id of the transactiondbName
- database nametableName
- table namepartNames
- partition name, as constructed by Warehouse.makePartNameorg.apache.thrift.TException
@InterfaceAudience.LimitedPrivate(value="HCatalog") public NotificationEventResponse getNextNotification(long lastEventId, int maxEvents, IMetaStoreClient.NotificationFilter filter) throws org.apache.thrift.TException
IMetaStoreClient
getNextNotification
in interface IMetaStoreClient
lastEventId
- The last event id that was consumed by this reader. The returned
notifications will start at the next eventId available after this eventId.maxEvents
- Maximum number of events to return. If < 1, then all available events will
be returned.filter
- User provided filter to remove unwanted events. If null, all events will be
returned.org.apache.thrift.TException
@InterfaceAudience.LimitedPrivate(value="HCatalog") public CurrentNotificationEventId getCurrentNotificationEventId() throws org.apache.thrift.TException
IMetaStoreClient
getCurrentNotificationEventId
in interface IMetaStoreClient
org.apache.thrift.TException
@InterfaceAudience.LimitedPrivate(value="Apache Hive, HCatalog") public FireEventResponse fireListenerEvent(FireEventRequest rqst) throws org.apache.thrift.TException
IMetaStoreClient
fireListenerEvent
in interface IMetaStoreClient
org.apache.thrift.TException
public static IMetaStoreClient newSynchronizedClient(IMetaStoreClient client)
IMetaStoreClient
.
This may be used by multi-threaded applications until we have
fixed all reentrancy bugs.client
- unsynchronized clientpublic void markPartitionForEvent(String db_name, String tbl_name, Map<String,String> partKVs, PartitionEventType eventType) throws MetaException, org.apache.thrift.TException, NoSuchObjectException, UnknownDBException, UnknownTableException, InvalidPartitionException, UnknownPartitionException
markPartitionForEvent
in interface IMetaStoreClient
MetaException
NoSuchObjectException
org.apache.thrift.TException
UnknownTableException
UnknownDBException
UnknownPartitionException
InvalidPartitionException
public boolean isPartitionMarkedForEvent(String db_name, String tbl_name, Map<String,String> partKVs, PartitionEventType eventType) throws MetaException, NoSuchObjectException, UnknownTableException, UnknownDBException, org.apache.thrift.TException, InvalidPartitionException, UnknownPartitionException
isPartitionMarkedForEvent
in interface IMetaStoreClient
MetaException
NoSuchObjectException
UnknownTableException
UnknownDBException
org.apache.thrift.TException
UnknownPartitionException
InvalidPartitionException
public void createFunction(Function func) throws InvalidObjectException, MetaException, org.apache.thrift.TException
createFunction
in interface IMetaStoreClient
InvalidObjectException
MetaException
org.apache.thrift.TException
public void alterFunction(String dbName, String funcName, Function newFunction) throws InvalidObjectException, MetaException, org.apache.thrift.TException
alterFunction
in interface IMetaStoreClient
InvalidObjectException
MetaException
org.apache.thrift.TException
public void dropFunction(String dbName, String funcName) throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException, org.apache.thrift.TException
dropFunction
in interface IMetaStoreClient
MetaException
NoSuchObjectException
InvalidObjectException
InvalidInputException
org.apache.thrift.TException
public Function getFunction(String dbName, String funcName) throws MetaException, org.apache.thrift.TException
getFunction
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
public List<String> getFunctions(String dbName, String pattern) throws MetaException, org.apache.thrift.TException
getFunctions
in interface IMetaStoreClient
MetaException
org.apache.thrift.TException
protected void create_table_with_environment_context(Table tbl, EnvironmentContext envContext) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException
AlreadyExistsException
InvalidObjectException
MetaException
NoSuchObjectException
org.apache.thrift.TException
protected void drop_table_with_environment_context(String dbname, String name, boolean deleteData, EnvironmentContext envContext) throws MetaException, org.apache.thrift.TException, NoSuchObjectException, UnsupportedOperationException
MetaException
org.apache.thrift.TException
NoSuchObjectException
UnsupportedOperationException
public AggrStats getAggrColStatsFor(String dbName, String tblName, List<String> colNames, List<String> partNames) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
getAggrColStatsFor
in interface IMetaStoreClient
NoSuchObjectException
MetaException
org.apache.thrift.TException
Copyright © 2017 The Apache Software Foundation. All rights reserved.