public interface RawStore
extends org.apache.hadoop.conf.Configurable
Modifier and Type | Interface and Description |
---|---|
static interface |
RawStore.CanNotRetry
Annotation to skip retries
|
Modifier and Type | Method and Description |
---|---|
boolean |
addIndex(Index index) |
int |
addMasterKey(String key) |
void |
addNotificationEvent(NotificationEvent event)
Add a notification entry.
|
boolean |
addPartition(Partition part) |
boolean |
addPartitions(String dbName,
String tblName,
List<Partition> parts) |
boolean |
addPartitions(String dbName,
String tblName,
PartitionSpecProxy partitionSpec,
boolean ifNotExists) |
boolean |
addRole(String rowName,
String ownerName) |
boolean |
addToken(String tokenIdentifier,
String delegationToken) |
boolean |
alterDatabase(String dbname,
Database db) |
void |
alterFunction(String dbName,
String funcName,
Function newFunction)
Alter function based on new function specs.
|
void |
alterIndex(String dbname,
String baseTblName,
String name,
Index newIndex) |
void |
alterPartition(String db_name,
String tbl_name,
List<String> part_vals,
Partition new_part) |
void |
alterPartitions(String db_name,
String tbl_name,
List<List<String>> part_vals_list,
List<Partition> new_parts) |
void |
alterTable(String dbname,
String name,
Table newTable) |
void |
cleanNotificationEvents(int olderThan)
Remove older notification events.
|
long |
cleanupEvents() |
boolean |
commitTransaction()
if this is the commit of the first open call then an actual commit is
called.
|
void |
createDatabase(Database db) |
void |
createFunction(Function func)
Register a user-defined function based on the function specification passed in.
|
void |
createTable(Table tbl) |
boolean |
createType(Type type) |
boolean |
deletePartitionColumnStatistics(String dbName,
String tableName,
String partName,
List<String> partVals,
String colName)
Deletes column statistics if present associated with a given db, table, partition and col.
|
boolean |
deleteTableColumnStatistics(String dbName,
String tableName,
String colName)
Deletes column statistics if present associated with a given db, table and col.
|
boolean |
doesPartitionExist(String dbName,
String tableName,
List<String> part_vals) |
boolean |
dropDatabase(String dbname) |
void |
dropFunction(String dbName,
String funcName)
Drop a function definition.
|
boolean |
dropIndex(String dbName,
String origTableName,
String indexName) |
boolean |
dropPartition(String dbName,
String tableName,
List<String> part_vals) |
void |
dropPartitions(String dbName,
String tblName,
List<String> partNames) |
boolean |
dropTable(String dbName,
String tableName) |
boolean |
dropType(String typeName) |
AggrStats |
get_aggr_stats_for(String dbName,
String tblName,
List<String> partNames,
List<String> colNames) |
List<String> |
getAllDatabases() |
List<String> |
getAllTables(String dbName) |
List<String> |
getAllTokenIdentifiers() |
PrincipalPrivilegeSet |
getColumnPrivilegeSet(String dbName,
String tableName,
String partitionName,
String columnName,
String userName,
List<String> groupNames) |
CurrentNotificationEventId |
getCurrentNotificationEventId()
Get the last issued notification event id.
|
Database |
getDatabase(String name) |
List<String> |
getDatabases(String pattern) |
PrincipalPrivilegeSet |
getDBPrivilegeSet(String dbName,
String userName,
List<String> groupNames) |
Function |
getFunction(String dbName,
String funcName)
Retrieve function by name.
|
List<String> |
getFunctions(String dbName,
String pattern)
Retrieve list of function names based on name pattern.
|
Index |
getIndex(String dbName,
String origTableName,
String indexName) |
List<Index> |
getIndexes(String dbName,
String origTableName,
int max) |
String[] |
getMasterKeys() |
String |
getMetaStoreSchemaVersion() |
NotificationEventResponse |
getNextNotification(NotificationEventRequest rqst)
Get the next notification event.
|
Partition |
getPartition(String dbName,
String tableName,
List<String> part_vals) |
List<ColumnStatistics> |
getPartitionColumnStatistics(String dbName,
String tblName,
List<String> partNames,
List<String> colNames)
Returns the relevant column statistics for given columns in given partitions in a given
table in a given database if such statistics exist.
|
PrincipalPrivilegeSet |
getPartitionPrivilegeSet(String dbName,
String tableName,
String partition,
String userName,
List<String> groupNames) |
List<Partition> |
getPartitions(String dbName,
String tableName,
int max) |
boolean |
getPartitionsByExpr(String dbName,
String tblName,
byte[] expr,
String defaultPartitionName,
short maxParts,
List<Partition> result) |
List<Partition> |
getPartitionsByFilter(String dbName,
String tblName,
String filter,
short maxParts) |
List<Partition> |
getPartitionsByNames(String dbName,
String tblName,
List<String> partNames) |
List<Partition> |
getPartitionsWithAuth(String dbName,
String tblName,
short maxParts,
String userName,
List<String> groupNames) |
Partition |
getPartitionWithAuth(String dbName,
String tblName,
List<String> partVals,
String user_name,
List<String> group_names) |
Role |
getRole(String roleName) |
Table |
getTable(String dbName,
String tableName) |
ColumnStatistics |
getTableColumnStatistics(String dbName,
String tableName,
List<String> colName)
Returns the relevant column statistics for a given column in a given table in a given database
if such statistics exist.
|
List<Table> |
getTableObjectsByName(String dbname,
List<String> tableNames) |
PrincipalPrivilegeSet |
getTablePrivilegeSet(String dbName,
String tableName,
String userName,
List<String> groupNames) |
List<String> |
getTables(String dbName,
String pattern) |
String |
getToken(String tokenIdentifier) |
Type |
getType(String typeName) |
PrincipalPrivilegeSet |
getUserPrivilegeSet(String userName,
List<String> groupNames) |
boolean |
grantPrivileges(PrivilegeBag privileges) |
boolean |
grantRole(Role role,
String userName,
PrincipalType principalType,
String grantor,
PrincipalType grantorType,
boolean grantOption) |
boolean |
isPartitionMarkedForEvent(String dbName,
String tblName,
Map<String,String> partName,
PartitionEventType evtType) |
List<MTablePrivilege> |
listAllTableGrants(String principalName,
PrincipalType principalType,
String dbName,
String tableName) |
List<HiveObjectPrivilege> |
listDBGrantsAll(String dbName) |
List<HiveObjectPrivilege> |
listGlobalGrantsAll() |
List<String> |
listIndexNames(String dbName,
String origTableName,
short max) |
List<HiveObjectPrivilege> |
listPartitionColumnGrantsAll(String dbName,
String tableName,
String partitionName,
String columnName) |
List<HiveObjectPrivilege> |
listPartitionGrantsAll(String dbName,
String tableName,
String partitionName) |
List<String> |
listPartitionNames(String db_name,
String tbl_name,
short max_parts) |
List<String> |
listPartitionNamesByFilter(String db_name,
String tbl_name,
String filter,
short max_parts) |
List<String> |
listPartitionNamesPs(String db_name,
String tbl_name,
List<String> part_vals,
short max_parts)
Lists partition names that match a given partial specification
|
List<Partition> |
listPartitionsPsWithAuth(String db_name,
String tbl_name,
List<String> part_vals,
short max_parts,
String userName,
List<String> groupNames)
Lists partitions that match a given partial specification and sets their auth privileges.
|
List<MDBPrivilege> |
listPrincipalDBGrants(String principalName,
PrincipalType principalType,
String dbName) |
List<HiveObjectPrivilege> |
listPrincipalDBGrantsAll(String principalName,
PrincipalType principalType) |
List<MGlobalPrivilege> |
listPrincipalGlobalGrants(String principalName,
PrincipalType principalType) |
List<MPartitionColumnPrivilege> |
listPrincipalPartitionColumnGrants(String principalName,
PrincipalType principalType,
String dbName,
String tableName,
String partName,
String columnName) |
List<HiveObjectPrivilege> |
listPrincipalPartitionColumnGrantsAll(String principalName,
PrincipalType principalType) |
List<MPartitionPrivilege> |
listPrincipalPartitionGrants(String principalName,
PrincipalType principalType,
String dbName,
String tableName,
String partName) |
List<HiveObjectPrivilege> |
listPrincipalPartitionGrantsAll(String principalName,
PrincipalType principalType) |
List<MTableColumnPrivilege> |
listPrincipalTableColumnGrants(String principalName,
PrincipalType principalType,
String dbName,
String tableName,
String columnName) |
List<HiveObjectPrivilege> |
listPrincipalTableColumnGrantsAll(String principalName,
PrincipalType principalType) |
List<HiveObjectPrivilege> |
listPrincipalTableGrantsAll(String principalName,
PrincipalType principalType) |
List<MRoleMap> |
listRoleMembers(String roleName)
Get the role to principal grant mapping for given role
|
List<String> |
listRoleNames() |
List<MRoleMap> |
listRoles(String principalName,
PrincipalType principalType) |
List<HiveObjectPrivilege> |
listTableColumnGrantsAll(String dbName,
String tableName,
String columnName) |
List<HiveObjectPrivilege> |
listTableGrantsAll(String dbName,
String tableName) |
List<String> |
listTableNamesByFilter(String dbName,
String filter,
short max_tables)
Gets a list of tables based on a filter string and filter type.
|
Table |
markPartitionForEvent(String dbName,
String tblName,
Map<String,String> partVals,
PartitionEventType evtType) |
boolean |
openTransaction()
Opens a new one or the one already created Every call of this function must
have corresponding commit or rollback function call
|
boolean |
removeMasterKey(Integer keySeq) |
boolean |
removeRole(String roleName) |
boolean |
removeToken(String tokenIdentifier) |
boolean |
revokePrivileges(PrivilegeBag privileges,
boolean grantOption) |
boolean |
revokeRole(Role role,
String userName,
PrincipalType principalType,
boolean grantOption) |
void |
rollbackTransaction()
Rolls back the current transaction if it is active
|
void |
setMetaStoreSchemaVersion(String version,
String comment) |
void |
shutdown() |
void |
updateMasterKey(Integer seqNo,
String key) |
boolean |
updatePartitionColumnStatistics(ColumnStatistics statsObj,
List<String> partVals)
Persists the given column statistics object to the metastore
|
boolean |
updateTableColumnStatistics(ColumnStatistics colStats)
Persists the given column statistics object to the metastore
|
void |
verifySchema() |
void shutdown()
boolean openTransaction()
boolean commitTransaction()
void rollbackTransaction()
void createDatabase(Database db) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
Database getDatabase(String name) throws NoSuchObjectException
NoSuchObjectException
boolean dropDatabase(String dbname) throws NoSuchObjectException, MetaException
NoSuchObjectException
MetaException
boolean alterDatabase(String dbname, Database db) throws NoSuchObjectException, MetaException
NoSuchObjectException
MetaException
List<String> getDatabases(String pattern) throws MetaException
MetaException
List<String> getAllDatabases() throws MetaException
MetaException
boolean createType(Type type)
boolean dropType(String typeName)
void createTable(Table tbl) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
boolean dropTable(String dbName, String tableName) throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException
Table getTable(String dbName, String tableName) throws MetaException
MetaException
boolean addPartition(Partition part) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
boolean addPartitions(String dbName, String tblName, List<Partition> parts) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
boolean addPartitions(String dbName, String tblName, PartitionSpecProxy partitionSpec, boolean ifNotExists) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
Partition getPartition(String dbName, String tableName, List<String> part_vals) throws MetaException, NoSuchObjectException
MetaException
NoSuchObjectException
boolean doesPartitionExist(String dbName, String tableName, List<String> part_vals) throws MetaException, NoSuchObjectException
MetaException
NoSuchObjectException
boolean dropPartition(String dbName, String tableName, List<String> part_vals) throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException
List<Partition> getPartitions(String dbName, String tableName, int max) throws MetaException, NoSuchObjectException
MetaException
NoSuchObjectException
void alterTable(String dbname, String name, Table newTable) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
List<String> getTables(String dbName, String pattern) throws MetaException
MetaException
List<Table> getTableObjectsByName(String dbname, List<String> tableNames) throws MetaException, UnknownDBException
dbname
- The name of the database from which to retrieve the tablestableNames
- The names of the tables to retrieve.MetaException
UnknownDBException
List<String> getAllTables(String dbName) throws MetaException
MetaException
List<String> listTableNamesByFilter(String dbName, String filter, short max_tables) throws MetaException, UnknownDBException
dbName
- The name of the database from which you will retrieve the table namesfilter
- The filter stringmax_tables
- The maximum number of tables returnedMetaException
UnknownDBException
List<String> listPartitionNames(String db_name, String tbl_name, short max_parts) throws MetaException
MetaException
List<String> listPartitionNamesByFilter(String db_name, String tbl_name, String filter, short max_parts) throws MetaException
MetaException
void alterPartition(String db_name, String tbl_name, List<String> part_vals, Partition new_part) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
void alterPartitions(String db_name, String tbl_name, List<List<String>> part_vals_list, List<Partition> new_parts) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
boolean addIndex(Index index) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
Index getIndex(String dbName, String origTableName, String indexName) throws MetaException
MetaException
boolean dropIndex(String dbName, String origTableName, String indexName) throws MetaException
MetaException
List<Index> getIndexes(String dbName, String origTableName, int max) throws MetaException
MetaException
List<String> listIndexNames(String dbName, String origTableName, short max) throws MetaException
MetaException
void alterIndex(String dbname, String baseTblName, String name, Index newIndex) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
List<Partition> getPartitionsByFilter(String dbName, String tblName, String filter, short maxParts) throws MetaException, NoSuchObjectException
MetaException
NoSuchObjectException
boolean getPartitionsByExpr(String dbName, String tblName, byte[] expr, String defaultPartitionName, short maxParts, List<Partition> result) throws org.apache.thrift.TException
org.apache.thrift.TException
List<Partition> getPartitionsByNames(String dbName, String tblName, List<String> partNames) throws MetaException, NoSuchObjectException
MetaException
NoSuchObjectException
Table markPartitionForEvent(String dbName, String tblName, Map<String,String> partVals, PartitionEventType evtType) throws MetaException, UnknownTableException, InvalidPartitionException, UnknownPartitionException
boolean isPartitionMarkedForEvent(String dbName, String tblName, Map<String,String> partName, PartitionEventType evtType) throws MetaException, UnknownTableException, InvalidPartitionException, UnknownPartitionException
boolean addRole(String rowName, String ownerName) throws InvalidObjectException, MetaException, NoSuchObjectException
boolean removeRole(String roleName) throws MetaException, NoSuchObjectException
MetaException
NoSuchObjectException
boolean grantRole(Role role, String userName, PrincipalType principalType, String grantor, PrincipalType grantorType, boolean grantOption) throws MetaException, NoSuchObjectException, InvalidObjectException
boolean revokeRole(Role role, String userName, PrincipalType principalType, boolean grantOption) throws MetaException, NoSuchObjectException
MetaException
NoSuchObjectException
PrincipalPrivilegeSet getUserPrivilegeSet(String userName, List<String> groupNames) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
PrincipalPrivilegeSet getDBPrivilegeSet(String dbName, String userName, List<String> groupNames) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
PrincipalPrivilegeSet getTablePrivilegeSet(String dbName, String tableName, String userName, List<String> groupNames) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
PrincipalPrivilegeSet getPartitionPrivilegeSet(String dbName, String tableName, String partition, String userName, List<String> groupNames) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
PrincipalPrivilegeSet getColumnPrivilegeSet(String dbName, String tableName, String partitionName, String columnName, String userName, List<String> groupNames) throws InvalidObjectException, MetaException
InvalidObjectException
MetaException
List<MGlobalPrivilege> listPrincipalGlobalGrants(String principalName, PrincipalType principalType)
List<MDBPrivilege> listPrincipalDBGrants(String principalName, PrincipalType principalType, String dbName)
List<MTablePrivilege> listAllTableGrants(String principalName, PrincipalType principalType, String dbName, String tableName)
List<MPartitionPrivilege> listPrincipalPartitionGrants(String principalName, PrincipalType principalType, String dbName, String tableName, String partName)
List<MTableColumnPrivilege> listPrincipalTableColumnGrants(String principalName, PrincipalType principalType, String dbName, String tableName, String columnName)
List<MPartitionColumnPrivilege> listPrincipalPartitionColumnGrants(String principalName, PrincipalType principalType, String dbName, String tableName, String partName, String columnName)
boolean grantPrivileges(PrivilegeBag privileges) throws InvalidObjectException, MetaException, NoSuchObjectException
boolean revokePrivileges(PrivilegeBag privileges, boolean grantOption) throws InvalidObjectException, MetaException, NoSuchObjectException
Role getRole(String roleName) throws NoSuchObjectException
NoSuchObjectException
List<MRoleMap> listRoles(String principalName, PrincipalType principalType)
List<MRoleMap> listRoleMembers(String roleName)
roleName
- Partition getPartitionWithAuth(String dbName, String tblName, List<String> partVals, String user_name, List<String> group_names) throws MetaException, NoSuchObjectException, InvalidObjectException
List<Partition> getPartitionsWithAuth(String dbName, String tblName, short maxParts, String userName, List<String> groupNames) throws MetaException, NoSuchObjectException, InvalidObjectException
List<String> listPartitionNamesPs(String db_name, String tbl_name, List<String> part_vals, short max_parts) throws MetaException, NoSuchObjectException
db_name
- The name of the database which has the partitionstbl_name
- The name of the table which has the partitionspart_vals
- A partial list of values for partitions in order of the table's partition keys.
Entries can be empty if you only want to specify latter partitions.max_parts
- The maximum number of partitions to returnMetaException
NoSuchObjectException
List<Partition> listPartitionsPsWithAuth(String db_name, String tbl_name, List<String> part_vals, short max_parts, String userName, List<String> groupNames) throws MetaException, InvalidObjectException, NoSuchObjectException
db_name
- The name of the database which has the partitionstbl_name
- The name of the table which has the partitionspart_vals
- A partial list of values for partitions in order of the table's partition keys
Entries can be empty if you need to specify latter partitions.max_parts
- The maximum number of partitions to returnuserName
- The user name for the partition for authentication privilegesgroupNames
- The groupNames for the partition for authentication privilegesMetaException
NoSuchObjectException
InvalidObjectException
boolean updateTableColumnStatistics(ColumnStatistics colStats) throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException
partVals
- ColumnStats
- object to persistList
- of partValsNoSuchObjectException
MetaException
InvalidObjectException
InvalidInputException
boolean updatePartitionColumnStatistics(ColumnStatistics statsObj, List<String> partVals) throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException
partVals
- ColumnStats
- object to persistList
- of partValsNoSuchObjectException
MetaException
InvalidObjectException
InvalidInputException
ColumnStatistics getTableColumnStatistics(String dbName, String tableName, List<String> colName) throws MetaException, NoSuchObjectException
The
- name of the database, defaults to current databaseThe
- name of the tableThe
- name of the column for which statistics is requestedNoSuchObjectException
MetaException
InvalidInputException
List<ColumnStatistics> getPartitionColumnStatistics(String dbName, String tblName, List<String> partNames, List<String> colNames) throws MetaException, NoSuchObjectException
MetaException
NoSuchObjectException
boolean deletePartitionColumnStatistics(String dbName, String tableName, String partName, List<String> partVals, String colName) throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException
dbName
- tableName
- partName
- partVals
- colName
- NoSuchObjectException
MetaException
InvalidObjectException
InvalidInputException
boolean deleteTableColumnStatistics(String dbName, String tableName, String colName) throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException
dbName
- tableName
- colName
- NoSuchObjectException
MetaException
InvalidObjectException
InvalidInputException
long cleanupEvents()
boolean removeToken(String tokenIdentifier)
int addMasterKey(String key) throws MetaException
MetaException
void updateMasterKey(Integer seqNo, String key) throws NoSuchObjectException, MetaException
NoSuchObjectException
MetaException
boolean removeMasterKey(Integer keySeq)
String[] getMasterKeys()
void verifySchema() throws MetaException
MetaException
String getMetaStoreSchemaVersion() throws MetaException
MetaException
void setMetaStoreSchemaVersion(String version, String comment) throws MetaException
MetaException
void dropPartitions(String dbName, String tblName, List<String> partNames) throws MetaException, NoSuchObjectException
MetaException
NoSuchObjectException
List<HiveObjectPrivilege> listPrincipalDBGrantsAll(String principalName, PrincipalType principalType)
List<HiveObjectPrivilege> listPrincipalTableGrantsAll(String principalName, PrincipalType principalType)
List<HiveObjectPrivilege> listPrincipalPartitionGrantsAll(String principalName, PrincipalType principalType)
List<HiveObjectPrivilege> listPrincipalTableColumnGrantsAll(String principalName, PrincipalType principalType)
List<HiveObjectPrivilege> listPrincipalPartitionColumnGrantsAll(String principalName, PrincipalType principalType)
List<HiveObjectPrivilege> listGlobalGrantsAll()
List<HiveObjectPrivilege> listDBGrantsAll(String dbName)
List<HiveObjectPrivilege> listPartitionColumnGrantsAll(String dbName, String tableName, String partitionName, String columnName)
List<HiveObjectPrivilege> listTableGrantsAll(String dbName, String tableName)
List<HiveObjectPrivilege> listPartitionGrantsAll(String dbName, String tableName, String partitionName)
List<HiveObjectPrivilege> listTableColumnGrantsAll(String dbName, String tableName, String columnName)
void createFunction(Function func) throws InvalidObjectException, MetaException
func
- InvalidObjectException
MetaException
void alterFunction(String dbName, String funcName, Function newFunction) throws InvalidObjectException, MetaException
dbName
- name
- newFunction
- InvalidObjectException
MetaException
void dropFunction(String dbName, String funcName) throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException
dbName
- functionName
- MetaException
NoSuchObjectException
InvalidObjectException
InvalidInputException
Function getFunction(String dbName, String funcName) throws MetaException
dbName
- functionName
- MetaException
List<String> getFunctions(String dbName, String pattern) throws MetaException
dbName
- pattern
- MetaException
AggrStats get_aggr_stats_for(String dbName, String tblName, List<String> partNames, List<String> colNames) throws MetaException, NoSuchObjectException
MetaException
NoSuchObjectException
NotificationEventResponse getNextNotification(NotificationEventRequest rqst)
rqst
- Request containing information on the last processed notification.void addNotificationEvent(NotificationEvent event)
event
- the notification to addvoid cleanNotificationEvents(int olderThan)
olderThan
- Remove any events older than a given number of secondsCurrentNotificationEventId getCurrentNotificationEventId()
Copyright © 2017 The Apache Software Foundation. All rights reserved.