public static interface ThriftHiveMetastore.Iface
extends com.facebook.fb303.FacebookService.Iface
String getMetaConf(String key) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
void setMetaConf(String key, String value) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
void create_database(Database database) throws AlreadyExistsException, InvalidObjectException, MetaException, org.apache.thrift.TException
AlreadyExistsException
InvalidObjectException
MetaException
org.apache.thrift.TException
Database get_database(String name) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
void drop_database(String name, boolean deleteData, boolean cascade) throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException
NoSuchObjectException
InvalidOperationException
MetaException
org.apache.thrift.TException
List<String> get_databases(String pattern) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
List<String> get_all_databases() throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
void alter_database(String dbname, Database db) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
Type get_type(String name) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
boolean create_type(Type type) throws AlreadyExistsException, InvalidObjectException, MetaException, org.apache.thrift.TException
AlreadyExistsException
InvalidObjectException
MetaException
org.apache.thrift.TException
boolean drop_type(String type) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
Map<String,Type> get_type_all(String name) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
List<FieldSchema> get_fields(String db_name, String table_name) throws MetaException, UnknownTableException, UnknownDBException, org.apache.thrift.TException
MetaException
UnknownTableException
UnknownDBException
org.apache.thrift.TException
List<FieldSchema> get_fields_with_environment_context(String db_name, String table_name, EnvironmentContext environment_context) throws MetaException, UnknownTableException, UnknownDBException, org.apache.thrift.TException
MetaException
UnknownTableException
UnknownDBException
org.apache.thrift.TException
List<FieldSchema> get_schema(String db_name, String table_name) throws MetaException, UnknownTableException, UnknownDBException, org.apache.thrift.TException
MetaException
UnknownTableException
UnknownDBException
org.apache.thrift.TException
List<FieldSchema> get_schema_with_environment_context(String db_name, String table_name, EnvironmentContext environment_context) throws MetaException, UnknownTableException, UnknownDBException, org.apache.thrift.TException
MetaException
UnknownTableException
UnknownDBException
org.apache.thrift.TException
void create_table(Table tbl) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException
AlreadyExistsException
InvalidObjectException
MetaException
NoSuchObjectException
org.apache.thrift.TException
void create_table_with_environment_context(Table tbl, EnvironmentContext environment_context) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException
AlreadyExistsException
InvalidObjectException
MetaException
NoSuchObjectException
org.apache.thrift.TException
void drop_table(String dbname, String name, boolean deleteData) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
void drop_table_with_environment_context(String dbname, String name, boolean deleteData, EnvironmentContext environment_context) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
List<String> get_tables(String db_name, String pattern) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
List<String> get_all_tables(String db_name) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
Table get_table(String dbname, String tbl_name) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
List<Table> get_table_objects_by_name(String dbname, List<String> tbl_names) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException
MetaException
InvalidOperationException
UnknownDBException
org.apache.thrift.TException
List<String> get_table_names_by_filter(String dbname, String filter, short max_tables) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException
MetaException
InvalidOperationException
UnknownDBException
org.apache.thrift.TException
void alter_table(String dbname, String tbl_name, Table new_tbl) throws InvalidOperationException, MetaException, org.apache.thrift.TException
InvalidOperationException
MetaException
org.apache.thrift.TException
void alter_table_with_environment_context(String dbname, String tbl_name, Table new_tbl, EnvironmentContext environment_context) throws InvalidOperationException, MetaException, org.apache.thrift.TException
InvalidOperationException
MetaException
org.apache.thrift.TException
void alter_table_with_cascade(String dbname, String tbl_name, Table new_tbl, boolean cascade) throws InvalidOperationException, MetaException, org.apache.thrift.TException
InvalidOperationException
MetaException
org.apache.thrift.TException
Partition add_partition(Partition new_part) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
Partition add_partition_with_environment_context(Partition new_part, EnvironmentContext environment_context) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
int add_partitions(List<Partition> new_parts) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
int add_partitions_pspec(List<PartitionSpec> new_parts) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
Partition append_partition(String db_name, String tbl_name, List<String> part_vals) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
AddPartitionsResult add_partitions_req(AddPartitionsRequest request) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
Partition append_partition_with_environment_context(String db_name, String tbl_name, List<String> part_vals, EnvironmentContext environment_context) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
Partition append_partition_by_name(String db_name, String tbl_name, String part_name) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
Partition append_partition_by_name_with_environment_context(String db_name, String tbl_name, String part_name, EnvironmentContext environment_context) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
boolean drop_partition(String db_name, String tbl_name, List<String> part_vals, boolean deleteData) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
boolean drop_partition_with_environment_context(String db_name, String tbl_name, List<String> part_vals, boolean deleteData, EnvironmentContext environment_context) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
boolean drop_partition_by_name(String db_name, String tbl_name, String part_name, boolean deleteData) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
boolean drop_partition_by_name_with_environment_context(String db_name, String tbl_name, String part_name, boolean deleteData, EnvironmentContext environment_context) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
DropPartitionsResult drop_partitions_req(DropPartitionsRequest req) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
Partition get_partition(String db_name, String tbl_name, List<String> part_vals) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
Partition exchange_partition(Map<String,String> partitionSpecs, String source_db, String source_table_name, String dest_db, String dest_table_name) throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException, org.apache.thrift.TException
MetaException
NoSuchObjectException
InvalidObjectException
InvalidInputException
org.apache.thrift.TException
Partition get_partition_with_auth(String db_name, String tbl_name, List<String> part_vals, String user_name, List<String> group_names) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
Partition get_partition_by_name(String db_name, String tbl_name, String part_name) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
List<Partition> get_partitions(String db_name, String tbl_name, short max_parts) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
List<Partition> get_partitions_with_auth(String db_name, String tbl_name, short max_parts, String user_name, List<String> group_names) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
List<PartitionSpec> get_partitions_pspec(String db_name, String tbl_name, int max_parts) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
List<String> get_partition_names(String db_name, String tbl_name, short max_parts) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
List<Partition> get_partitions_ps(String db_name, String tbl_name, List<String> part_vals, short max_parts) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
List<Partition> get_partitions_ps_with_auth(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
NoSuchObjectException
MetaException
org.apache.thrift.TException
List<String> get_partition_names_ps(String db_name, String tbl_name, List<String> part_vals, short max_parts) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
List<Partition> get_partitions_by_filter(String db_name, String tbl_name, String filter, short max_parts) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
List<PartitionSpec> get_part_specs_by_filter(String db_name, String tbl_name, String filter, int max_parts) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
PartitionsByExprResult get_partitions_by_expr(PartitionsByExprRequest req) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
List<Partition> get_partitions_by_names(String db_name, String tbl_name, List<String> names) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
void alter_partition(String db_name, String tbl_name, Partition new_part) throws InvalidOperationException, MetaException, org.apache.thrift.TException
InvalidOperationException
MetaException
org.apache.thrift.TException
void alter_partitions(String db_name, String tbl_name, List<Partition> new_parts) throws InvalidOperationException, MetaException, org.apache.thrift.TException
InvalidOperationException
MetaException
org.apache.thrift.TException
void alter_partition_with_environment_context(String db_name, String tbl_name, Partition new_part, EnvironmentContext environment_context) throws InvalidOperationException, MetaException, org.apache.thrift.TException
InvalidOperationException
MetaException
org.apache.thrift.TException
void rename_partition(String db_name, String tbl_name, List<String> part_vals, Partition new_part) throws InvalidOperationException, MetaException, org.apache.thrift.TException
InvalidOperationException
MetaException
org.apache.thrift.TException
boolean partition_name_has_valid_characters(List<String> part_vals, boolean throw_exception) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
String get_config_value(String name, String defaultValue) throws ConfigValSecurityException, org.apache.thrift.TException
ConfigValSecurityException
org.apache.thrift.TException
List<String> partition_name_to_vals(String part_name) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
Map<String,String> partition_name_to_spec(String part_name) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
void markPartitionForEvent(String db_name, String tbl_name, Map<String,String> part_vals, PartitionEventType eventType) throws MetaException, NoSuchObjectException, UnknownDBException, UnknownTableException, UnknownPartitionException, InvalidPartitionException, org.apache.thrift.TException
MetaException
NoSuchObjectException
UnknownDBException
UnknownTableException
UnknownPartitionException
InvalidPartitionException
org.apache.thrift.TException
boolean isPartitionMarkedForEvent(String db_name, String tbl_name, Map<String,String> part_vals, PartitionEventType eventType) throws MetaException, NoSuchObjectException, UnknownDBException, UnknownTableException, UnknownPartitionException, InvalidPartitionException, org.apache.thrift.TException
MetaException
NoSuchObjectException
UnknownDBException
UnknownTableException
UnknownPartitionException
InvalidPartitionException
org.apache.thrift.TException
Index add_index(Index new_index, Table index_table) throws InvalidObjectException, AlreadyExistsException, MetaException, org.apache.thrift.TException
InvalidObjectException
AlreadyExistsException
MetaException
org.apache.thrift.TException
void alter_index(String dbname, String base_tbl_name, String idx_name, Index new_idx) throws InvalidOperationException, MetaException, org.apache.thrift.TException
InvalidOperationException
MetaException
org.apache.thrift.TException
boolean drop_index_by_name(String db_name, String tbl_name, String index_name, boolean deleteData) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
Index get_index_by_name(String db_name, String tbl_name, String index_name) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
List<Index> get_indexes(String db_name, String tbl_name, short max_indexes) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
List<String> get_index_names(String db_name, String tbl_name, short max_indexes) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
boolean update_table_column_statistics(ColumnStatistics stats_obj) throws NoSuchObjectException, InvalidObjectException, MetaException, InvalidInputException, org.apache.thrift.TException
NoSuchObjectException
InvalidObjectException
MetaException
InvalidInputException
org.apache.thrift.TException
boolean update_partition_column_statistics(ColumnStatistics stats_obj) throws NoSuchObjectException, InvalidObjectException, MetaException, InvalidInputException, org.apache.thrift.TException
NoSuchObjectException
InvalidObjectException
MetaException
InvalidInputException
org.apache.thrift.TException
ColumnStatistics get_table_column_statistics(String db_name, String tbl_name, String col_name) throws NoSuchObjectException, MetaException, InvalidInputException, InvalidObjectException, org.apache.thrift.TException
NoSuchObjectException
MetaException
InvalidInputException
InvalidObjectException
org.apache.thrift.TException
ColumnStatistics get_partition_column_statistics(String db_name, String tbl_name, String part_name, String col_name) throws NoSuchObjectException, MetaException, InvalidInputException, InvalidObjectException, org.apache.thrift.TException
NoSuchObjectException
MetaException
InvalidInputException
InvalidObjectException
org.apache.thrift.TException
TableStatsResult get_table_statistics_req(TableStatsRequest request) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
PartitionsStatsResult get_partitions_statistics_req(PartitionsStatsRequest request) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
AggrStats get_aggr_stats_for(PartitionsStatsRequest request) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
boolean set_aggr_stats_for(SetPartitionsStatsRequest request) throws NoSuchObjectException, InvalidObjectException, MetaException, InvalidInputException, org.apache.thrift.TException
NoSuchObjectException
InvalidObjectException
MetaException
InvalidInputException
org.apache.thrift.TException
boolean delete_partition_column_statistics(String db_name, String tbl_name, String part_name, String col_name) throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException, org.apache.thrift.TException
NoSuchObjectException
MetaException
InvalidObjectException
InvalidInputException
org.apache.thrift.TException
boolean delete_table_column_statistics(String db_name, String tbl_name, String col_name) throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException, org.apache.thrift.TException
NoSuchObjectException
MetaException
InvalidObjectException
InvalidInputException
org.apache.thrift.TException
void create_function(Function func) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, org.apache.thrift.TException
AlreadyExistsException
InvalidObjectException
MetaException
NoSuchObjectException
org.apache.thrift.TException
void drop_function(String dbName, String funcName) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
NoSuchObjectException
MetaException
org.apache.thrift.TException
void alter_function(String dbName, String funcName, Function newFunc) throws InvalidOperationException, MetaException, org.apache.thrift.TException
InvalidOperationException
MetaException
org.apache.thrift.TException
List<String> get_functions(String dbName, String pattern) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
Function get_function(String dbName, String funcName) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
MetaException
NoSuchObjectException
org.apache.thrift.TException
boolean create_role(Role role) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
boolean drop_role(String role_name) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
List<String> get_role_names() throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
boolean grant_role(String role_name, String principal_name, PrincipalType principal_type, String grantor, PrincipalType grantorType, boolean grant_option) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
boolean revoke_role(String role_name, String principal_name, PrincipalType principal_type) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
List<Role> list_roles(String principal_name, PrincipalType principal_type) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
GrantRevokeRoleResponse grant_revoke_role(GrantRevokeRoleRequest request) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
GetPrincipalsInRoleResponse get_principals_in_role(GetPrincipalsInRoleRequest request) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
GetRoleGrantsForPrincipalResponse get_role_grants_for_principal(GetRoleGrantsForPrincipalRequest request) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
PrincipalPrivilegeSet get_privilege_set(HiveObjectRef hiveObject, String user_name, List<String> group_names) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
List<HiveObjectPrivilege> list_privileges(String principal_name, PrincipalType principal_type, HiveObjectRef hiveObject) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
boolean grant_privileges(PrivilegeBag privileges) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
boolean revoke_privileges(PrivilegeBag privileges) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
GrantRevokePrivilegeResponse grant_revoke_privileges(GrantRevokePrivilegeRequest request) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
List<String> set_ugi(String user_name, List<String> group_names) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
String get_delegation_token(String token_owner, String renewer_kerberos_principal_name) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
long renew_delegation_token(String token_str_form) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
void cancel_delegation_token(String token_str_form) throws MetaException, org.apache.thrift.TException
MetaException
org.apache.thrift.TException
GetOpenTxnsResponse get_open_txns() throws org.apache.thrift.TException
org.apache.thrift.TException
GetOpenTxnsInfoResponse get_open_txns_info() throws org.apache.thrift.TException
org.apache.thrift.TException
OpenTxnsResponse open_txns(OpenTxnRequest rqst) throws org.apache.thrift.TException
org.apache.thrift.TException
void abort_txn(AbortTxnRequest rqst) throws NoSuchTxnException, org.apache.thrift.TException
NoSuchTxnException
org.apache.thrift.TException
void commit_txn(CommitTxnRequest rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException
NoSuchTxnException
TxnAbortedException
org.apache.thrift.TException
LockResponse lock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException
NoSuchTxnException
TxnAbortedException
org.apache.thrift.TException
LockResponse check_lock(CheckLockRequest rqst) throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, org.apache.thrift.TException
NoSuchTxnException
TxnAbortedException
NoSuchLockException
org.apache.thrift.TException
void unlock(UnlockRequest rqst) throws NoSuchLockException, TxnOpenException, org.apache.thrift.TException
NoSuchLockException
TxnOpenException
org.apache.thrift.TException
ShowLocksResponse show_locks(ShowLocksRequest rqst) throws org.apache.thrift.TException
org.apache.thrift.TException
void heartbeat(HeartbeatRequest ids) throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException
NoSuchLockException
NoSuchTxnException
TxnAbortedException
org.apache.thrift.TException
HeartbeatTxnRangeResponse heartbeat_txn_range(HeartbeatTxnRangeRequest txns) throws org.apache.thrift.TException
org.apache.thrift.TException
void compact(CompactionRequest rqst) throws org.apache.thrift.TException
org.apache.thrift.TException
ShowCompactResponse show_compact(ShowCompactRequest rqst) throws org.apache.thrift.TException
org.apache.thrift.TException
void add_dynamic_partitions(AddDynamicPartitions rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException
NoSuchTxnException
TxnAbortedException
org.apache.thrift.TException
NotificationEventResponse get_next_notification(NotificationEventRequest rqst) throws org.apache.thrift.TException
org.apache.thrift.TException
CurrentNotificationEventId get_current_notificationEventId() throws org.apache.thrift.TException
org.apache.thrift.TException
FireEventResponse fire_listener_event(FireEventRequest rqst) throws org.apache.thrift.TException
org.apache.thrift.TException
Copyright © 2017 The Apache Software Foundation. All rights reserved.