Kudu C++ client API
client.h
1 // Licensed to the Apache Software Foundation (ASF) under one
2 // or more contributor license agreements. See the NOTICE file
3 // distributed with this work for additional information
4 // regarding copyright ownership. The ASF licenses this file
5 // to you under the Apache License, Version 2.0 (the
6 // "License"); you may not use this file except in compliance
7 // with the License. You may obtain a copy of the License at
8 //
9 // http://www.apache.org/licenses/LICENSE-2.0
10 //
11 // Unless required by applicable law or agreed to in writing,
12 // software distributed under the License is distributed on an
13 // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14 // KIND, either express or implied. See the License for the
15 // specific language governing permissions and limitations
16 // under the License.
17 
24 
25 #ifndef KUDU_CLIENT_CLIENT_H
26 #define KUDU_CLIENT_CLIENT_H
27 
28 #include <stdint.h>
29 #include <string>
30 #include <vector>
31 
32 #include "kudu/client/resource_metrics.h"
33 #include "kudu/client/row_result.h"
34 #include "kudu/client/scan_batch.h"
35 #include "kudu/client/scan_predicate.h"
36 #include "kudu/client/schema.h"
37 #include "kudu/client/shared_ptr.h"
38 #ifdef KUDU_HEADERS_NO_STUBS
39 #include <gtest/gtest_prod.h>
40 #include "kudu/gutil/macros.h"
41 #include "kudu/gutil/port.h"
42 #else
43 #include "kudu/client/stubs.h"
44 #endif
45 #include "kudu/client/write_op.h"
46 #include "kudu/util/kudu_export.h"
47 #include "kudu/util/monotime.h"
48 #include "kudu/util/status.h"
49 
50 namespace kudu {
51 
52 class ClientStressTest_TestUniqueClientIds_Test;
53 class LinkedListTester;
54 class PartitionSchema;
55 
56 namespace client {
57 
58 class KuduLoggingCallback;
59 class KuduScanToken;
60 class KuduSession;
61 class KuduStatusCallback;
62 class KuduTable;
63 class KuduTableAlterer;
64 class KuduTableCreator;
65 class KuduTablet;
66 class KuduTabletServer;
67 class KuduValue;
68 class KuduWriteOperation;
69 
70 namespace internal {
71 class Batcher;
72 class ErrorCollector;
73 class GetTableSchemaRpc;
74 class LookupRpc;
75 class MetaCache;
76 class RemoteTablet;
77 class RemoteTabletServer;
78 class WriteRpc;
79 } // namespace internal
80 
93 void KUDU_EXPORT InstallLoggingCallback(KuduLoggingCallback* cb);
94 
101 void KUDU_EXPORT UninstallLoggingCallback();
102 
118 void KUDU_EXPORT SetVerboseLogLevel(int level);
119 
129 Status KUDU_EXPORT SetInternalSignalNumber(int signum);
130 
142 Status KUDU_EXPORT DisableSaslInitialization();
143 
144 
167 Status KUDU_EXPORT DisableOpenSSLInitialization();
168 
171 std::string KUDU_EXPORT GetShortVersionString();
172 
175 std::string KUDU_EXPORT GetAllVersionInfo();
176 
181 class KUDU_EXPORT KuduClientBuilder {
182  public:
185 
189  KuduClientBuilder& clear_master_server_addrs();
190 
196  KuduClientBuilder& master_server_addrs(const std::vector<std::string>& addrs);
197 
205  KuduClientBuilder& add_master_server_addr(const std::string& addr);
206 
216  KuduClientBuilder& default_admin_operation_timeout(const MonoDelta& timeout);
217 
225  KuduClientBuilder& default_rpc_timeout(const MonoDelta& timeout);
226 
234  KuduClientBuilder& import_authentication_credentials(std::string authn_creds);
235 
246  Status Build(sp::shared_ptr<KuduClient>* client);
247 
248  private:
249  class KUDU_NO_EXPORT Data;
250 
251  // Owned.
252  Data* data_;
253 
254  DISALLOW_COPY_AND_ASSIGN(KuduClientBuilder);
255 };
256 
283 class KUDU_EXPORT KuduClient : public sp::enable_shared_from_this<KuduClient> {
284  public:
285  ~KuduClient();
286 
291  KuduTableCreator* NewTableCreator();
292 
301  Status IsCreateTableInProgress(const std::string& table_name,
302  bool *create_in_progress);
303 
309  Status DeleteTable(const std::string& table_name);
310 
317  KuduTableAlterer* NewTableAlterer(const std::string& table_name);
318 
327  Status IsAlterTableInProgress(const std::string& table_name,
328  bool *alter_in_progress);
336  Status GetTableSchema(const std::string& table_name,
337  KuduSchema* schema);
338 
345  Status ListTabletServers(std::vector<KuduTabletServer*>* tablet_servers);
346 
354  Status ListTables(std::vector<std::string>* tables,
355  const std::string& filter = "");
356 
364  Status TableExists(const std::string& table_name, bool* exists);
365 
379  Status OpenTable(const std::string& table_name,
380  sp::shared_ptr<KuduTable>* table);
381 
387  sp::shared_ptr<KuduSession> NewSession();
388 
390 
403  Status KUDU_NO_EXPORT GetTablet(const std::string& tablet_id,
404  KuduTablet** tablet);
405 
407 
411 
412  CLOSEST_REPLICA,
413 
415  FIRST_REPLICA
416  };
417 
420  bool IsMultiMaster() const;
421 
423  const MonoDelta& default_admin_operation_timeout() const;
424 
426  const MonoDelta& default_rpc_timeout() const;
427 
430  static const uint64_t kNoTimestamp;
431 
471  uint64_t GetLatestObservedTimestamp() const;
472 
487  void SetLatestObservedTimestamp(uint64_t ht_timestamp);
488 
500  Status ExportAuthenticationCredentials(std::string* authn_creds) const;
501 
502  private:
503  class KUDU_NO_EXPORT Data;
504 
505  friend class internal::Batcher;
506  friend class internal::GetTableSchemaRpc;
507  friend class internal::LookupRpc;
508  friend class internal::MetaCache;
509  friend class internal::RemoteTablet;
510  friend class internal::RemoteTabletServer;
511  friend class internal::WriteRpc;
512  friend class ClientTest;
513  friend class KuduClientBuilder;
514  friend class KuduScanner;
515  friend class KuduScanToken;
516  friend class KuduScanTokenBuilder;
517  friend class KuduSession;
518  friend class KuduTable;
519  friend class KuduTableAlterer;
520  friend class KuduTableCreator;
521 
522  FRIEND_TEST(kudu::ClientStressTest, TestUniqueClientIds);
523  FRIEND_TEST(ClientTest, TestGetSecurityInfoFromMaster);
524  FRIEND_TEST(ClientTest, TestGetTabletServerBlacklist);
525  FRIEND_TEST(ClientTest, TestMasterDown);
526  FRIEND_TEST(ClientTest, TestMasterLookupPermits);
527  FRIEND_TEST(ClientTest, TestMetaCacheExpiry);
528  FRIEND_TEST(ClientTest, TestNonCoveringRangePartitions);
529  FRIEND_TEST(ClientTest, TestReplicatedTabletWritesWithLeaderElection);
530  FRIEND_TEST(ClientTest, TestScanFaultTolerance);
531  FRIEND_TEST(ClientTest, TestScanTimeout);
532  FRIEND_TEST(ClientTest, TestWriteWithDeadMaster);
533  FRIEND_TEST(MasterFailoverTest, TestPauseAfterCreateTableIssued);
534 
535  KuduClient();
536 
537  // Owned.
538  Data* data_;
539 
540  DISALLOW_COPY_AND_ASSIGN(KuduClient);
541 };
542 
544 class KUDU_EXPORT KuduTabletServer {
545  public:
546  ~KuduTabletServer();
547 
550  const std::string& uuid() const;
551 
554  const std::string& hostname() const;
555 
558  uint16_t port() const;
559 
560  private:
561  class KUDU_NO_EXPORT Data;
562 
563  friend class KuduClient;
564  friend class KuduScanner;
565  friend class KuduScanTokenBuilder;
566 
568 
569  // Owned.
570  Data* data_;
571 
572  DISALLOW_COPY_AND_ASSIGN(KuduTabletServer);
573 };
574 
576 class KUDU_EXPORT KuduReplica {
577  public:
578  ~KuduReplica();
579 
584  bool is_leader() const;
585 
587  const KuduTabletServer& ts() const;
588 
589  private:
590  friend class KuduClient;
591  friend class KuduScanTokenBuilder;
592 
593  class KUDU_NO_EXPORT Data;
594 
595  KuduReplica();
596 
597  // Owned.
598  Data* data_;
599 
600  DISALLOW_COPY_AND_ASSIGN(KuduReplica);
601 };
602 
604 class KUDU_EXPORT KuduTablet {
605  public:
606  ~KuduTablet();
607 
610  const std::string& id() const;
611 
617  const std::vector<const KuduReplica*>& replicas() const;
618 
619  private:
620  friend class KuduClient;
621  friend class KuduScanTokenBuilder;
622 
623  class KUDU_NO_EXPORT Data;
624 
625  KuduTablet();
626 
627  // Owned.
628  Data* data_;
629 
630  DISALLOW_COPY_AND_ASSIGN(KuduTablet);
631 };
632 
634 class KUDU_EXPORT KuduTableCreator {
635  public:
636  ~KuduTableCreator();
637 
649  KuduTableCreator& table_name(const std::string& name);
650 
661  KuduTableCreator& schema(const KuduSchema* schema);
662 
679  KuduTableCreator& add_hash_partitions(const std::vector<std::string>& columns,
680  int32_t num_buckets);
681 
697  KuduTableCreator& add_hash_partitions(const std::vector<std::string>& columns,
698  int32_t num_buckets, int32_t seed);
699 
712  KuduTableCreator& set_range_partition_columns(const std::vector<std::string>& columns);
713 
718  };
719 
748  KuduTableCreator& add_range_partition(KuduPartialRow* lower_bound,
749  KuduPartialRow* upper_bound,
750  RangePartitionBound lower_bound_type = INCLUSIVE_BOUND,
751  RangePartitionBound upper_bound_type = EXCLUSIVE_BOUND);
752 
761  KuduTableCreator& add_range_partition_split(KuduPartialRow* split_row);
762 
768  KuduTableCreator& split_rows(const std::vector<const KuduPartialRow*>& split_rows)
769  ATTRIBUTE_DEPRECATED("use add_range_partition_split() instead");
770 
780  KuduTableCreator& num_replicas(int n_replicas);
781 
791  KuduTableCreator& timeout(const MonoDelta& timeout);
792 
800  KuduTableCreator& wait(bool wait);
801 
815  Status Create();
816 
817  private:
818  class KUDU_NO_EXPORT Data;
819 
820  friend class KuduClient;
821 
822  explicit KuduTableCreator(KuduClient* client);
823 
824  // Owned.
825  Data* data_;
826 
827  DISALLOW_COPY_AND_ASSIGN(KuduTableCreator);
828 };
829 
850 class KUDU_EXPORT KuduTable : public sp::enable_shared_from_this<KuduTable> {
851  public:
852  ~KuduTable();
853 
855  const std::string& name() const;
856 
864  const std::string& id() const;
865 
867  const KuduSchema& schema() const;
868 
870  int num_replicas() const;
871 
875  KuduInsert* NewInsert();
876 
880  KuduUpsert* NewUpsert();
881 
885  KuduUpdate* NewUpdate();
886 
890  KuduDelete* NewDelete();
891 
915  KuduPredicate* NewComparisonPredicate(const Slice& col_name,
917  KuduValue* value);
918 
942  KuduPredicate* NewInListPredicate(const Slice& col_name,
943  std::vector<KuduValue*>* values);
944 
955  KuduPredicate* NewIsNotNullPredicate(const Slice& col_name);
956 
967  KuduPredicate* NewIsNullPredicate(const Slice& col_name);
968 
971  KuduClient* client() const;
972 
974  const PartitionSchema& partition_schema() const;
975 
976  private:
977  class KUDU_NO_EXPORT Data;
978 
979  friend class KuduClient;
980 
981  KuduTable(const sp::shared_ptr<KuduClient>& client,
982  const std::string& name,
983  const std::string& id,
984  int num_replicas,
985  const KuduSchema& schema,
986  const PartitionSchema& partition_schema);
987 
988  // Owned.
989  Data* data_;
990 
991  DISALLOW_COPY_AND_ASSIGN(KuduTable);
992 };
993 
1005 class KUDU_EXPORT KuduTableAlterer {
1006  public:
1007  ~KuduTableAlterer();
1008 
1014  KuduTableAlterer* RenameTo(const std::string& new_name);
1015 
1025  KuduColumnSpec* AddColumn(const std::string& name);
1026 
1035  KuduColumnSpec* AlterColumn(const std::string& name);
1036 
1044  KuduTableAlterer* DropColumn(const std::string& name);
1045 
1077  KuduTableAlterer* AddRangePartition(
1078  KuduPartialRow* lower_bound,
1079  KuduPartialRow* upper_bound,
1082 
1110  KuduTableAlterer* DropRangePartition(
1111  KuduPartialRow* lower_bound,
1112  KuduPartialRow* upper_bound,
1115 
1125  KuduTableAlterer* timeout(const MonoDelta& timeout);
1126 
1138  KuduTableAlterer* wait(bool wait);
1139 
1144  Status Alter();
1145 
1146  private:
1147  class KUDU_NO_EXPORT Data;
1148  friend class KuduClient;
1149 
1150  KuduTableAlterer(KuduClient* client,
1151  const std::string& name);
1152 
1153  // Owned.
1154  Data* data_;
1155 
1156  DISALLOW_COPY_AND_ASSIGN(KuduTableAlterer);
1157 };
1158 
1164 class KUDU_EXPORT KuduError {
1165  public:
1166  ~KuduError();
1167 
1169  const Status& status() const;
1170 
1172  const KuduWriteOperation& failed_op() const;
1173 
1181  KuduWriteOperation* release_failed_op();
1182 
1193  bool was_possibly_successful() const;
1194 
1195  private:
1196  class KUDU_NO_EXPORT Data;
1197 
1198  friend class internal::Batcher;
1199  friend class internal::ErrorCollector;
1200  friend class KuduSession;
1201 
1202  KuduError(KuduWriteOperation* failed_op, const Status& error);
1203 
1204  // Owned.
1205  Data* data_;
1206 
1207  DISALLOW_COPY_AND_ASSIGN(KuduError);
1208 };
1209 
1210 
1269 class KUDU_EXPORT KuduSession : public sp::enable_shared_from_this<KuduSession> {
1270  public:
1271  ~KuduSession();
1272 
1274  enum FlushMode {
1280 
1308 
1321  MANUAL_FLUSH
1322  };
1323 
1332  Status SetFlushMode(FlushMode m) WARN_UNUSED_RESULT;
1333 
1355 
1370  COMMIT_WAIT
1371  };
1372 
1378  Status SetExternalConsistencyMode(ExternalConsistencyMode m)
1379  WARN_UNUSED_RESULT;
1380 
1398  Status SetMutationBufferSpace(size_t size_bytes) WARN_UNUSED_RESULT;
1399 
1423  Status SetMutationBufferFlushWatermark(double watermark_pct)
1424  WARN_UNUSED_RESULT;
1425 
1447  Status SetMutationBufferFlushInterval(unsigned int millis) WARN_UNUSED_RESULT;
1448 
1473  Status SetMutationBufferMaxNum(unsigned int max_num) WARN_UNUSED_RESULT;
1474 
1480  void SetTimeoutMillis(int millis);
1481 
1485 
1509  Status Apply(KuduWriteOperation* write_op) WARN_UNUSED_RESULT;
1510 
1525  Status Flush() WARN_UNUSED_RESULT;
1526 
1567  void FlushAsync(KuduStatusCallback* cb);
1568 
1571  Status Close() WARN_UNUSED_RESULT;
1572 
1581  bool HasPendingOperations() const;
1582 
1604  int CountBufferedOperations() const
1605  ATTRIBUTE_DEPRECATED("this method is experimental and will disappear "
1606  "in a future release");
1607 
1632  Status SetErrorBufferSpace(size_t size_bytes);
1633 
1643  int CountPendingErrors() const;
1644 
1655  void GetPendingErrors(std::vector<KuduError*>* errors, bool* overflowed);
1656 
1658  KuduClient* client() const;
1659 
1660  private:
1661  class KUDU_NO_EXPORT Data;
1662 
1663  friend class KuduClient;
1664  friend class internal::Batcher;
1665  friend class ClientTest;
1666  FRIEND_TEST(ClientTest, TestAutoFlushBackgroundApplyBlocks);
1667  FRIEND_TEST(ClientTest, TestAutoFlushBackgroundAndErrorCollector);
1668 
1669  explicit KuduSession(const sp::shared_ptr<KuduClient>& client);
1670 
1671  // Owned.
1672  Data* data_;
1673 
1674  DISALLOW_COPY_AND_ASSIGN(KuduSession);
1675 };
1676 
1677 
1682 class KUDU_EXPORT KuduScanner {
1683  public:
1685  enum ReadMode {
1694 
1714  READ_AT_SNAPSHOT
1715  };
1716 
1720  enum OrderMode {
1727 
1732  ORDERED
1733  };
1734 
1738  enum { kScanTimeoutMillis = 30000 };
1739 
1745  explicit KuduScanner(KuduTable* table);
1746  ~KuduScanner();
1747 
1757  Status SetProjectedColumnNames(const std::vector<std::string>& col_names)
1758  WARN_UNUSED_RESULT;
1759 
1769  Status SetProjectedColumnIndexes(const std::vector<int>& col_indexes)
1770  WARN_UNUSED_RESULT;
1771 
1777  Status SetProjectedColumns(const std::vector<std::string>& col_names)
1778  WARN_UNUSED_RESULT
1779  ATTRIBUTE_DEPRECATED("use SetProjectedColumnNames() instead");
1780 
1789  Status AddConjunctPredicate(KuduPredicate* pred) WARN_UNUSED_RESULT;
1790 
1799  Status AddLowerBound(const KuduPartialRow& key);
1800 
1808  Status AddLowerBoundRaw(const Slice& key)
1809  ATTRIBUTE_DEPRECATED("use AddLowerBound() instead");
1810 
1819  Status AddExclusiveUpperBound(const KuduPartialRow& key);
1820 
1828  Status AddExclusiveUpperBoundRaw(const Slice& key)
1829  ATTRIBUTE_DEPRECATED("use AddExclusiveUpperBound() instead");
1830 
1839  Status AddLowerBoundPartitionKeyRaw(const Slice& partition_key);
1840 
1849  Status AddExclusiveUpperBoundPartitionKeyRaw(const Slice& partition_key);
1850 
1857  Status SetCacheBlocks(bool cache_blocks);
1858 
1860  Status Open();
1861 
1877  Status KeepAlive();
1878 
1887  void Close();
1888 
1897  bool HasMoreRows() const;
1898 
1910  Status NextBatch(std::vector<KuduRowResult>* rows)
1911  ATTRIBUTE_DEPRECATED("use NextBatch(KuduScanBatch*) instead");
1912 
1921  Status NextBatch(KuduScanBatch* batch);
1922 
1931  Status GetCurrentServer(KuduTabletServer** server);
1932 
1934  const ResourceMetrics& GetResourceMetrics() const;
1935 
1942  Status SetBatchSizeBytes(uint32_t batch_size);
1943 
1951  Status SetSelection(KuduClient::ReplicaSelection selection)
1952  WARN_UNUSED_RESULT;
1953 
1959  Status SetReadMode(ReadMode read_mode) WARN_UNUSED_RESULT;
1960 
1966  Status SetOrderMode(OrderMode order_mode) WARN_UNUSED_RESULT
1967  ATTRIBUTE_DEPRECATED("use SetFaultTolerant() instead");
1968 
1981  Status SetFaultTolerant() WARN_UNUSED_RESULT;
1982 
1988  Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros) WARN_UNUSED_RESULT;
1989 
2002  Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT;
2003 
2009  Status SetTimeoutMillis(int millis);
2010 
2012  KuduSchema GetProjectionSchema() const;
2013 
2019  std::string ToString() const;
2020 
2021  private:
2022  class KUDU_NO_EXPORT Data;
2023 
2024  friend class KuduScanToken;
2025  FRIEND_TEST(ClientTest, TestScanCloseProxy);
2026  FRIEND_TEST(ClientTest, TestScanFaultTolerance);
2027  FRIEND_TEST(ClientTest, TestScanNoBlockCaching);
2028  FRIEND_TEST(ClientTest, TestScanTimeout);
2029  FRIEND_TEST(ClientTest, TestReadAtSnapshotNoTimestampSet);
2030  FRIEND_TEST(ConsistencyITest, TestSnapshotScanTimestampReuse);
2031 
2032  // Owned.
2033  Data* data_;
2034 
2035  DISALLOW_COPY_AND_ASSIGN(KuduScanner);
2036 };
2037 
2058 class KUDU_EXPORT KuduScanToken {
2059  public:
2060 
2061  ~KuduScanToken();
2062 
2073  Status IntoKuduScanner(KuduScanner** scanner) const WARN_UNUSED_RESULT;
2074 
2076  const KuduTablet& tablet() const;
2077 
2085  Status Serialize(std::string* buf) const WARN_UNUSED_RESULT;
2086 
2098  static Status DeserializeIntoScanner(KuduClient* client,
2099  const std::string& serialized_token,
2100  KuduScanner** scanner) WARN_UNUSED_RESULT;
2101 
2102  private:
2103  class KUDU_NO_EXPORT Data;
2104 
2105  friend class KuduScanTokenBuilder;
2106 
2107  KuduScanToken();
2108 
2109  // Owned.
2110  Data* data_;
2111 
2112  DISALLOW_COPY_AND_ASSIGN(KuduScanToken);
2113 };
2114 
2118 class KUDU_EXPORT KuduScanTokenBuilder {
2119  public:
2120 
2126  explicit KuduScanTokenBuilder(KuduTable* table);
2128 
2138  Status SetProjectedColumnNames(const std::vector<std::string>& col_names)
2139  WARN_UNUSED_RESULT;
2140 
2142  Status SetProjectedColumnIndexes(const std::vector<int>& col_indexes)
2143  WARN_UNUSED_RESULT;
2144 
2146  Status AddConjunctPredicate(KuduPredicate* pred) WARN_UNUSED_RESULT;
2147 
2149  Status AddLowerBound(const KuduPartialRow& key) WARN_UNUSED_RESULT;
2150 
2159  Status AddUpperBound(const KuduPartialRow& key) WARN_UNUSED_RESULT;
2160 
2162  Status SetCacheBlocks(bool cache_blocks) WARN_UNUSED_RESULT;
2163 
2170  Status SetBatchSizeBytes(uint32_t batch_size) WARN_UNUSED_RESULT;
2171 
2179  Status SetSelection(KuduClient::ReplicaSelection selection)
2180  WARN_UNUSED_RESULT;
2181 
2183  Status SetReadMode(KuduScanner::ReadMode read_mode) WARN_UNUSED_RESULT;
2184 
2186  Status SetFaultTolerant() WARN_UNUSED_RESULT;
2187 
2189  Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros)
2190  WARN_UNUSED_RESULT;
2191 
2193  Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT;
2194 
2196  Status SetTimeoutMillis(int millis) WARN_UNUSED_RESULT;
2197 
2206  Status Build(std::vector<KuduScanToken*>* tokens) WARN_UNUSED_RESULT;
2207 
2208  private:
2209  class KUDU_NO_EXPORT Data;
2210 
2211  // Owned.
2212  Data* data_;
2213 
2214  DISALLOW_COPY_AND_ASSIGN(KuduScanTokenBuilder);
2215 };
2216 
2217 } // namespace client
2218 } // namespace kudu
2219 #endif
A single row update to be sent to the cluster.
Definition: write_op.h:184
A representation of a table&#39;s schema.
Definition: schema.h:417
A representation of an operation&#39;s outcome.
Definition: status.h:130
A constant cell value with a specific type.
Definition: value.h:33
ExternalConsistencyMode
The possible external consistency modes on which Kudu operates.
Definition: client.h:1335
Definition: callbacks.h:28
A single row insert to be sent to the cluster.
Definition: write_op.h:132
Builder API for specifying or altering a column within a table schema.
Definition: schema.h:230
static const uint64_t kNoTimestamp
Definition: client.h:430
ReplicaSelection
Policy with which to choose amongst multiple replicas.
Definition: client.h:409
A single row upsert to be sent to the cluster.
Definition: write_op.h:157
Builds scan tokens for a table.
Definition: client.h:2118
ComparisonOp
Supported comparison operators.
Definition: scan_predicate.h:39
Alters an existing table based on the provided steps.
Definition: client.h:1005
OrderMode
Definition: client.h:1720
Definition: client.h:1726
Smart pointer typedefs for externally-faced code.
A representation of comparison predicate for Kudu queries.
Definition: scan_predicate.h:36
An exclusive bound.
Definition: client.h:716
This class represents an error which occurred in a write operation.
Definition: client.h:1164
A handle for a connection to a cluster.
Definition: client.h:283
An inclusive bound.
Definition: client.h:717
In-memory representation of a remote tablet server.
Definition: client.h:544
The interface for all status callbacks.
Definition: callbacks.h:161
A wrapper around externally allocated data.
Definition: slice.h:43
A representation of a table on a particular cluster.
Definition: client.h:850
This class is a representation of a single scan.
Definition: client.h:1682
ReadMode
The read modes for scanners.
Definition: client.h:1685
A "factory" for KuduClient objects.
Definition: client.h:181
A single-row write operation to be sent to a Kudu table.
Definition: write_op.h:56
Select the LEADER replica.
Definition: client.h:410
RangePartitionBound
Range partition bound type.
Definition: client.h:715
In-memory representation of a remote tablet.
Definition: client.h:604
In-memory representation of a remote tablet&#39;s replica.
Definition: client.h:576
FlushMode
Modes of flush operations.
Definition: client.h:1274
A scan descriptor limited to a single physical contiguous location.
Definition: client.h:2058
A single row delete to be sent to the cluster.
Definition: write_op.h:211
A generic catalog of simple metrics.
Definition: resource_metrics.h:30
A helper class to create a new table with the desired options.
Definition: client.h:634
A row which may only contain values for a subset of the columns.
Definition: partial_row.h:61
Representation of a Kudu client session.
Definition: client.h:1269
A batch of zero or more rows returned by a scan operation.
Definition: scan_batch.h:75
A representation of a time interval.
Definition: monotime.h:43