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 
30 #include <cstddef>
31 #include <string>
32 #include <vector>
33 
34 #include "kudu/client/row_result.h"
35 #include "kudu/client/scan_predicate.h"
36 #include "kudu/client/schema.h"
37 #include "kudu/client/shared_ptr.h" // IWYU pragma: keep
38 #ifdef KUDU_HEADERS_NO_STUBS
39 #include <gtest/gtest_prod.h>
40 
41 #include "kudu/gutil/macros.h"
42 #include "kudu/gutil/port.h"
43 #else
44 #include "kudu/client/stubs.h"
45 #endif
46 #include "kudu/util/kudu_export.h"
47 #include "kudu/util/slice.h"
48 #include "kudu/util/status.h"
49 
50 namespace kudu {
51 
52 class ClientStressTest_TestUniqueClientIds_Test;
53 class KuduPartialRow;
54 class MonoDelta;
55 class Partition;
56 class PartitionSchema;
57 class SecurityUnknownTskTest;
58 
59 namespace client {
60 class KuduClient;
61 class KuduTable;
62 } // namespace client
63 
64 namespace tools {
65 class LeaderMasterProxy;
66 } // namespace tools
67 
68 namespace client {
69 
70 class KuduDelete;
71 class KuduInsert;
72 class KuduLoggingCallback;
73 class KuduPartitioner;
74 class KuduScanBatch;
75 class KuduSession;
76 class KuduStatusCallback;
77 class KuduTableAlterer;
78 class KuduTableCreator;
79 class KuduTablet;
80 class KuduTabletServer;
81 class KuduUpdate;
82 class KuduUpsert;
83 class KuduValue;
84 class KuduWriteOperation;
85 class ResourceMetrics;
86 
87 namespace internal {
88 template <class ReqClass, class RespClass>
89 class AsyncLeaderMasterRpc; // IWYU pragma: keep
90 class Batcher;
91 class ErrorCollector;
92 class GetTableSchemaRpc;
93 class LookupRpc;
94 class MetaCache;
95 class RemoteTablet;
96 class RemoteTabletServer;
97 class ReplicaController;
98 class WriteRpc;
99 } // namespace internal
100 
113 void KUDU_EXPORT InstallLoggingCallback(KuduLoggingCallback* cb);
114 
121 void KUDU_EXPORT UninstallLoggingCallback();
122 
138 void KUDU_EXPORT SetVerboseLogLevel(int level);
139 
149 Status KUDU_EXPORT SetInternalSignalNumber(int signum) WARN_UNUSED_RESULT;
150 
162 Status KUDU_EXPORT DisableSaslInitialization() WARN_UNUSED_RESULT;
163 
164 
187 Status KUDU_EXPORT DisableOpenSSLInitialization() WARN_UNUSED_RESULT;
188 
191 std::string KUDU_EXPORT GetShortVersionString();
192 
195 std::string KUDU_EXPORT GetAllVersionInfo();
196 
201 class KUDU_EXPORT KuduClientBuilder {
202  public:
203  KuduClientBuilder();
204  ~KuduClientBuilder();
205 
209  KuduClientBuilder& clear_master_server_addrs();
210 
216  KuduClientBuilder& master_server_addrs(const std::vector<std::string>& addrs);
217 
225  KuduClientBuilder& add_master_server_addr(const std::string& addr);
226 
236  KuduClientBuilder& default_admin_operation_timeout(const MonoDelta& timeout);
237 
245  KuduClientBuilder& default_rpc_timeout(const MonoDelta& timeout);
246 
254  KuduClientBuilder& import_authentication_credentials(std::string authn_creds);
255 
265  KuduClientBuilder& num_reactors(int num_reactors);
266 
277  Status Build(sp::shared_ptr<KuduClient>* client);
278 
279  private:
280  class KUDU_NO_EXPORT Data;
281 
282  friend class internal::ReplicaController;
283 
284  // Owned.
285  Data* data_;
286 
287  DISALLOW_COPY_AND_ASSIGN(KuduClientBuilder);
288 };
289 
316 class KUDU_EXPORT KuduClient : public sp::enable_shared_from_this<KuduClient> {
317  public:
318  ~KuduClient();
319 
324  KuduTableCreator* NewTableCreator();
325 
334  Status IsCreateTableInProgress(const std::string& table_name,
335  bool* create_in_progress);
336 
342  Status DeleteTable(const std::string& table_name);
343 
345 
356  Status DeleteTableInCatalogs(const std::string& table_name,
357  bool modify_external_catalogs) KUDU_NO_EXPORT;
359 
366  KuduTableAlterer* NewTableAlterer(const std::string& table_name);
367 
376  Status IsAlterTableInProgress(const std::string& table_name,
377  bool* alter_in_progress);
385  Status GetTableSchema(const std::string& table_name,
386  KuduSchema* schema);
387 
394  Status ListTabletServers(std::vector<KuduTabletServer*>* tablet_servers);
395 
403  Status ListTables(std::vector<std::string>* tables,
404  const std::string& filter = "");
405 
413  Status TableExists(const std::string& table_name, bool* exists);
414 
431  Status OpenTable(const std::string& table_name,
432  sp::shared_ptr<KuduTable>* table);
433 
439  sp::shared_ptr<KuduSession> NewSession();
440 
442 
455  Status GetTablet(const std::string& tablet_id,
456  KuduTablet** tablet) KUDU_NO_EXPORT;
457 
465  std::string GetMasterAddresses() const KUDU_NO_EXPORT;
466 
468 
472 
473  CLOSEST_REPLICA,
474 
479  FIRST_REPLICA
480  };
481 
484  bool IsMultiMaster() const;
485 
487  const MonoDelta& default_admin_operation_timeout() const;
488 
490  const MonoDelta& default_rpc_timeout() const;
491 
494  static const uint64_t kNoTimestamp;
495 
535  uint64_t GetLatestObservedTimestamp() const;
536 
551  void SetLatestObservedTimestamp(uint64_t ht_timestamp);
552 
564  Status ExportAuthenticationCredentials(std::string* authn_creds) const;
565 
567 
573  std::string GetHiveMetastoreUris() const KUDU_NO_EXPORT;
574 
580  bool GetHiveMetastoreSaslEnabled() const KUDU_NO_EXPORT;
581 
591  std::string GetHiveMetastoreUuid() const KUDU_NO_EXPORT;
592 
599  std::string location() const KUDU_NO_EXPORT;
601 
602  private:
603  class KUDU_NO_EXPORT Data;
604 
605  template <class ReqClass, class RespClass>
606  friend class internal::AsyncLeaderMasterRpc;
607 
608  friend class ClientTest;
609  friend class ConnectToClusterBaseTest;
610  friend class KuduClientBuilder;
611  friend class KuduPartitionerBuilder;
612  friend class KuduScanToken;
613  friend class KuduScanTokenBuilder;
614  friend class KuduScanner;
615  friend class KuduSession;
616  friend class KuduTable;
617  friend class KuduTableAlterer;
618  friend class KuduTableCreator;
619  friend class internal::Batcher;
620  friend class internal::GetTableSchemaRpc;
621  friend class internal::LookupRpc;
622  friend class internal::MetaCache;
623  friend class internal::RemoteTablet;
624  friend class internal::RemoteTabletServer;
625  friend class internal::WriteRpc;
626  friend class kudu::SecurityUnknownTskTest;
627  friend class tools::LeaderMasterProxy;
628 
629  FRIEND_TEST(kudu::ClientStressTest, TestUniqueClientIds);
630  FRIEND_TEST(ClientTest, TestGetSecurityInfoFromMaster);
631  FRIEND_TEST(ClientTest, TestGetTabletServerBlacklist);
632  FRIEND_TEST(ClientTest, TestMasterDown);
633  FRIEND_TEST(ClientTest, TestMasterLookupPermits);
634  FRIEND_TEST(ClientTest, TestMetaCacheExpiry);
635  FRIEND_TEST(ClientTest, TestNonCoveringRangePartitions);
636  FRIEND_TEST(ClientTest, TestReplicatedTabletWritesWithLeaderElection);
637  FRIEND_TEST(ClientTest, TestScanFaultTolerance);
638  FRIEND_TEST(ClientTest, TestScanTimeout);
639  FRIEND_TEST(ClientTest, TestWriteWithDeadMaster);
640  FRIEND_TEST(MasterFailoverTest, TestPauseAfterCreateTableIssued);
641 
642  KuduClient();
643 
644  // Owned.
645  Data* data_;
646 
647  DISALLOW_COPY_AND_ASSIGN(KuduClient);
648 };
649 
651 class KUDU_EXPORT KuduTabletServer {
652  public:
653  ~KuduTabletServer();
654 
657  const std::string& uuid() const;
658 
661  const std::string& hostname() const;
662 
665  uint16_t port() const;
666 
673  const std::string& location() const KUDU_NO_EXPORT;
675 
676  private:
677  class KUDU_NO_EXPORT Data;
678 
679  friend class KuduClient;
680  friend class KuduScanner;
681  friend class KuduScanTokenBuilder;
682 
683  KuduTabletServer();
684 
685  // Owned.
686  Data* data_;
687 
688  DISALLOW_COPY_AND_ASSIGN(KuduTabletServer);
689 };
690 
692 class KUDU_EXPORT KuduReplica {
693  public:
694  ~KuduReplica();
695 
700  bool is_leader() const;
701 
703  const KuduTabletServer& ts() const;
704 
705  private:
706  friend class KuduClient;
707  friend class KuduScanTokenBuilder;
708  friend class internal::ReplicaController;
709 
710  class KUDU_NO_EXPORT Data;
711 
712  KuduReplica();
713 
714  // Owned.
715  Data* data_;
716 
717  DISALLOW_COPY_AND_ASSIGN(KuduReplica);
718 };
719 
721 class KUDU_EXPORT KuduTablet {
722  public:
723  ~KuduTablet();
724 
727  const std::string& id() const;
728 
734  const std::vector<const KuduReplica*>& replicas() const;
735 
736  private:
737  friend class KuduClient;
738  friend class KuduScanTokenBuilder;
739 
740  class KUDU_NO_EXPORT Data;
741 
742  KuduTablet();
743 
744  // Owned.
745  Data* data_;
746 
747  DISALLOW_COPY_AND_ASSIGN(KuduTablet);
748 };
749 
751 class KUDU_EXPORT KuduTableCreator {
752  public:
753  ~KuduTableCreator();
754 
766  KuduTableCreator& table_name(const std::string& name);
767 
778  KuduTableCreator& schema(const KuduSchema* schema);
779 
796  KuduTableCreator& add_hash_partitions(const std::vector<std::string>& columns,
797  int32_t num_buckets);
798 
814  KuduTableCreator& add_hash_partitions(const std::vector<std::string>& columns,
815  int32_t num_buckets, int32_t seed);
816 
829  KuduTableCreator& set_range_partition_columns(const std::vector<std::string>& columns);
830 
835  };
836 
865  KuduTableCreator& add_range_partition(KuduPartialRow* lower_bound,
866  KuduPartialRow* upper_bound,
867  RangePartitionBound lower_bound_type = INCLUSIVE_BOUND,
868  RangePartitionBound upper_bound_type = EXCLUSIVE_BOUND);
869 
878  KuduTableCreator& add_range_partition_split(KuduPartialRow* split_row);
879 
885  KuduTableCreator& split_rows(const std::vector<const KuduPartialRow*>& split_rows)
886  ATTRIBUTE_DEPRECATED("use add_range_partition_split() instead");
887 
897  KuduTableCreator& num_replicas(int n_replicas);
898 
908  KuduTableCreator& timeout(const MonoDelta& timeout);
909 
917  KuduTableCreator& wait(bool wait);
918 
932  Status Create();
933 
934  private:
935  class KUDU_NO_EXPORT Data;
936 
937  friend class KuduClient;
938 
939  explicit KuduTableCreator(KuduClient* client);
940 
941  // Owned.
942  Data* data_;
943 
944  DISALLOW_COPY_AND_ASSIGN(KuduTableCreator);
945 };
946 
967 class KUDU_EXPORT KuduTable : public sp::enable_shared_from_this<KuduTable> {
968  public:
969  ~KuduTable();
970 
972  const std::string& name() const;
973 
981  const std::string& id() const;
982 
984  const KuduSchema& schema() const;
985 
987  int num_replicas() const;
988 
992  KuduInsert* NewInsert();
993 
997  KuduUpsert* NewUpsert();
998 
1002  KuduUpdate* NewUpdate();
1003 
1007  KuduDelete* NewDelete();
1008 
1032  KuduPredicate* NewComparisonPredicate(const Slice& col_name,
1034  KuduValue* value);
1035 
1059  KuduPredicate* NewInListPredicate(const Slice& col_name,
1060  std::vector<KuduValue*>* values);
1061 
1072  KuduPredicate* NewIsNotNullPredicate(const Slice& col_name);
1073 
1084  KuduPredicate* NewIsNullPredicate(const Slice& col_name);
1085 
1088  KuduClient* client() const;
1089 
1091  const PartitionSchema& partition_schema() const;
1092 
1094 
1104  Status ListPartitions(std::vector<Partition>* partitions) KUDU_NO_EXPORT;
1105 
1107 
1108  private:
1109  class KUDU_NO_EXPORT Data;
1110 
1111  friend class KuduClient;
1112  friend class KuduPartitioner;
1113 
1114  KuduTable(const sp::shared_ptr<KuduClient>& client,
1115  const std::string& name,
1116  const std::string& id,
1117  int num_replicas,
1118  const KuduSchema& schema,
1119  const PartitionSchema& partition_schema);
1120 
1121  // Owned.
1122  Data* data_;
1123 
1124  DISALLOW_COPY_AND_ASSIGN(KuduTable);
1125 };
1126 
1138 class KUDU_EXPORT KuduTableAlterer {
1139  public:
1140  ~KuduTableAlterer();
1141 
1147  KuduTableAlterer* RenameTo(const std::string& new_name);
1148 
1158  KuduColumnSpec* AddColumn(const std::string& name);
1159 
1168  KuduColumnSpec* AlterColumn(const std::string& name);
1169 
1177  KuduTableAlterer* DropColumn(const std::string& name);
1178 
1210  KuduTableAlterer* AddRangePartition(
1211  KuduPartialRow* lower_bound,
1212  KuduPartialRow* upper_bound,
1215 
1243  KuduTableAlterer* DropRangePartition(
1244  KuduPartialRow* lower_bound,
1245  KuduPartialRow* upper_bound,
1248 
1258  KuduTableAlterer* timeout(const MonoDelta& timeout);
1259 
1271  KuduTableAlterer* wait(bool wait);
1272 
1274 
1283  KuduTableAlterer* modify_external_catalogs(bool modify_external_catalogs) KUDU_NO_EXPORT;
1284 
1286 
1291  Status Alter();
1292 
1293  private:
1294  class KUDU_NO_EXPORT Data;
1295 
1296  friend class KuduClient;
1297 
1298  KuduTableAlterer(KuduClient* client,
1299  const std::string& name);
1300 
1301  // Owned.
1302  Data* data_;
1303 
1304  DISALLOW_COPY_AND_ASSIGN(KuduTableAlterer);
1305 };
1306 
1312 class KUDU_EXPORT KuduError {
1313  public:
1314  ~KuduError();
1315 
1317  const Status& status() const;
1318 
1320  const KuduWriteOperation& failed_op() const;
1321 
1329  KuduWriteOperation* release_failed_op();
1330 
1341  bool was_possibly_successful() const;
1342 
1343  private:
1344  class KUDU_NO_EXPORT Data;
1345 
1346  friend class internal::Batcher;
1347  friend class internal::ErrorCollector;
1348  friend class KuduSession;
1349 
1350  KuduError(KuduWriteOperation* failed_op, const Status& error);
1351 
1352  // Owned.
1353  Data* data_;
1354 
1355  DISALLOW_COPY_AND_ASSIGN(KuduError);
1356 };
1357 
1358 
1417 class KUDU_EXPORT KuduSession : public sp::enable_shared_from_this<KuduSession> {
1418  public:
1419  ~KuduSession();
1420 
1422  enum FlushMode {
1428 
1456 
1469  MANUAL_FLUSH
1470  };
1471 
1480  Status SetFlushMode(FlushMode m) WARN_UNUSED_RESULT;
1481 
1503 
1518  COMMIT_WAIT
1519  };
1520 
1526  Status SetExternalConsistencyMode(ExternalConsistencyMode m)
1527  WARN_UNUSED_RESULT;
1528 
1546  Status SetMutationBufferSpace(size_t size_bytes) WARN_UNUSED_RESULT;
1547 
1571  Status SetMutationBufferFlushWatermark(double watermark_pct)
1572  WARN_UNUSED_RESULT;
1573 
1595  Status SetMutationBufferFlushInterval(unsigned int millis) WARN_UNUSED_RESULT;
1596 
1621  Status SetMutationBufferMaxNum(unsigned int max_num) WARN_UNUSED_RESULT;
1622 
1628  void SetTimeoutMillis(int millis);
1629 
1633 
1657  Status Apply(KuduWriteOperation* write_op) WARN_UNUSED_RESULT;
1658 
1673  Status Flush() WARN_UNUSED_RESULT;
1674 
1715  void FlushAsync(KuduStatusCallback* cb);
1716 
1719  Status Close() WARN_UNUSED_RESULT;
1720 
1729  bool HasPendingOperations() const;
1730 
1752  int CountBufferedOperations() const
1753  ATTRIBUTE_DEPRECATED("this method is experimental and will disappear "
1754  "in a future release");
1755 
1780  Status SetErrorBufferSpace(size_t size_bytes);
1781 
1791  int CountPendingErrors() const;
1792 
1803  void GetPendingErrors(std::vector<KuduError*>* errors, bool* overflowed);
1804 
1806  KuduClient* client() const;
1807 
1808  private:
1809  class KUDU_NO_EXPORT Data;
1810 
1811  friend class KuduClient;
1812  friend class internal::Batcher;
1813  friend class ClientTest;
1814  FRIEND_TEST(ClientTest, TestAutoFlushBackgroundApplyBlocks);
1815  FRIEND_TEST(ClientTest, TestAutoFlushBackgroundAndErrorCollector);
1816 
1817  explicit KuduSession(const sp::shared_ptr<KuduClient>& client);
1818 
1819  // Owned.
1820  Data* data_;
1821 
1822  DISALLOW_COPY_AND_ASSIGN(KuduSession);
1823 };
1824 
1825 
1830 class KUDU_EXPORT KuduScanner {
1831  public:
1833  enum ReadMode {
1842 
1863 
1874  READ_YOUR_WRITES
1875  };
1876 
1880  enum OrderMode {
1887 
1892  ORDERED
1893  };
1894 
1898  enum { kScanTimeoutMillis = 30000 };
1899 
1905  explicit KuduScanner(KuduTable* table);
1906  ~KuduScanner();
1907 
1917  Status SetProjectedColumnNames(const std::vector<std::string>& col_names)
1918  WARN_UNUSED_RESULT;
1919 
1929  Status SetProjectedColumnIndexes(const std::vector<int>& col_indexes)
1930  WARN_UNUSED_RESULT;
1931 
1937  Status SetProjectedColumns(const std::vector<std::string>& col_names)
1938  WARN_UNUSED_RESULT
1939  ATTRIBUTE_DEPRECATED("use SetProjectedColumnNames() instead");
1940 
1949  Status AddConjunctPredicate(KuduPredicate* pred) WARN_UNUSED_RESULT;
1950 
1959  Status AddLowerBound(const KuduPartialRow& key);
1960 
1968  Status AddLowerBoundRaw(const Slice& key)
1969  ATTRIBUTE_DEPRECATED("use AddLowerBound() instead");
1970 
1979  Status AddExclusiveUpperBound(const KuduPartialRow& key);
1980 
1988  Status AddExclusiveUpperBoundRaw(const Slice& key)
1989  ATTRIBUTE_DEPRECATED("use AddExclusiveUpperBound() instead");
1990 
1999  Status AddLowerBoundPartitionKeyRaw(const Slice& partition_key);
2000 
2009  Status AddExclusiveUpperBoundPartitionKeyRaw(const Slice& partition_key);
2010 
2017  Status SetCacheBlocks(bool cache_blocks);
2018 
2020  Status Open();
2021 
2040  Status KeepAlive();
2041 
2050  void Close();
2051 
2060  bool HasMoreRows() const;
2061 
2073  Status NextBatch(std::vector<KuduRowResult>* rows)
2074  ATTRIBUTE_DEPRECATED("use NextBatch(KuduScanBatch*) instead");
2075 
2084  Status NextBatch(KuduScanBatch* batch);
2085 
2094  Status GetCurrentServer(KuduTabletServer** server);
2095 
2097  const ResourceMetrics& GetResourceMetrics() const;
2098 
2105  Status SetBatchSizeBytes(uint32_t batch_size);
2106 
2114  Status SetSelection(KuduClient::ReplicaSelection selection)
2115  WARN_UNUSED_RESULT;
2116 
2122  Status SetReadMode(ReadMode read_mode) WARN_UNUSED_RESULT;
2123 
2129  Status SetOrderMode(OrderMode order_mode) WARN_UNUSED_RESULT
2130  ATTRIBUTE_DEPRECATED("use SetFaultTolerant() instead");
2131 
2144  Status SetFaultTolerant() WARN_UNUSED_RESULT;
2145 
2151  Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros) WARN_UNUSED_RESULT;
2152 
2165  Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT;
2166 
2172  Status SetTimeoutMillis(int millis);
2173 
2175  KuduSchema GetProjectionSchema() const;
2176 
2178  //
2184  static const uint64_t NO_FLAGS = 0;
2192  static const uint64_t PAD_UNIXTIME_MICROS_TO_16_BYTES = 1 << 0;
2193 
2221  Status SetRowFormatFlags(uint64_t flags);
2223 
2229  Status SetLimit(int64_t limit) WARN_UNUSED_RESULT;
2230 
2236  std::string ToString() const;
2237 
2238  private:
2239  class KUDU_NO_EXPORT Data;
2240 
2241  friend class KuduScanToken;
2242  FRIEND_TEST(ClientTest, TestBlockScannerHijackingAttempts);
2243  FRIEND_TEST(ClientTest, TestScanCloseProxy);
2244  FRIEND_TEST(ClientTest, TestScanFaultTolerance);
2245  FRIEND_TEST(ClientTest, TestScanNoBlockCaching);
2246  FRIEND_TEST(ClientTest, TestScanTimeout);
2247  FRIEND_TEST(ClientTest, TestReadAtSnapshotNoTimestampSet);
2248  FRIEND_TEST(ConsistencyITest, TestSnapshotScanTimestampReuse);
2249  FRIEND_TEST(ScanTokenTest, TestScanTokens);
2250 
2251  // Owned.
2252  Data* data_;
2253 
2254  DISALLOW_COPY_AND_ASSIGN(KuduScanner);
2255 };
2256 
2277 class KUDU_EXPORT KuduScanToken {
2278  public:
2279 
2280  ~KuduScanToken();
2281 
2292  Status IntoKuduScanner(KuduScanner** scanner) const WARN_UNUSED_RESULT;
2293 
2295  const KuduTablet& tablet() const;
2296 
2304  Status Serialize(std::string* buf) const WARN_UNUSED_RESULT;
2305 
2317  static Status DeserializeIntoScanner(KuduClient* client,
2318  const std::string& serialized_token,
2319  KuduScanner** scanner) WARN_UNUSED_RESULT;
2320 
2321  private:
2322  class KUDU_NO_EXPORT Data;
2323 
2324  friend class KuduScanTokenBuilder;
2325 
2326  KuduScanToken();
2327 
2328  // Owned.
2329  Data* data_;
2330 
2331  DISALLOW_COPY_AND_ASSIGN(KuduScanToken);
2332 };
2333 
2337 class KUDU_EXPORT KuduScanTokenBuilder {
2338  public:
2339 
2345  explicit KuduScanTokenBuilder(KuduTable* table);
2347 
2357  Status SetProjectedColumnNames(const std::vector<std::string>& col_names)
2358  WARN_UNUSED_RESULT;
2359 
2361  Status SetProjectedColumnIndexes(const std::vector<int>& col_indexes)
2362  WARN_UNUSED_RESULT;
2363 
2365  Status AddConjunctPredicate(KuduPredicate* pred) WARN_UNUSED_RESULT;
2366 
2368  Status AddLowerBound(const KuduPartialRow& key) WARN_UNUSED_RESULT;
2369 
2378  Status AddUpperBound(const KuduPartialRow& key) WARN_UNUSED_RESULT;
2379 
2381  Status SetCacheBlocks(bool cache_blocks) WARN_UNUSED_RESULT;
2382 
2389  Status SetBatchSizeBytes(uint32_t batch_size) WARN_UNUSED_RESULT;
2390 
2398  Status SetSelection(KuduClient::ReplicaSelection selection)
2399  WARN_UNUSED_RESULT;
2400 
2402  Status SetReadMode(KuduScanner::ReadMode read_mode) WARN_UNUSED_RESULT;
2403 
2405  Status SetFaultTolerant() WARN_UNUSED_RESULT;
2406 
2408  Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros)
2409  WARN_UNUSED_RESULT;
2410 
2412  Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT;
2413 
2415  Status SetTimeoutMillis(int millis) WARN_UNUSED_RESULT;
2416 
2425  Status Build(std::vector<KuduScanToken*>* tokens) WARN_UNUSED_RESULT;
2426 
2427  private:
2428  class KUDU_NO_EXPORT Data;
2429 
2430  // Owned.
2431  Data* data_;
2432 
2433  DISALLOW_COPY_AND_ASSIGN(KuduScanTokenBuilder);
2434 };
2435 
2437 class KUDU_EXPORT KuduPartitionerBuilder {
2438  public:
2443  explicit KuduPartitionerBuilder(sp::shared_ptr<KuduTable> table);
2445 
2451  KuduPartitionerBuilder* SetBuildTimeout(MonoDelta timeout);
2452 
2471  Status Build(KuduPartitioner** partitioner);
2472  private:
2473  class KUDU_NO_EXPORT Data;
2474 
2475  // Owned.
2476  Data* data_;
2477 
2478  DISALLOW_COPY_AND_ASSIGN(KuduPartitionerBuilder);
2479 };
2480 
2491 class KUDU_EXPORT KuduPartitioner {
2492  public:
2493  ~KuduPartitioner();
2494 
2498  int NumPartitions() const;
2499 
2511  Status PartitionRow(const KuduPartialRow& row, int* partition);
2512  private:
2513  class KUDU_NO_EXPORT Data;
2514 
2515  friend class KuduPartitionerBuilder;
2516 
2517  explicit KuduPartitioner(Data* data);
2518  Data* data_; // Owned.
2519 };
2520 
2521 
2522 } // namespace client
2523 } // namespace kudu
2524 #endif
The interface for all logging callbacks.
Definition: callbacks.h:44
A single row update to be sent to the cluster.
Definition: write_op.h:193
A representation of a table&#39;s schema.
Definition: schema.h:512
A representation of an operation&#39;s outcome.
Definition: status.h:145
A constant cell value with a specific type.
Definition: value.h:35
Builder for Partitioner instances.
Definition: client.h:2437
ExternalConsistencyMode
The possible external consistency modes on which Kudu operates.
Definition: client.h:1483
Definition: callbacks.h:28
A single row insert to be sent to the cluster.
Definition: write_op.h:141
Builder API for specifying or altering a column within a table schema.
Definition: schema.h:283
static const uint64_t kNoTimestamp
Definition: client.h:494
ReplicaSelection
Policy with which to choose amongst multiple replicas.
Definition: client.h:470
A single row upsert to be sent to the cluster.
Definition: write_op.h:166
Builds scan tokens for a table.
Definition: client.h:2337
ComparisonOp
Supported comparison operators.
Definition: scan_predicate.h:37
Alters an existing table based on the provided steps.
Definition: client.h:1138
OrderMode
Definition: client.h:1880
Definition: client.h:1886
Smart pointer typedefs for externally-faced code.
Definition: client.h:2491
A representation of comparison predicate for Kudu queries.
Definition: scan_predicate.h:34
An exclusive bound.
Definition: client.h:833
This class represents an error which occurred in a write operation.
Definition: client.h:1312
A handle for a connection to a cluster.
Definition: client.h:316
An inclusive bound.
Definition: client.h:834
In-memory representation of a remote tablet server.
Definition: client.h:651
The interface for all status callbacks.
Definition: callbacks.h:161
A wrapper around externally allocated data.
Definition: slice.h:50
A representation of a table on a particular cluster.
Definition: client.h:967
This class is a representation of a single scan.
Definition: client.h:1830
ReadMode
The read modes for scanners.
Definition: client.h:1833
A "factory" for KuduClient objects.
Definition: client.h:201
A single-row write operation to be sent to a Kudu table.
Definition: write_op.h:65
Select the LEADER replica.
Definition: client.h:471
RangePartitionBound
Range partition bound type.
Definition: client.h:832
In-memory representation of a remote tablet.
Definition: client.h:721
In-memory representation of a remote tablet&#39;s replica.
Definition: client.h:692
FlushMode
Modes of flush operations.
Definition: client.h:1422
A scan descriptor limited to a single physical contiguous location.
Definition: client.h:2277
A single row delete to be sent to the cluster.
Definition: write_op.h:220
A generic catalog of simple metrics.
Definition: resource_metrics.h:33
A helper class to create a new table with the desired options.
Definition: client.h:751
A row which may only contain values for a subset of the columns.
Definition: partial_row.h:64
Representation of a Kudu client session.
Definition: client.h:1417
A batch of zero or more rows returned by a scan operation.
Definition: scan_batch.h:83
A representation of a time interval.
Definition: monotime.h:44