25#ifndef KUDU_CLIENT_CLIENT_H
26#define KUDU_CLIENT_CLIENT_H
38#include "kudu/client/row_result.h"
39#include "kudu/client/scan_predicate.h"
40#include "kudu/client/schema.h"
42#ifdef KUDU_HEADERS_NO_STUBS
43#include <gtest/gtest_prod.h>
45#include "kudu/gutil/macros.h"
46#include "kudu/gutil/port.h"
48#include "kudu/client/stubs.h"
50#include "kudu/util/kudu_export.h"
51#include "kudu/util/slice.h"
58class ClientStressTest_TestUniqueClientIds_Test;
59class DisableWriteWhenExceedingQuotaTest;
61class MetaCacheLookupStressTest_PerfSynthetic_Test;
65class SecurityUnknownTskTest;
77namespace transactions {
84class LeaderMasterProxy;
85class RemoteKsckCluster;
88Status ShowTabletInfo(
const std::vector<std::string>& master_addresses,
89 const std::vector<std::string>& tablet_id);
120class GetTableSchemaRpc;
124class RemoteTabletServer;
125class ReplicaController;
126class RetrieveAuthzTokenRpc;
127class ScanBatchDataInterface;
128class TabletInfoProvider;
130template <
class ReqClass,
class RespClass>
132template <
class ReqClass,
class RespClass>
156void KUDU_EXPORT UninstallLoggingCallback();
173void KUDU_EXPORT SetVerboseLogLevel(
int level);
184Status KUDU_EXPORT SetInternalSignalNumber(
int signum) WARN_UNUSED_RESULT;
197Status KUDU_EXPORT DisableSaslInitialization() WARN_UNUSED_RESULT;
222Status KUDU_EXPORT DisableOpenSSLInitialization() WARN_UNUSED_RESULT;
226std::
string KUDU_EXPORT GetShortVersionString();
230std::
string KUDU_EXPORT GetAllVersionInfo();
236class KUDU_EXPORT KuduClientBuilder {
239 ~KuduClientBuilder();
329 KuduClientBuilder&
jwt(
const std::string&
jwt);
418 class KUDU_NO_EXPORT Data;
420 friend class internal::ReplicaController;
425 DISALLOW_COPY_AND_ASSIGN(KuduClientBuilder);
445class KUDU_EXPORT KuduTransaction :
446 public sp::enable_shared_from_this<KuduTransaction> {
526 Status* completion_status) WARN_UNUSED_RESULT;
542 class KUDU_EXPORT SerializationOptions {
544 SerializationOptions();
545 ~SerializationOptions();
604 friend class KuduTransaction;
605 class KUDU_NO_EXPORT Data;
609 DISALLOW_COPY_AND_ASSIGN(SerializationOptions);
638 std::string* serialized_txn,
657 const std::
string& serialized_txn,
658 sp::shared_ptr<KuduTransaction>* txn) WARN_UNUSED_RESULT;
660 DISALLOW_COPY_AND_ASSIGN(KuduTransaction);
662 friend class KuduClient;
663 friend class KuduSession;
664 FRIEND_TEST(ClientTest, TxnIdOfTransactionalSession);
665 FRIEND_TEST(ClientTest, TxnToken);
667 class KUDU_NO_EXPORT Data;
669 explicit KuduTransaction(const sp::shared_ptr<KuduClient>& client);
699class KUDU_EXPORT KuduClient : public sp::enable_shared_from_this<KuduClient> {
718 bool* create_in_progress);
761 uint32_t reserve_seconds = 0);
779 Status DeleteTableInCatalogs(
const std::string& table_name,
780 bool modify_external_catalogs,
781 int32_t reserve_seconds = -1) KUDU_NO_EXPORT;
791 Status RecallTable(const std::
string& table_id, const std::
string& new_table_name =
"");
812 bool* alter_in_progress);
840 const std::
string& filter =
"");
851 const std::
string& filter =
"");
879 sp::shared_ptr<KuduTable>* table);
927 Status GetTablet(const std::
string& tablet_id,
937 Status GetTableStatistics(const std::
string& table_name,
947 std::
string GetMasterAddresses() const KUDU_NO_EXPORT;
1029 std::string GetHiveMetastoreUris() const KUDU_NO_EXPORT;
1036 bool GetHiveMetastoreSaslEnabled() const KUDU_NO_EXPORT;
1047 std::
string GetHiveMetastoreUuid() const KUDU_NO_EXPORT;
1055 std::
string location() const KUDU_NO_EXPORT;
1060 std::
string cluster_id() const KUDU_NO_EXPORT;
1064 class KUDU_NO_EXPORT Data;
1066 template <class ReqClass, class RespClass>
1067 friend class internal::AsyncLeaderMasterRpc;
1068 template <class ReqClass, class RespClass>
1069 friend class internal::AsyncRandomTxnManagerRpc;
1071 friend class ClientTest;
1072 friend class ConnectToClusterBaseTest;
1073 friend class KuduClientBuilder;
1074 friend class KuduPartitionerBuilder;
1075 friend class KuduTransaction;
1076 friend class KuduScanToken;
1077 friend class KuduScanTokenBuilder;
1078 friend class KuduScanner;
1079 friend class KuduSession;
1080 friend class KuduTable;
1081 friend class KuduTableAlterer;
1082 friend class KuduTableCreator;
1083 friend class internal::Batcher;
1084 friend class internal::GetTableSchemaRpc;
1085 friend class internal::LookupRpc;
1086 friend class internal::MetaCache;
1087 friend class internal::RemoteTablet;
1088 friend class internal::RemoteTabletServer;
1089 friend class internal::RetrieveAuthzTokenRpc;
1090 friend class internal::TabletInfoProvider;
1091 friend class internal::WriteRpc;
1092 friend class kudu::AuthzTokenTest;
1093 friend class kudu::DisableWriteWhenExceedingQuotaTest;
1094 friend class kudu::SecurityUnknownTskTest;
1095 friend class transactions::CoordinatorRpc;
1096 friend class transactions::ParticipantRpc;
1097 friend class transactions::TxnSystemClient;
1098 friend class tools::LeaderMasterProxy;
1099 friend class tools::RemoteKsckCluster;
1100 friend class tools::TableLister;
1101 friend class KeepAlivePeriodicallyTest;
1102 friend class ScanTokenTest;
1103 friend
Status tools::ShowTabletInfo(const std::vector<std::
string>& master_addresses,
1104 const std::vector<std::
string>& tablet_id);
1106 FRIEND_TEST(kudu::ClientStressTest, TestUniqueClientIds);
1107 FRIEND_TEST(kudu::MetaCacheLookupStressTest, PerfSynthetic);
1108 FRIEND_TEST(ClientTest, ClearCacheAndConcurrentWorkload);
1109 FRIEND_TEST(ClientTest, ConnectionNegotiationTimeout);
1110 FRIEND_TEST(ClientTest, TestBasicIdBasedLookup);
1111 FRIEND_TEST(ClientTest, TestCacheAuthzTokens);
1112 FRIEND_TEST(ClientTest, TestGetSecurityInfoFromMaster);
1113 FRIEND_TEST(ClientTest, TestGetTabletServerBlacklist);
1114 FRIEND_TEST(ClientTest, TestGetTabletServerDeterministic);
1115 FRIEND_TEST(ClientTest, TestMasterDown);
1116 FRIEND_TEST(ClientTest, TestMasterLookupPermits);
1117 FRIEND_TEST(ClientTest, TestMetaCacheExpiry);
1118 FRIEND_TEST(ClientTest, TestMetaCacheExpiryById);
1119 FRIEND_TEST(ClientTest, TestMetaCacheExpiryWithKeysAndIds);
1120 FRIEND_TEST(ClientTest, TestMetaCacheLookupNoLeaders);
1121 FRIEND_TEST(ClientTest, TestMetaCacheWithKeysAndIds);
1122 FRIEND_TEST(ClientTest, TestNonCoveringRangePartitions);
1123 FRIEND_TEST(ClientTest, TestRetrieveAuthzTokenInParallel);
1124 FRIEND_TEST(ClientTest, TestReplicatedTabletWritesWithLeaderElection);
1125 FRIEND_TEST(ClientTest, TestScanFaultTolerance);
1126 FRIEND_TEST(ClientTest, TestScanTimeout);
1127 FRIEND_TEST(ClientTest, TestWriteWithDeadMaster);
1128 FRIEND_TEST(MasterFailoverTest, TestPauseAfterCreateTableIssued);
1129 FRIEND_TEST(MultiTServerClientTest, TestSetReplicationFactor);
1136 DISALLOW_COPY_AND_ASSIGN(KuduClient);
1140class KUDU_EXPORT KuduTabletServer {
1142 ~KuduTabletServer();
1162 const std::string& location() const KUDU_NO_EXPORT;
1166 class KUDU_NO_EXPORT Data;
1168 friend class KuduClient;
1169 friend class KuduScanner;
1170 friend class KuduScanTokenBuilder;
1171 friend
Status tools::ShowTabletInfo(const std::vector<std::
string>& master_addresses,
1172 const std::vector<std::
string>& tablet_id);
1179 DISALLOW_COPY_AND_ASSIGN(KuduTabletServer);
1183class KUDU_EXPORT KuduReplica {
1197 friend class KuduClient;
1198 friend class KuduScanTokenBuilder;
1199 friend class internal::ReplicaController;
1201 class KUDU_NO_EXPORT Data;
1208 DISALLOW_COPY_AND_ASSIGN(KuduReplica);
1212class KUDU_EXPORT KuduTablet {
1218 const std::string&
id()
const;
1225 const std::vector<const KuduReplica*>&
replicas()
const;
1228 friend class KuduClient;
1229 friend class KuduScanTokenBuilder;
1231 class KUDU_NO_EXPORT Data;
1238 DISALLOW_COPY_AND_ASSIGN(KuduTablet);
1242class KUDU_EXPORT KuduTableCreator {
1244 ~KuduTableCreator();
1288 int32_t num_buckets);
1306 int32_t num_buckets,
1420 ATTRIBUTE_DEPRECATED(
"use add_range_partition_split() instead");
1491 class KUDU_NO_EXPORT Data;
1493 friend class KuduClient;
1494 friend class transactions::TxnSystemClient;
1496 explicit KuduTableCreator(KuduClient* client);
1501 DISALLOW_COPY_AND_ASSIGN(KuduTableCreator);
1554 int32_t num_buckets,
1557 class KUDU_NO_EXPORT Data;
1559 friend class KuduTableCreator;
1560 friend class KuduTableAlterer;
1569class KUDU_EXPORT KuduTableStatistics {
1571 KuduTableStatistics();
1572 ~KuduTableStatistics();
1608 class KUDU_NO_EXPORT Data;
1610 friend class KuduClient;
1615 DISALLOW_COPY_AND_ASSIGN(KuduTableStatistics);
1638class KUDU_EXPORT KuduTable :
public sp::enable_shared_from_this<KuduTable> {
1652 const std::string&
id()
const;
1765 std::vector<KuduBloomFilter*>* bloom_filters);
1808 const std::vector<Slice>& bloom_filters);
1835 std::vector<KuduValue*>* values);
1882 Status ListPartitions(std::vector<Partition>* partitions) KUDU_NO_EXPORT;
1887 class KUDU_NO_EXPORT Data;
1889 friend class KuduClient;
1890 friend class KuduPartitioner;
1891 friend class KuduScanToken;
1892 friend class KuduScanner;
1894 KuduTable(
const sp::shared_ptr<KuduClient>&
client,
1895 const std::string&
name,
1896 const std::string&
id,
1898 const std::string&
owner,
1907 DISALLOW_COPY_AND_ASSIGN(KuduTable);
1921class KUDU_EXPORT KuduTableAlterer {
1923 ~KuduTableAlterer();
1930 KuduTableAlterer*
RenameTo(
const std::string& new_name);
1937 KuduTableAlterer*
SetOwner(
const std::string& new_owner);
2069 const std::string& dimension_label,
2179 KuduTableAlterer* modify_external_catalogs(
bool modify_external_catalogs) KUDU_NO_EXPORT;
2190 class KUDU_NO_EXPORT Data;
2192 friend class KuduClient;
2193 friend class tools::TableAlter;
2194 friend class kudu::AlterTableTest;
2196 FRIEND_TEST(MultiTServerClientTest, TestSetReplicationFactor);
2198 KuduTableAlterer(KuduClient* client,
2199 const std::string& name);
2204 DISALLOW_COPY_AND_ASSIGN(KuduTableAlterer);
2212class KUDU_EXPORT KuduError {
2244 class KUDU_NO_EXPORT Data;
2246 friend class internal::Batcher;
2247 friend class internal::ErrorCollector;
2248 friend class KuduSession;
2255 DISALLOW_COPY_AND_ASSIGN(KuduError);
2317class KUDU_EXPORT KuduSession :
public sp::enable_shared_from_this<KuduSession> {
2653 ATTRIBUTE_DEPRECATED(
"this method is experimental and will disappear "
2654 "in a future release");
2712 class KUDU_NO_EXPORT Data;
2714 friend class ClientTest;
2715 friend class KuduClient;
2716 friend class KuduTransaction;
2717 friend class internal::Batcher;
2718 friend class tablet::FuzzTest;
2719 FRIEND_TEST(ClientTest, TestAutoFlushBackgroundAndErrorCollector);
2720 FRIEND_TEST(ClientTest, TestAutoFlushBackgroundApplyBlocks);
2721 FRIEND_TEST(ClientTest, TxnIdOfTransactionalSession);
2723 explicit KuduSession(const sp::shared_ptr<KuduClient>&
client);
2724 KuduSession(const sp::shared_ptr<KuduClient>&
client, const TxnId& txn_id);
2729 DISALLOW_COPY_AND_ASSIGN(KuduSession);
2805 enum { kScanTimeoutMillis = 30000 };
2847 ATTRIBUTE_DEPRECATED(
"use SetProjectedColumnNames() instead");
2898 ATTRIBUTE_DEPRECATED(
"use AddLowerBound() instead");
2918 ATTRIBUTE_DEPRECATED(
"use AddExclusiveUpperBound() instead");
3029 ATTRIBUTE_DEPRECATED(
"use NextBatch(KuduScanBatch*) instead");
3101 ATTRIBUTE_DEPRECATED(
"use SetFaultTolerant() instead");
3151 Status SetDiffScan(uint64_t start_timestamp, uint64_t end_timestamp)
3152 WARN_UNUSED_RESULT KUDU_NO_EXPORT;
3241 class KUDU_NO_EXPORT Data;
3245 friend class KuduScanToken;
3246 friend class FlexPartitioningTest;
3247 FRIEND_TEST(ClientTest, TestBlockScannerHijackingAttempts);
3248 FRIEND_TEST(ClientTest, TestScanCloseProxy);
3249 FRIEND_TEST(ClientTest, TestScanFaultTolerance);
3250 FRIEND_TEST(ClientTest, TestScanNoBlockCaching);
3251 FRIEND_TEST(ClientTest, TestScanTimeout);
3252 FRIEND_TEST(ClientTest, TestScanWithQueryId);
3253 FRIEND_TEST(ClientTest, TestReadAtSnapshotNoTimestampSet);
3254 FRIEND_TEST(ConsistencyITest, TestSnapshotScanTimestampReuse);
3255 FRIEND_TEST(KeepAlivePeriodicallyTest, TestScannerKeepAlivePeriodicallyCrossServers);
3256 FRIEND_TEST(KeepAlivePeriodicallyTest, TestScannerKeepAlivePeriodicallyScannerTolerate);
3257 FRIEND_TEST(KeepAlivePeriodicallyTest, TestStopKeepAlivePeriodically);
3258 FRIEND_TEST(ScanTokenTest, TestScanTokens);
3259 FRIEND_TEST(ScanTokenTest, TestScanTokens_NonUniquePrimaryKey);
3260 FRIEND_TEST(ScanTokenTest, TestScanTokensWithQueryId);
3288class KUDU_EXPORT KuduScanToken {
3330 const std::string& serialized_token,
3334 class KUDU_NO_EXPORT Data;
3336 friend class KuduScanTokenBuilder;
3343 DISALLOW_COPY_AND_ASSIGN(KuduScanToken);
3429 Status SetDiffScan(uint64_t start_timestamp, uint64_t end_timestamp)
3430 WARN_UNUSED_RESULT KUDU_NO_EXPORT;
3486 class KUDU_NO_EXPORT Data;
3531 class KUDU_NO_EXPORT Data;
3549class KUDU_EXPORT KuduPartitioner {
3571 class KUDU_NO_EXPORT Data;
3573 friend class KuduPartitionerBuilder;
3575 explicit KuduPartitioner(Data* data);
3578 DISALLOW_COPY_AND_ASSIGN(KuduPartitioner);
A row which may only contain values for a subset of the columns.
Definition partial_row.h:72
A representation of a time interval.
Definition monotime.h:58
A wrapper around externally allocated data.
Definition slice.h:51
A representation of an operation's outcome.
Definition status.h:191
KuduClientBuilder & encryption_policy(EncryptionPolicy encryption_policy)
KuduClientBuilder & sasl_protocol_name(const std::string &sasl_protocol_name)
KuduClientBuilder & require_authentication(bool require_authentication)
Status Build(sp::shared_ptr< KuduClient > *client)
KuduClientBuilder & default_rpc_timeout(const MonoDelta &timeout)
KuduClientBuilder & master_server_addrs(const std::vector< std::string > &addrs)
KuduClientBuilder & clear_master_server_addrs()
EncryptionPolicy
Policy for on-the-wire encryption.
Definition client.h:242
@ REQUIRED
Definition client.h:249
@ REQUIRED_REMOTE
Definition client.h:246
@ OPTIONAL
Definition client.h:243
KuduClientBuilder & rpc_max_message_size(int64_t size)
KuduClientBuilder & import_authentication_credentials(std::string authn_creds)
KuduClientBuilder & jwt(const std::string &jwt)
KuduClientBuilder & default_admin_operation_timeout(const MonoDelta &timeout)
KuduClientBuilder & num_reactors(int num_reactors)
Set the number of reactors for the RPC messenger.
KuduClientBuilder & trusted_certificate(const std::string &cert_pem)
KuduClientBuilder & add_master_server_addr(const std::string &addr)
KuduClientBuilder & connection_negotiation_timeout(const MonoDelta &timeout)
A handle for a connection to a cluster.
Definition client.h:699
void SetLatestObservedTimestamp(uint64_t ht_timestamp)
static const uint64_t kNoTimestamp
Definition client.h:979
bool IsMultiMaster() const
Status GetTableSchema(const std::string &table_name, KuduSchema *schema)
KuduTableCreator * NewTableCreator()
Status IsCreateTableInProgress(const std::string &table_name, bool *create_in_progress)
Status ListTables(std::vector< std::string > *tables, const std::string &filter="")
KuduTableAlterer * NewTableAlterer(const std::string &table_name)
const MonoDelta & default_admin_operation_timeout() const
MonoDelta connection_negotiation_timeout() const
const MonoDelta & default_rpc_timeout() const
Status ListSoftDeletedTables(std::vector< std::string > *tables, const std::string &filter="")
Status IsAlterTableInProgress(const std::string &table_name, bool *alter_in_progress)
Status TableExists(const std::string &table_name, bool *exists)
Status DeleteTable(const std::string &table_name)
uint64_t GetLatestObservedTimestamp() const
Status OpenTable(const std::string &table_name, sp::shared_ptr< KuduTable > *table)
Status SoftDeleteTable(const std::string &table_name, uint32_t reserve_seconds=0)
sp::shared_ptr< KuduSession > NewSession()
Status NewTransaction(sp::shared_ptr< KuduTransaction > *txn) WARN_UNUSED_RESULT
Status ListTabletServers(std::vector< KuduTabletServer * > *tablet_servers)
Status ExportAuthenticationCredentials(std::string *authn_creds) const
ReplicaSelection
Policy with which to choose amongst multiple replicas.
Definition client.h:952
@ CLOSEST_REPLICA
Definition client.h:955
@ FIRST_REPLICA
Select the first replica in the list.
Definition client.h:961
@ LEADER_ONLY
Select the LEADER replica.
Definition client.h:953
Builder API for specifying or altering a column within a table schema.
Definition schema.h:370
A batch of columnar data returned from a scanner.
Definition columnar_scan_batch.h:51
A single row delete ignore to be sent to the cluster.
Definition write_op.h:331
A single row delete to be sent to the cluster.
Definition write_op.h:305
This class represents an error which occurred in a write operation.
Definition client.h:2212
KuduWriteOperation * release_failed_op()
const KuduWriteOperation & failed_op() const
bool was_possibly_successful() const
const Status & status() const
A single row insert ignore to be sent to the cluster, duplicate row errors are ignored.
Definition write_op.h:172
A single row insert to be sent to the cluster.
Definition write_op.h:145
The interface for all logging callbacks.
Definition callbacks.h:44
KuduPartitionerBuilder(sp::shared_ptr< KuduTable > table)
Status Build(KuduPartitioner **partitioner)
KuduPartitionerBuilder * SetBuildTimeout(MonoDelta timeout)
Status PartitionRow(const KuduPartialRow &row, int *partition)
int NumPartitions() const
A representation of comparison predicate for Kudu queries.
Definition scan_predicate.h:43
ComparisonOp
Supported comparison operators.
Definition scan_predicate.h:46
Status add_hash_partitions(const std::vector< std::string > &columns, int32_t num_buckets, int32_t seed=0)
KuduRangePartition(KuduPartialRow *lower_bound, KuduPartialRow *upper_bound, KuduTableCreator::RangePartitionBound lower_bound_type=KuduTableCreator::INCLUSIVE_BOUND, KuduTableCreator::RangePartitionBound upper_bound_type=KuduTableCreator::EXCLUSIVE_BOUND)
const KuduTabletServer & ts() const
A batch of zero or more rows returned by a scan operation.
Definition scan_batch.h:84
Status SetProjectedColumnIndexes(const std::vector< int > &col_indexes) WARN_UNUSED_RESULT
Status SetProjectedColumnNames(const std::vector< std::string > &col_names) WARN_UNUSED_RESULT
Status SetTimeoutMillis(int millis) WARN_UNUSED_RESULT
Status SetSelection(KuduClient::ReplicaSelection selection) WARN_UNUSED_RESULT
Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT
Status SetBatchSizeBytes(uint32_t batch_size) WARN_UNUSED_RESULT
Status SetFaultTolerant() WARN_UNUSED_RESULT
Status IncludeTableMetadata(bool include_metadata) WARN_UNUSED_RESULT
Status SetCacheBlocks(bool cache_blocks) WARN_UNUSED_RESULT
Status SetQueryId(const std::string &query_id)
Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros) WARN_UNUSED_RESULT
KuduScanTokenBuilder(KuduTable *table)
Status AddConjunctPredicate(KuduPredicate *pred) WARN_UNUSED_RESULT
Status AddUpperBound(const KuduPartialRow &key) WARN_UNUSED_RESULT
Status Build(std::vector< KuduScanToken * > *tokens) WARN_UNUSED_RESULT
Status IncludeTabletMetadata(bool include_metadata) WARN_UNUSED_RESULT
void SetSplitSizeBytes(uint64_t split_size_bytes)
Status SetReadMode(KuduScanner::ReadMode read_mode) WARN_UNUSED_RESULT
Status AddLowerBound(const KuduPartialRow &key) WARN_UNUSED_RESULT
A scan descriptor limited to a single physical contiguous location.
Definition client.h:3288
Status Serialize(std::string *buf) const WARN_UNUSED_RESULT
const KuduTablet & tablet() const
Status IntoKuduScanner(KuduScanner **scanner) const WARN_UNUSED_RESULT
static Status DeserializeIntoScanner(KuduClient *client, const std::string &serialized_token, KuduScanner **scanner) WARN_UNUSED_RESULT
This class is a representation of a single scan.
Definition client.h:2737
Status SetQueryId(const std::string &query_id)
Status SetFaultTolerant() WARN_UNUSED_RESULT
Status SetReadMode(ReadMode read_mode) WARN_UNUSED_RESULT
KuduScanner(KuduTable *table)
ReadMode
The read modes for scanners.
Definition client.h:2740
@ READ_AT_SNAPSHOT
Definition client.h:2769
@ READ_LATEST
Definition client.h:2748
@ READ_YOUR_WRITES
Definition client.h:2781
OrderMode
Definition client.h:2787
@ ORDERED
Definition client.h:2799
@ UNORDERED
Definition client.h:2793
Status SetRowFormatFlags(uint64_t flags)
Status NextBatch(std::vector< KuduRowResult > *rows)
Status AddConjunctPredicate(KuduPredicate *pred) WARN_UNUSED_RESULT
Status SetCacheBlocks(bool cache_blocks)
Status AddExclusiveUpperBound(const KuduPartialRow &key)
Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros) WARN_UNUSED_RESULT
Status SetProjectedColumns(const std::vector< std::string > &col_names) WARN_UNUSED_RESULT
static const uint64_t PAD_UNIXTIME_MICROS_TO_16_BYTES
Definition client.h:3185
Status NextBatch(KuduColumnarScanBatch *batch)
Status AddLowerBound(const KuduPartialRow &key)
static const uint64_t NO_FLAGS
No flags set.
Definition client.h:3178
KuduSchema GetProjectionSchema() const
Status SetLimit(int64_t limit) WARN_UNUSED_RESULT
Status StartKeepAlivePeriodically(uint64_t keep_alive_interval_ms=30000)
Status SetProjectedColumnIndexes(const std::vector< int > &col_indexes) WARN_UNUSED_RESULT
Status SetOrderMode(OrderMode order_mode) WARN_UNUSED_RESULT
Status SetTimeoutMillis(int millis)
Status AddExclusiveUpperBoundPartitionKeyRaw(const Slice &partition_key)
Status AddExclusiveUpperBoundRaw(const Slice &key)
std::string ToString() const
Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT
Status AddLowerBoundPartitionKeyRaw(const Slice &partition_key)
Status SetProjectedColumnNames(const std::vector< std::string > &col_names) WARN_UNUSED_RESULT
Status GetCurrentServer(KuduTabletServer **server)
static const uint64_t COLUMNAR_LAYOUT
Definition client.h:3194
sp::shared_ptr< KuduTable > GetKuduTable()
const ResourceMetrics & GetResourceMetrics() const
Status NextBatch(KuduScanBatch *batch)
Status SetSelection(KuduClient::ReplicaSelection selection) WARN_UNUSED_RESULT
void StopKeepAlivePeriodically()
Status SetBatchSizeBytes(uint32_t batch_size)
Status AddLowerBoundRaw(const Slice &key)
A representation of a table's schema.
Definition schema.h:688
Representation of a Kudu client session.
Definition client.h:2317
void SetTimeoutMillis(int millis)
int CountBufferedOperations() const
Status SetExternalConsistencyMode(ExternalConsistencyMode m) WARN_UNUSED_RESULT
Status Flush() WARN_UNUSED_RESULT
Status SetMutationBufferFlushWatermark(double watermark_pct) WARN_UNUSED_RESULT
const ResourceMetrics & GetWriteOpMetrics() const
Status SetFlushMode(FlushMode m) WARN_UNUSED_RESULT
int CountPendingErrors() const
bool HasPendingOperations() const
Status Close() WARN_UNUSED_RESULT
ExternalConsistencyMode
The possible external consistency modes on which Kudu operates.
Definition client.h:2383
@ COMMIT_WAIT
Definition client.h:2418
@ CLIENT_PROPAGATED
Definition client.h:2402
Status SetMutationBufferFlushInterval(unsigned int millis) WARN_UNUSED_RESULT
FlushMode
Modes of flush operations.
Definition client.h:2322
@ MANUAL_FLUSH
Definition client.h:2369
@ AUTO_FLUSH_BACKGROUND
Definition client.h:2355
@ AUTO_FLUSH_SYNC
Definition client.h:2327
KuduClient * client() const
Status SetMutationBufferMaxNum(unsigned int max_num) WARN_UNUSED_RESULT
Status Apply(KuduWriteOperation *write_op) WARN_UNUSED_RESULT
void GetPendingErrors(std::vector< KuduError * > *errors, bool *overflowed)
void FlushAsync(KuduStatusCallback *cb)
Status SetMutationBufferSpace(size_t size_bytes) WARN_UNUSED_RESULT
Status SetErrorBufferSpace(size_t size_bytes)
The interface for all status callbacks.
Definition callbacks.h:161
Alters an existing table based on the provided steps.
Definition client.h:1921
KuduColumnSpec * AddColumn(const std::string &name)
KuduTableAlterer * timeout(const MonoDelta &timeout)
KuduTableAlterer * AddRangePartition(KuduPartialRow *lower_bound, KuduPartialRow *upper_bound, KuduTableCreator::RangePartitionBound lower_bound_type=KuduTableCreator::INCLUSIVE_BOUND, KuduTableCreator::RangePartitionBound upper_bound_type=KuduTableCreator::EXCLUSIVE_BOUND)
KuduTableAlterer * DropRangePartition(KuduPartialRow *lower_bound, KuduPartialRow *upper_bound, KuduTableCreator::RangePartitionBound lower_bound_type=KuduTableCreator::INCLUSIVE_BOUND, KuduTableCreator::RangePartitionBound upper_bound_type=KuduTableCreator::EXCLUSIVE_BOUND)
KuduTableAlterer * SetOwner(const std::string &new_owner)
KuduTableAlterer * AddRangePartitionWithDimension(KuduPartialRow *lower_bound, KuduPartialRow *upper_bound, const std::string &dimension_label, KuduTableCreator::RangePartitionBound lower_bound_type=KuduTableCreator::INCLUSIVE_BOUND, KuduTableCreator::RangePartitionBound upper_bound_type=KuduTableCreator::EXCLUSIVE_BOUND)
KuduTableAlterer * SetTableDiskSizeLimit(int64_t disk_size_limit)
KuduTableAlterer * AddRangePartition(KuduRangePartition *partition)
KuduTableAlterer * DropColumn(const std::string &name)
KuduTableAlterer * SetTableRowCountLimit(int64_t row_count_limit)
KuduTableAlterer * AlterExtraConfig(const std::map< std::string, std::string > &extra_configs)
KuduTableAlterer * wait(bool wait)
KuduTableAlterer * RenameTo(const std::string &new_name)
KuduColumnSpec * AlterColumn(const std::string &name)
KuduTableAlterer * SetComment(const std::string &new_comment)
A helper class to create a new table with the desired options.
Definition client.h:1242
KuduTableCreator & timeout(const MonoDelta &timeout)
KuduTableCreator & add_range_partition_split(KuduPartialRow *split_row)
RangePartitionBound
Range partition bound type.
Definition client.h:1324
@ INCLUSIVE_BOUND
An inclusive bound.
Definition client.h:1326
@ EXCLUSIVE_BOUND
An exclusive bound.
Definition client.h:1325
KuduTableCreator & set_allow_empty_partition(bool allow_empty_partition)
KuduTableCreator & add_hash_partitions(const std::vector< std::string > &columns, int32_t num_buckets, int32_t seed)
KuduTableCreator & add_hash_partitions(const std::vector< std::string > &columns, int32_t num_buckets)
KuduTableCreator & set_range_partition_columns(const std::vector< std::string > &columns)
KuduTableCreator & table_name(const std::string &name)
KuduTableCreator & split_rows(const std::vector< const KuduPartialRow * > &split_rows)
KuduTableCreator & add_range_partition(KuduPartialRow *lower_bound, KuduPartialRow *upper_bound, RangePartitionBound lower_bound_type=INCLUSIVE_BOUND, RangePartitionBound upper_bound_type=EXCLUSIVE_BOUND)
KuduTableCreator & dimension_label(const std::string &dimension_label)
KuduTableCreator & num_replicas(int n_replicas)
KuduTableCreator & set_owner(const std::string &owner)
KuduTableCreator & schema(const KuduSchema *schema)
KuduTableCreator & set_comment(const std::string &comment)
KuduTableCreator & extra_configs(const std::map< std::string, std::string > &extra_configs)
KuduTableCreator & add_custom_range_partition(class KuduRangePartition *partition)
KuduTableCreator & wait(bool wait)
In-memory statistics of table.
Definition client.h:1569
int64_t on_disk_size_limit() const
int64_t live_row_count() const
std::string ToString() const
int64_t live_row_count_limit() const
int64_t on_disk_size() const
A representation of a table on a particular cluster.
Definition client.h:1638
KuduUpsertIgnore * NewUpsertIgnore()
KuduPredicate * NewInBloomFilterPredicate(const Slice &col_name, std::vector< KuduBloomFilter * > *bloom_filters)
KuduDeleteIgnore * NewDeleteIgnore()
KuduPredicate * NewIsNullPredicate(const Slice &col_name)
const std::map< std::string, std::string > & extra_configs() const
KuduInsertIgnore * NewInsertIgnore()
KuduUpdateIgnore * NewUpdateIgnore()
const std::string & name() const
KuduPredicate * NewInBloomFilterPredicate(const Slice &col_name, const std::vector< Slice > &bloom_filters)
KuduPredicate * NewInListPredicate(const Slice &col_name, std::vector< KuduValue * > *values)
const std::string & id() const
const PartitionSchema & partition_schema() const
KuduPredicate * NewIsNotNullPredicate(const Slice &col_name)
KuduPredicate * NewComparisonPredicate(const Slice &col_name, KuduPredicate::ComparisonOp op, KuduValue *value)
KuduClient * client() const
const KuduSchema & schema() const
const std::string & comment() const
const std::string & owner() const
In-memory representation of a remote tablet server.
Definition client.h:1140
const std::string & hostname() const
const std::string & uuid() const
In-memory representation of a remote tablet.
Definition client.h:1212
const std::string & id() const
const std::vector< const KuduReplica * > & replicas() const
SerializationOptions & enable_keepalive(bool enable)
Status StartCommit() WARN_UNUSED_RESULT
Status Rollback() WARN_UNUSED_RESULT
Status IsCommitComplete(bool *is_complete, Status *completion_status) WARN_UNUSED_RESULT
Status Commit() WARN_UNUSED_RESULT
static Status Deserialize(const sp::shared_ptr< KuduClient > &client, const std::string &serialized_txn, sp::shared_ptr< KuduTransaction > *txn) WARN_UNUSED_RESULT
Status Serialize(std::string *serialized_txn, const SerializationOptions &options=SerializationOptions()) const WARN_UNUSED_RESULT
Status CreateSession(sp::shared_ptr< KuduSession > *session) WARN_UNUSED_RESULT
A single row update ignore to be sent to the cluster, missing row errors and errors on updating immut...
Definition write_op.h:279
A single row update to be sent to the cluster.
Definition write_op.h:252
A single row upsert ignore to be sent to the cluster, errors on updating immutable cells are ignored.
Definition write_op.h:225
A single row upsert to be sent to the cluster.
Definition write_op.h:198
A constant cell value with a specific type.
Definition value.h:37
A single-row write operation to be sent to a Kudu table.
Definition write_op.h:66
A generic catalog of simple metrics.
Definition resource_metrics.h:41
Smart pointer typedefs for externally-faced code.