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;
92class KuduColumnarScanBatch;
94class KuduDeleteIgnore;
96class KuduInsertIgnore;
97class KuduLoggingCallback;
101class KuduStatusCallback;
102class KuduTableAlterer;
103class KuduTableCreator;
104class KuduTableStatistics;
106class KuduTabletServer;
108class KuduUpdateIgnore;
110class KuduUpsertIgnore;
112class KuduWriteOperation;
113class ResourceMetrics;
118class GetTableSchemaRpc;
122class RemoteTabletServer;
123class ReplicaController;
124class RetrieveAuthzTokenRpc;
125class ScanBatchDataInterface;
126class TabletInfoProvider;
128template <
class ReqClass,
class RespClass>
130template <
class ReqClass,
class RespClass>
154void KUDU_EXPORT UninstallLoggingCallback();
171void KUDU_EXPORT SetVerboseLogLevel(
int level);
182Status KUDU_EXPORT SetInternalSignalNumber(
int signum) WARN_UNUSED_RESULT;
195Status KUDU_EXPORT DisableSaslInitialization() WARN_UNUSED_RESULT;
220Status KUDU_EXPORT DisableOpenSSLInitialization() WARN_UNUSED_RESULT;
224std::
string KUDU_EXPORT GetShortVersionString();
228std::
string KUDU_EXPORT GetAllVersionInfo();
406 class KUDU_NO_EXPORT Data;
408 friend class internal::ReplicaController;
434 public sp::enable_shared_from_this<KuduTransaction> {
513 Status IsCommitComplete(
bool* is_complete,
514 Status* completion_status) WARN_UNUSED_RESULT;
593 class KUDU_NO_EXPORT Data;
626 std::string* serialized_txn,
645 const std::
string& serialized_txn,
652 FRIEND_TEST(ClientTest, TxnIdOfTransactionalSession);
653 FRIEND_TEST(ClientTest, TxnToken);
655 class KUDU_NO_EXPORT Data;
706 bool* create_in_progress);
749 uint32_t reserve_seconds = 0);
767 Status DeleteTableInCatalogs(
const std::string& table_name,
768 bool modify_external_catalogs,
769 int32_t reserve_seconds = -1) KUDU_NO_EXPORT;
779 Status RecallTable(const std::
string& table_id, const std::
string& new_table_name = "");
799 Status IsAlterTableInProgress(const std::
string& table_name,
800 bool* alter_in_progress);
808 Status GetTableSchema(const std::
string& table_name,
827 Status ListTables(std::vector<std::
string>* tables,
828 const std::
string& filter = "");
838 Status ListSoftDeletedTables(std::vector<std::
string>* tables,
839 const std::
string& filter = "");
848 Status TableExists(const std::
string& table_name,
bool* exists);
866 Status OpenTable(const std::
string& table_name,
915 Status GetTablet(const std::
string& tablet_id,
925 Status GetTableStatistics(const std::
string& table_name,
935 std::
string GetMasterAddresses() const KUDU_NO_EXPORT;
1017 std::string GetHiveMetastoreUris() const KUDU_NO_EXPORT;
1024 bool GetHiveMetastoreSaslEnabled() const KUDU_NO_EXPORT;
1035 std::
string GetHiveMetastoreUuid() const KUDU_NO_EXPORT;
1043 std::
string location() const KUDU_NO_EXPORT;
1048 std::
string cluster_id() const KUDU_NO_EXPORT;
1052 class KUDU_NO_EXPORT Data;
1054 template <class ReqClass, class RespClass>
1055 friend class internal::AsyncLeaderMasterRpc;
1056 template <class ReqClass, class RespClass>
1057 friend class internal::AsyncRandomTxnManagerRpc;
1059 friend class ClientTest;
1060 friend class ConnectToClusterBaseTest;
1071 friend class internal::Batcher;
1072 friend class internal::GetTableSchemaRpc;
1073 friend class internal::LookupRpc;
1074 friend class internal::MetaCache;
1075 friend class internal::RemoteTablet;
1076 friend class internal::RemoteTabletServer;
1077 friend class internal::RetrieveAuthzTokenRpc;
1078 friend class internal::TabletInfoProvider;
1079 friend class internal::WriteRpc;
1080 friend class kudu::AuthzTokenTest;
1081 friend class kudu::DisableWriteWhenExceedingQuotaTest;
1082 friend class kudu::SecurityUnknownTskTest;
1083 friend class transactions::CoordinatorRpc;
1084 friend class transactions::ParticipantRpc;
1085 friend class transactions::TxnSystemClient;
1086 friend class tools::LeaderMasterProxy;
1087 friend class tools::RemoteKsckCluster;
1088 friend class tools::TableLister;
1089 friend class ScanTokenTest;
1091 FRIEND_TEST(kudu::ClientStressTest, TestUniqueClientIds);
1092 FRIEND_TEST(kudu::MetaCacheLookupStressTest, PerfSynthetic);
1093 FRIEND_TEST(ClientTest, ClearCacheAndConcurrentWorkload);
1094 FRIEND_TEST(ClientTest, ConnectionNegotiationTimeout);
1095 FRIEND_TEST(ClientTest, TestBasicIdBasedLookup);
1096 FRIEND_TEST(ClientTest, TestCacheAuthzTokens);
1097 FRIEND_TEST(ClientTest, TestGetSecurityInfoFromMaster);
1098 FRIEND_TEST(ClientTest, TestGetTabletServerBlacklist);
1099 FRIEND_TEST(ClientTest, TestGetTabletServerDeterministic);
1100 FRIEND_TEST(ClientTest, TestMasterDown);
1101 FRIEND_TEST(ClientTest, TestMasterLookupPermits);
1102 FRIEND_TEST(ClientTest, TestMetaCacheExpiry);
1103 FRIEND_TEST(ClientTest, TestMetaCacheExpiryById);
1104 FRIEND_TEST(ClientTest, TestMetaCacheExpiryWithKeysAndIds);
1105 FRIEND_TEST(ClientTest, TestMetaCacheLookupNoLeaders);
1106 FRIEND_TEST(ClientTest, TestMetaCacheWithKeysAndIds);
1107 FRIEND_TEST(ClientTest, TestNonCoveringRangePartitions);
1108 FRIEND_TEST(ClientTest, TestRetrieveAuthzTokenInParallel);
1109 FRIEND_TEST(ClientTest, TestReplicatedTabletWritesWithLeaderElection);
1110 FRIEND_TEST(ClientTest, TestScanFaultTolerance);
1111 FRIEND_TEST(ClientTest, TestScanTimeout);
1112 FRIEND_TEST(ClientTest, TestWriteWithDeadMaster);
1113 FRIEND_TEST(MasterFailoverTest, TestPauseAfterCreateTableIssued);
1114 FRIEND_TEST(MultiTServerClientTest, TestSetReplicationFactor);
1147 const std::string& location() const KUDU_NO_EXPORT;
1151 class KUDU_NO_EXPORT Data;
1182 friend class internal::ReplicaController;
1184 class KUDU_NO_EXPORT Data;
1201 const std::string&
id()
const;
1208 const std::vector<const KuduReplica*>&
replicas()
const;
1214 class KUDU_NO_EXPORT Data;
1271 int32_t num_buckets);
1289 int32_t num_buckets,
1395 ATTRIBUTE_DEPRECATED(
"use add_range_partition_split() instead");
1466 class KUDU_NO_EXPORT Data;
1469 friend class transactions::TxnSystemClient;
1504 KuduTableCreator::INCLUSIVE_BOUND,
1506 KuduTableCreator::EXCLUSIVE_BOUND);
1529 int32_t num_buckets,
1532 class KUDU_NO_EXPORT Data;
1583 class KUDU_NO_EXPORT Data;
1613class KUDU_EXPORT
KuduTable :
public sp::enable_shared_from_this<KuduTable> {
1627 const std::string&
id()
const;
1740 std::vector<KuduBloomFilter*>* bloom_filters);
1783 const std::vector<Slice>& bloom_filters);
1810 std::vector<KuduValue*>* values);
1857 Status ListPartitions(std::vector<Partition>* partitions) KUDU_NO_EXPORT;
1862 class KUDU_NO_EXPORT Data;
1869 KuduTable(
const sp::shared_ptr<KuduClient>& client,
1870 const std::string& name,
1871 const std::string&
id,
1873 const std::string& owner,
1874 const std::string& comment,
1876 const PartitionSchema& partition_schema,
1877 const std::map<std::string, std::string>& extra_configs);
2044 const std::string& dimension_label,
2154 KuduTableAlterer* modify_external_catalogs(
bool modify_external_catalogs) KUDU_NO_EXPORT;
2165 class KUDU_NO_EXPORT Data;
2168 friend class tools::TableAlter;
2169 friend class kudu::AlterTableTest;
2171 FRIEND_TEST(MultiTServerClientTest, TestSetReplicationFactor);
2174 const std::string& name);
2219 class KUDU_NO_EXPORT Data;
2221 friend class internal::Batcher;
2222 friend class internal::ErrorCollector;
2292class KUDU_EXPORT
KuduSession :
public sp::enable_shared_from_this<KuduSession> {
2604 bool HasPendingOperations() const;
2627 int CountBufferedOperations() const
2628 ATTRIBUTE_DEPRECATED("this method is experimental and will disappear "
2629 "in a future release");
2655 Status SetErrorBufferSpace(
size_t size_bytes);
2666 int CountPendingErrors() const;
2678 void GetPendingErrors(std::vector<
KuduError*>* errors,
bool* overflowed);
2687 class KUDU_NO_EXPORT Data;
2689 friend class ClientTest;
2692 friend class internal::Batcher;
2693 friend class tablet::FuzzTest;
2694 FRIEND_TEST(ClientTest, TestAutoFlushBackgroundAndErrorCollector);
2695 FRIEND_TEST(ClientTest, TestAutoFlushBackgroundApplyBlocks);
2696 FRIEND_TEST(ClientTest, TxnIdOfTransactionalSession);
2780 enum { kScanTimeoutMillis = 30000 };
2821 ATTRIBUTE_DEPRECATED(
"use SetProjectedColumnNames() instead");
2851 ATTRIBUTE_DEPRECATED(
"use AddLowerBound() instead");
2871 ATTRIBUTE_DEPRECATED(
"use AddExclusiveUpperBound() instead");
2956 ATTRIBUTE_DEPRECATED(
"use NextBatch(KuduScanBatch*) instead");
3028 ATTRIBUTE_DEPRECATED(
"use SetFaultTolerant() instead");
3049 Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros) WARN_UNUSED_RESULT;
3060 Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT;
3078 Status SetDiffScan(uint64_t start_timestamp, uint64_t end_timestamp)
3079 WARN_UNUSED_RESULT KUDU_NO_EXPORT;
3105 static const uint64_t NO_FLAGS = 0;
3112 static const uint64_t PAD_UNIXTIME_MICROS_TO_16_BYTES = 1 << 0;
3121 static const uint64_t COLUMNAR_LAYOUT = 1 << 1;
3158 Status SetLimit(int64_t limit) WARN_UNUSED_RESULT;
3165 std::
string ToString() const;
3168 class KUDU_NO_EXPORT Data;
3170 Status NextBatch(internal::ScanBatchDataInterface* batch);
3173 friend class FlexPartitioningTest;
3174 FRIEND_TEST(ClientTest, TestBlockScannerHijackingAttempts);
3175 FRIEND_TEST(ClientTest, TestScanCloseProxy);
3176 FRIEND_TEST(ClientTest, TestScanFaultTolerance);
3177 FRIEND_TEST(ClientTest, TestScanNoBlockCaching);
3178 FRIEND_TEST(ClientTest, TestScanTimeout);
3179 FRIEND_TEST(ClientTest, TestReadAtSnapshotNoTimestampSet);
3180 FRIEND_TEST(ConsistencyITest, TestSnapshotScanTimestampReuse);
3181 FRIEND_TEST(ScanTokenTest, TestScanTokens);
3182 FRIEND_TEST(ScanTokenTest, TestScanTokens_NonUniquePrimaryKey);
3252 const std::string& serialized_token,
3256 class KUDU_NO_EXPORT Data;
3342 Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros)
3346 Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT;
3351 Status SetDiffScan(uint64_t start_timestamp, uint64_t end_timestamp)
3352 WARN_UNUSED_RESULT KUDU_NO_EXPORT;
3356 Status SetTimeoutMillis(
int millis) WARN_UNUSED_RESULT;
3365 Status IncludeTableMetadata(
bool include_metadata) WARN_UNUSED_RESULT;
3374 Status IncludeTabletMetadata(
bool include_metadata) WARN_UNUSED_RESULT;
3386 void SetSplitSizeBytes(uint64_t split_size_bytes);
3399 class KUDU_NO_EXPORT Data;
3444 class KUDU_NO_EXPORT Data;
3484 class KUDU_NO_EXPORT Data;
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:165
A "factory" for KuduClient objects.
Definition client.h:234
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:240
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:687
void SetLatestObservedTimestamp(uint64_t ht_timestamp)
static const uint64_t kNoTimestamp
Definition client.h:967
bool IsMultiMaster() const
KuduTableCreator * NewTableCreator()
Status IsCreateTableInProgress(const std::string &table_name, bool *create_in_progress)
const MonoDelta & default_admin_operation_timeout() const
MonoDelta connection_negotiation_timeout() const
const MonoDelta & default_rpc_timeout() const
Status DeleteTable(const std::string &table_name)
uint64_t GetLatestObservedTimestamp() const
Status SoftDeleteTable(const std::string &table_name, uint32_t reserve_seconds=0)
Status ExportAuthenticationCredentials(std::string *authn_creds) const
ReplicaSelection
Policy with which to choose amongst multiple replicas.
Definition client.h:940
@ LEADER_ONLY
Select the LEADER replica.
Definition client.h:941
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:2187
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
Builder for Partitioner instances.
Definition client.h:3408
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)
In-memory representation of a remote tablet's replica.
Definition client.h:1166
const KuduTabletServer & ts() const
A batch of zero or more rows returned by a scan operation.
Definition scan_batch.h:84
Builds scan tokens for a table.
Definition client.h:3271
Status SetProjectedColumnIndexes(const std::vector< int > &col_indexes) WARN_UNUSED_RESULT
Status SetProjectedColumnNames(const std::vector< std::string > &col_names) WARN_UNUSED_RESULT
Status SetSelection(KuduClient::ReplicaSelection selection) WARN_UNUSED_RESULT
Status SetBatchSizeBytes(uint32_t batch_size) WARN_UNUSED_RESULT
Status SetFaultTolerant() WARN_UNUSED_RESULT
Status SetCacheBlocks(bool cache_blocks) WARN_UNUSED_RESULT
KuduScanTokenBuilder(KuduTable *table)
Status AddConjunctPredicate(KuduPredicate *pred) WARN_UNUSED_RESULT
Status AddUpperBound(const KuduPartialRow &key) WARN_UNUSED_RESULT
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:3210
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:2712
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:2715
@ READ_AT_SNAPSHOT
Definition client.h:2744
@ READ_LATEST
Definition client.h:2723
OrderMode
Definition client.h:2762
@ UNORDERED
Definition client.h:2768
Status NextBatch(std::vector< KuduRowResult > *rows)
Status AddConjunctPredicate(KuduPredicate *pred) WARN_UNUSED_RESULT
Status SetCacheBlocks(bool cache_blocks)
Status AddExclusiveUpperBound(const KuduPartialRow &key)
Status SetProjectedColumns(const std::vector< std::string > &col_names) WARN_UNUSED_RESULT
Status NextBatch(KuduColumnarScanBatch *batch)
Status AddLowerBound(const KuduPartialRow &key)
Status SetProjectedColumnIndexes(const std::vector< int > &col_indexes) WARN_UNUSED_RESULT
Status SetOrderMode(OrderMode order_mode) WARN_UNUSED_RESULT
Status AddExclusiveUpperBoundPartitionKeyRaw(const Slice &partition_key)
Status AddExclusiveUpperBoundRaw(const Slice &key)
Status AddLowerBoundPartitionKeyRaw(const Slice &partition_key)
Status SetProjectedColumnNames(const std::vector< std::string > &col_names) WARN_UNUSED_RESULT
Status GetCurrentServer(KuduTabletServer **server)
const ResourceMetrics & GetResourceMetrics() const
Status NextBatch(KuduScanBatch *batch)
Status SetSelection(KuduClient::ReplicaSelection selection) WARN_UNUSED_RESULT
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:2292
void SetTimeoutMillis(int millis)
Status SetExternalConsistencyMode(ExternalConsistencyMode m) WARN_UNUSED_RESULT
Status Flush() WARN_UNUSED_RESULT
Status SetMutationBufferFlushWatermark(double watermark_pct) WARN_UNUSED_RESULT
Status SetFlushMode(FlushMode m) WARN_UNUSED_RESULT
ExternalConsistencyMode
The possible external consistency modes on which Kudu operates.
Definition client.h:2358
@ CLIENT_PROPAGATED
Definition client.h:2377
Status SetMutationBufferFlushInterval(unsigned int millis) WARN_UNUSED_RESULT
FlushMode
Modes of flush operations.
Definition client.h:2297
@ AUTO_FLUSH_BACKGROUND
Definition client.h:2330
@ AUTO_FLUSH_SYNC
Definition client.h:2302
Status SetMutationBufferMaxNum(unsigned int max_num) WARN_UNUSED_RESULT
Status Apply(KuduWriteOperation *write_op) WARN_UNUSED_RESULT
Status SetMutationBufferSpace(size_t size_bytes) WARN_UNUSED_RESULT
The interface for all status callbacks.
Definition callbacks.h:161
Alters an existing table based on the provided steps.
Definition client.h:1896
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:1225
KuduTableCreator & timeout(const MonoDelta &timeout)
KuduTableCreator & add_range_partition_split(KuduPartialRow *split_row)
RangePartitionBound
Range partition bound type.
Definition client.h:1307
@ INCLUSIVE_BOUND
An inclusive bound.
Definition client.h:1309
@ EXCLUSIVE_BOUND
An exclusive bound.
Definition client.h:1308
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:1544
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:1613
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:1125
const std::string & hostname() const
const std::string & uuid() const
In-memory representation of a remote tablet.
Definition client.h:1195
const std::string & id() const
const std::vector< const KuduReplica * > & replicas() const
SerializationOptions & enable_keepalive(bool enable)
Status Commit() 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.