Kudu C++ client API
 All Classes Files Functions Variables Typedefs Enumerations Enumerator Friends Macros Pages
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 #include "kudu/gutil/macros.h"
41 #include "kudu/gutil/port.h"
42 #else
43 #include "kudu/client/stubs.h"
44 #endif
45 #include "kudu/util/kudu_export.h"
46 #include "kudu/util/slice.h"
47 #include "kudu/util/status.h"
48 
49 namespace kudu {
50 
51 class ClientStressTest_TestUniqueClientIds_Test;
52 class KuduPartialRow;
53 class MonoDelta;
54 class PartitionSchema;
55 class SecurityUnknownTskTest;
56 
57 namespace tools {
58 class LeaderMasterProxy;
59 } // namespace tools
60 
61 namespace client {
62 
63 class KuduClient;
64 class KuduDelete;
65 class KuduInsert;
66 class KuduLoggingCallback;
67 class KuduPartitioner;
68 class KuduScanBatch;
69 class KuduSession;
70 class KuduStatusCallback;
71 class KuduTable;
72 class KuduTableAlterer;
73 class KuduTableCreator;
74 class KuduTablet;
75 class KuduTabletServer;
76 class KuduUpdate;
77 class KuduUpsert;
78 class KuduValue;
79 class KuduWriteOperation;
80 class ResourceMetrics;
81 
82 namespace internal {
83 class Batcher;
84 class ErrorCollector;
85 class GetTableSchemaRpc;
86 class LookupRpc;
87 class MetaCache;
88 class RemoteTablet;
89 class RemoteTabletServer;
90 class ReplicaController;
91 class WriteRpc;
92 } // namespace internal
93 
106 void KUDU_EXPORT InstallLoggingCallback(KuduLoggingCallback* cb);
107 
114 void KUDU_EXPORT UninstallLoggingCallback();
115 
131 void KUDU_EXPORT SetVerboseLogLevel(int level);
132 
142 Status KUDU_EXPORT SetInternalSignalNumber(int signum);
143 
155 Status KUDU_EXPORT DisableSaslInitialization();
156 
157 
180 Status KUDU_EXPORT DisableOpenSSLInitialization();
181 
184 std::string KUDU_EXPORT GetShortVersionString();
185 
188 std::string KUDU_EXPORT GetAllVersionInfo();
189 
194 class KUDU_EXPORT KuduClientBuilder {
195  public:
198 
202  KuduClientBuilder& clear_master_server_addrs();
203 
209  KuduClientBuilder& master_server_addrs(const std::vector<std::string>& addrs);
210 
218  KuduClientBuilder& add_master_server_addr(const std::string& addr);
219 
229  KuduClientBuilder& default_admin_operation_timeout(const MonoDelta& timeout);
230 
238  KuduClientBuilder& default_rpc_timeout(const MonoDelta& timeout);
239 
247  KuduClientBuilder& import_authentication_credentials(std::string authn_creds);
248 
259  Status Build(sp::shared_ptr<KuduClient>* client);
260 
261  private:
262  class KUDU_NO_EXPORT Data;
263 
264  friend class internal::ReplicaController;
265 
266  // Owned.
267  Data* data_;
268 
269  DISALLOW_COPY_AND_ASSIGN(KuduClientBuilder);
270 };
271 
298 class KUDU_EXPORT KuduClient : public sp::enable_shared_from_this<KuduClient> {
299  public:
300  ~KuduClient();
301 
306  KuduTableCreator* NewTableCreator();
307 
316  Status IsCreateTableInProgress(const std::string& table_name,
317  bool* create_in_progress);
318 
324  Status DeleteTable(const std::string& table_name);
325 
332  KuduTableAlterer* NewTableAlterer(const std::string& table_name);
333 
342  Status IsAlterTableInProgress(const std::string& table_name,
343  bool* alter_in_progress);
351  Status GetTableSchema(const std::string& table_name,
352  KuduSchema* schema);
353 
360  Status ListTabletServers(std::vector<KuduTabletServer*>* tablet_servers);
361 
369  Status ListTables(std::vector<std::string>* tables,
370  const std::string& filter = "");
371 
379  Status TableExists(const std::string& table_name, bool* exists);
380 
397  Status OpenTable(const std::string& table_name,
398  sp::shared_ptr<KuduTable>* table);
399 
405  sp::shared_ptr<KuduSession> NewSession();
406 
408 
421  Status KUDU_NO_EXPORT GetTablet(const std::string& tablet_id,
422  KuduTablet** tablet);
423 
425 
429 
430  CLOSEST_REPLICA,
431 
433  FIRST_REPLICA
434  };
435 
438  bool IsMultiMaster() const;
439 
441  const MonoDelta& default_admin_operation_timeout() const;
442 
444  const MonoDelta& default_rpc_timeout() const;
445 
448  static const uint64_t kNoTimestamp;
449 
489  uint64_t GetLatestObservedTimestamp() const;
490 
505  void SetLatestObservedTimestamp(uint64_t ht_timestamp);
506 
518  Status ExportAuthenticationCredentials(std::string* authn_creds) const;
519 
520  private:
521  class KUDU_NO_EXPORT Data;
522 
523  friend class internal::Batcher;
524  friend class internal::GetTableSchemaRpc;
525  friend class internal::LookupRpc;
526  friend class internal::MetaCache;
527  friend class internal::RemoteTablet;
528  friend class internal::RemoteTabletServer;
529  friend class internal::WriteRpc;
530  friend class ConnectToClusterBaseTest;
531  friend class ClientTest;
532  friend class KuduClientBuilder;
533  friend class KuduPartitionerBuilder;
534  friend class KuduScanner;
535  friend class KuduScanToken;
536  friend class KuduScanTokenBuilder;
537  friend class KuduSession;
538  friend class KuduTable;
539  friend class KuduTableAlterer;
540  friend class KuduTableCreator;
541  friend class ::kudu::SecurityUnknownTskTest;
542  friend class tools::LeaderMasterProxy;
543 
544  FRIEND_TEST(kudu::ClientStressTest, TestUniqueClientIds);
545  FRIEND_TEST(ClientTest, TestGetSecurityInfoFromMaster);
546  FRIEND_TEST(ClientTest, TestGetTabletServerBlacklist);
547  FRIEND_TEST(ClientTest, TestMasterDown);
548  FRIEND_TEST(ClientTest, TestMasterLookupPermits);
549  FRIEND_TEST(ClientTest, TestMetaCacheExpiry);
550  FRIEND_TEST(ClientTest, TestNonCoveringRangePartitions);
551  FRIEND_TEST(ClientTest, TestReplicatedTabletWritesWithLeaderElection);
552  FRIEND_TEST(ClientTest, TestScanFaultTolerance);
553  FRIEND_TEST(ClientTest, TestScanTimeout);
554  FRIEND_TEST(ClientTest, TestWriteWithDeadMaster);
555  FRIEND_TEST(MasterFailoverTest, TestPauseAfterCreateTableIssued);
556 
557  KuduClient();
558 
559  // Owned.
560  Data* data_;
561 
562  DISALLOW_COPY_AND_ASSIGN(KuduClient);
563 };
564 
566 class KUDU_EXPORT KuduTabletServer {
567  public:
568  ~KuduTabletServer();
569 
572  const std::string& uuid() const;
573 
576  const std::string& hostname() const;
577 
580  uint16_t port() const;
581 
582  private:
583  class KUDU_NO_EXPORT Data;
584 
585  friend class KuduClient;
586  friend class KuduScanner;
587  friend class KuduScanTokenBuilder;
588 
590 
591  // Owned.
592  Data* data_;
593 
594  DISALLOW_COPY_AND_ASSIGN(KuduTabletServer);
595 };
596 
598 class KUDU_EXPORT KuduReplica {
599  public:
600  ~KuduReplica();
601 
606  bool is_leader() const;
607 
609  const KuduTabletServer& ts() const;
610 
611  private:
612  friend class KuduClient;
613  friend class KuduScanTokenBuilder;
614  friend class internal::ReplicaController;
615 
616  class KUDU_NO_EXPORT Data;
617 
618  KuduReplica();
619 
620  // Owned.
621  Data* data_;
622 
623  DISALLOW_COPY_AND_ASSIGN(KuduReplica);
624 };
625 
627 class KUDU_EXPORT KuduTablet {
628  public:
629  ~KuduTablet();
630 
633  const std::string& id() const;
634 
640  const std::vector<const KuduReplica*>& replicas() const;
641 
642  private:
643  friend class KuduClient;
644  friend class KuduScanTokenBuilder;
645 
646  class KUDU_NO_EXPORT Data;
647 
648  KuduTablet();
649 
650  // Owned.
651  Data* data_;
652 
653  DISALLOW_COPY_AND_ASSIGN(KuduTablet);
654 };
655 
657 class KUDU_EXPORT KuduTableCreator {
658  public:
659  ~KuduTableCreator();
660 
672  KuduTableCreator& table_name(const std::string& name);
673 
684  KuduTableCreator& schema(const KuduSchema* schema);
685 
702  KuduTableCreator& add_hash_partitions(const std::vector<std::string>& columns,
703  int32_t num_buckets);
704 
720  KuduTableCreator& add_hash_partitions(const std::vector<std::string>& columns,
721  int32_t num_buckets, int32_t seed);
722 
735  KuduTableCreator& set_range_partition_columns(const std::vector<std::string>& columns);
736 
741  };
742 
771  KuduTableCreator& add_range_partition(KuduPartialRow* lower_bound,
772  KuduPartialRow* upper_bound,
773  RangePartitionBound lower_bound_type = INCLUSIVE_BOUND,
774  RangePartitionBound upper_bound_type = EXCLUSIVE_BOUND);
775 
784  KuduTableCreator& add_range_partition_split(KuduPartialRow* split_row);
785 
791  KuduTableCreator& split_rows(const std::vector<const KuduPartialRow*>& split_rows)
792  ATTRIBUTE_DEPRECATED("use add_range_partition_split() instead");
793 
803  KuduTableCreator& num_replicas(int n_replicas);
804 
814  KuduTableCreator& timeout(const MonoDelta& timeout);
815 
823  KuduTableCreator& wait(bool wait);
824 
838  Status Create();
839 
840  private:
841  class KUDU_NO_EXPORT Data;
842 
843  friend class KuduClient;
844 
845  explicit KuduTableCreator(KuduClient* client);
846 
847  // Owned.
848  Data* data_;
849 
850  DISALLOW_COPY_AND_ASSIGN(KuduTableCreator);
851 };
852 
873 class KUDU_EXPORT KuduTable : public sp::enable_shared_from_this<KuduTable> {
874  public:
875  ~KuduTable();
876 
878  const std::string& name() const;
879 
887  const std::string& id() const;
888 
890  const KuduSchema& schema() const;
891 
893  int num_replicas() const;
894 
898  KuduInsert* NewInsert();
899 
903  KuduUpsert* NewUpsert();
904 
908  KuduUpdate* NewUpdate();
909 
913  KuduDelete* NewDelete();
914 
938  KuduPredicate* NewComparisonPredicate(const Slice& col_name,
940  KuduValue* value);
941 
965  KuduPredicate* NewInListPredicate(const Slice& col_name,
966  std::vector<KuduValue*>* values);
967 
978  KuduPredicate* NewIsNotNullPredicate(const Slice& col_name);
979 
990  KuduPredicate* NewIsNullPredicate(const Slice& col_name);
991 
994  KuduClient* client() const;
995 
997  const PartitionSchema& partition_schema() const;
998 
999  private:
1000  class KUDU_NO_EXPORT Data;
1001 
1002  friend class KuduClient;
1003  friend class KuduPartitioner;
1004 
1005  KuduTable(const sp::shared_ptr<KuduClient>& client,
1006  const std::string& name,
1007  const std::string& id,
1008  int num_replicas,
1009  const KuduSchema& schema,
1010  const PartitionSchema& partition_schema);
1011 
1012  // Owned.
1013  Data* data_;
1014 
1015  DISALLOW_COPY_AND_ASSIGN(KuduTable);
1016 };
1017 
1029 class KUDU_EXPORT KuduTableAlterer {
1030  public:
1031  ~KuduTableAlterer();
1032 
1038  KuduTableAlterer* RenameTo(const std::string& new_name);
1039 
1049  KuduColumnSpec* AddColumn(const std::string& name);
1050 
1059  KuduColumnSpec* AlterColumn(const std::string& name);
1060 
1068  KuduTableAlterer* DropColumn(const std::string& name);
1069 
1101  KuduTableAlterer* AddRangePartition(
1102  KuduPartialRow* lower_bound,
1103  KuduPartialRow* upper_bound,
1106 
1134  KuduTableAlterer* DropRangePartition(
1135  KuduPartialRow* lower_bound,
1136  KuduPartialRow* upper_bound,
1139 
1149  KuduTableAlterer* timeout(const MonoDelta& timeout);
1150 
1162  KuduTableAlterer* wait(bool wait);
1163 
1168  Status Alter();
1169 
1170  private:
1171  class KUDU_NO_EXPORT Data;
1172  friend class KuduClient;
1173 
1174  KuduTableAlterer(KuduClient* client,
1175  const std::string& name);
1176 
1177  // Owned.
1178  Data* data_;
1179 
1180  DISALLOW_COPY_AND_ASSIGN(KuduTableAlterer);
1181 };
1182 
1188 class KUDU_EXPORT KuduError {
1189  public:
1190  ~KuduError();
1191 
1193  const Status& status() const;
1194 
1196  const KuduWriteOperation& failed_op() const;
1197 
1205  KuduWriteOperation* release_failed_op();
1206 
1217  bool was_possibly_successful() const;
1218 
1219  private:
1220  class KUDU_NO_EXPORT Data;
1221 
1222  friend class internal::Batcher;
1223  friend class internal::ErrorCollector;
1224  friend class KuduSession;
1225 
1226  KuduError(KuduWriteOperation* failed_op, const Status& error);
1227 
1228  // Owned.
1229  Data* data_;
1230 
1231  DISALLOW_COPY_AND_ASSIGN(KuduError);
1232 };
1233 
1234 
1293 class KUDU_EXPORT KuduSession : public sp::enable_shared_from_this<KuduSession> {
1294  public:
1295  ~KuduSession();
1296 
1298  enum FlushMode {
1304 
1332 
1345  MANUAL_FLUSH
1346  };
1347 
1356  Status SetFlushMode(FlushMode m) WARN_UNUSED_RESULT;
1357 
1379 
1394  COMMIT_WAIT
1395  };
1396 
1402  Status SetExternalConsistencyMode(ExternalConsistencyMode m)
1403  WARN_UNUSED_RESULT;
1404 
1422  Status SetMutationBufferSpace(size_t size_bytes) WARN_UNUSED_RESULT;
1423 
1447  Status SetMutationBufferFlushWatermark(double watermark_pct)
1448  WARN_UNUSED_RESULT;
1449 
1471  Status SetMutationBufferFlushInterval(unsigned int millis) WARN_UNUSED_RESULT;
1472 
1497  Status SetMutationBufferMaxNum(unsigned int max_num) WARN_UNUSED_RESULT;
1498 
1504  void SetTimeoutMillis(int millis);
1505 
1509 
1533  Status Apply(KuduWriteOperation* write_op) WARN_UNUSED_RESULT;
1534 
1549  Status Flush() WARN_UNUSED_RESULT;
1550 
1591  void FlushAsync(KuduStatusCallback* cb);
1592 
1595  Status Close() WARN_UNUSED_RESULT;
1596 
1605  bool HasPendingOperations() const;
1606 
1628  int CountBufferedOperations() const
1629  ATTRIBUTE_DEPRECATED("this method is experimental and will disappear "
1630  "in a future release");
1631 
1656  Status SetErrorBufferSpace(size_t size_bytes);
1657 
1667  int CountPendingErrors() const;
1668 
1679  void GetPendingErrors(std::vector<KuduError*>* errors, bool* overflowed);
1680 
1682  KuduClient* client() const;
1683 
1684  private:
1685  class KUDU_NO_EXPORT Data;
1686 
1687  friend class KuduClient;
1688  friend class internal::Batcher;
1689  friend class ClientTest;
1690  FRIEND_TEST(ClientTest, TestAutoFlushBackgroundApplyBlocks);
1691  FRIEND_TEST(ClientTest, TestAutoFlushBackgroundAndErrorCollector);
1692 
1693  explicit KuduSession(const sp::shared_ptr<KuduClient>& client);
1694 
1695  // Owned.
1696  Data* data_;
1697 
1698  DISALLOW_COPY_AND_ASSIGN(KuduSession);
1699 };
1700 
1701 
1706 class KUDU_EXPORT KuduScanner {
1707  public:
1709  enum ReadMode {
1718 
1739 
1750  READ_YOUR_WRITES
1751  };
1752 
1756  enum OrderMode {
1763 
1768  ORDERED
1769  };
1770 
1774  enum { kScanTimeoutMillis = 30000 };
1775 
1781  explicit KuduScanner(KuduTable* table);
1782  ~KuduScanner();
1783 
1793  Status SetProjectedColumnNames(const std::vector<std::string>& col_names)
1794  WARN_UNUSED_RESULT;
1795 
1805  Status SetProjectedColumnIndexes(const std::vector<int>& col_indexes)
1806  WARN_UNUSED_RESULT;
1807 
1813  Status SetProjectedColumns(const std::vector<std::string>& col_names)
1814  WARN_UNUSED_RESULT
1815  ATTRIBUTE_DEPRECATED("use SetProjectedColumnNames() instead");
1816 
1825  Status AddConjunctPredicate(KuduPredicate* pred) WARN_UNUSED_RESULT;
1826 
1835  Status AddLowerBound(const KuduPartialRow& key);
1836 
1844  Status AddLowerBoundRaw(const Slice& key)
1845  ATTRIBUTE_DEPRECATED("use AddLowerBound() instead");
1846 
1855  Status AddExclusiveUpperBound(const KuduPartialRow& key);
1856 
1864  Status AddExclusiveUpperBoundRaw(const Slice& key)
1865  ATTRIBUTE_DEPRECATED("use AddExclusiveUpperBound() instead");
1866 
1875  Status AddLowerBoundPartitionKeyRaw(const Slice& partition_key);
1876 
1885  Status AddExclusiveUpperBoundPartitionKeyRaw(const Slice& partition_key);
1886 
1893  Status SetCacheBlocks(bool cache_blocks);
1894 
1896  Status Open();
1897 
1913  Status KeepAlive();
1914 
1923  void Close();
1924 
1933  bool HasMoreRows() const;
1934 
1946  Status NextBatch(std::vector<KuduRowResult>* rows)
1947  ATTRIBUTE_DEPRECATED("use NextBatch(KuduScanBatch*) instead");
1948 
1957  Status NextBatch(KuduScanBatch* batch);
1958 
1967  Status GetCurrentServer(KuduTabletServer** server);
1968 
1970  const ResourceMetrics& GetResourceMetrics() const;
1971 
1978  Status SetBatchSizeBytes(uint32_t batch_size);
1979 
1987  Status SetSelection(KuduClient::ReplicaSelection selection)
1988  WARN_UNUSED_RESULT;
1989 
1995  Status SetReadMode(ReadMode read_mode) WARN_UNUSED_RESULT;
1996 
2002  Status SetOrderMode(OrderMode order_mode) WARN_UNUSED_RESULT
2003  ATTRIBUTE_DEPRECATED("use SetFaultTolerant() instead");
2004 
2017  Status SetFaultTolerant() WARN_UNUSED_RESULT;
2018 
2024  Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros) WARN_UNUSED_RESULT;
2025 
2038  Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT;
2039 
2045  Status SetTimeoutMillis(int millis);
2046 
2048  KuduSchema GetProjectionSchema() const;
2049 
2051  //
2057  static const uint64_t NO_FLAGS = 0;
2065  static const uint64_t PAD_UNIXTIME_MICROS_TO_16_BYTES = 1 << 0;
2089  Status SetRowFormatFlags(uint64_t flags);
2091 
2097  std::string ToString() const;
2098 
2099  private:
2100  class KUDU_NO_EXPORT Data;
2101 
2102  friend class KuduScanToken;
2103  FRIEND_TEST(ClientTest, TestScanCloseProxy);
2104  FRIEND_TEST(ClientTest, TestScanFaultTolerance);
2105  FRIEND_TEST(ClientTest, TestScanNoBlockCaching);
2106  FRIEND_TEST(ClientTest, TestScanTimeout);
2107  FRIEND_TEST(ClientTest, TestReadAtSnapshotNoTimestampSet);
2108  FRIEND_TEST(ConsistencyITest, TestSnapshotScanTimestampReuse);
2109  FRIEND_TEST(ScanTokenTest, TestScanTokens);
2110 
2111  // Owned.
2112  Data* data_;
2113 
2114  DISALLOW_COPY_AND_ASSIGN(KuduScanner);
2115 };
2116 
2137 class KUDU_EXPORT KuduScanToken {
2138  public:
2139 
2140  ~KuduScanToken();
2141 
2152  Status IntoKuduScanner(KuduScanner** scanner) const WARN_UNUSED_RESULT;
2153 
2155  const KuduTablet& tablet() const;
2156 
2164  Status Serialize(std::string* buf) const WARN_UNUSED_RESULT;
2165 
2177  static Status DeserializeIntoScanner(KuduClient* client,
2178  const std::string& serialized_token,
2179  KuduScanner** scanner) WARN_UNUSED_RESULT;
2180 
2181  private:
2182  class KUDU_NO_EXPORT Data;
2183 
2184  friend class KuduScanTokenBuilder;
2185 
2186  KuduScanToken();
2187 
2188  // Owned.
2189  Data* data_;
2190 
2191  DISALLOW_COPY_AND_ASSIGN(KuduScanToken);
2192 };
2193 
2197 class KUDU_EXPORT KuduScanTokenBuilder {
2198  public:
2199 
2205  explicit KuduScanTokenBuilder(KuduTable* table);
2207 
2217  Status SetProjectedColumnNames(const std::vector<std::string>& col_names)
2218  WARN_UNUSED_RESULT;
2219 
2221  Status SetProjectedColumnIndexes(const std::vector<int>& col_indexes)
2222  WARN_UNUSED_RESULT;
2223 
2225  Status AddConjunctPredicate(KuduPredicate* pred) WARN_UNUSED_RESULT;
2226 
2228  Status AddLowerBound(const KuduPartialRow& key) WARN_UNUSED_RESULT;
2229 
2238  Status AddUpperBound(const KuduPartialRow& key) WARN_UNUSED_RESULT;
2239 
2241  Status SetCacheBlocks(bool cache_blocks) WARN_UNUSED_RESULT;
2242 
2249  Status SetBatchSizeBytes(uint32_t batch_size) WARN_UNUSED_RESULT;
2250 
2258  Status SetSelection(KuduClient::ReplicaSelection selection)
2259  WARN_UNUSED_RESULT;
2260 
2262  Status SetReadMode(KuduScanner::ReadMode read_mode) WARN_UNUSED_RESULT;
2263 
2265  Status SetFaultTolerant() WARN_UNUSED_RESULT;
2266 
2268  Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros)
2269  WARN_UNUSED_RESULT;
2270 
2272  Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT;
2273 
2275  Status SetTimeoutMillis(int millis) WARN_UNUSED_RESULT;
2276 
2285  Status Build(std::vector<KuduScanToken*>* tokens) WARN_UNUSED_RESULT;
2286 
2287  private:
2288  class KUDU_NO_EXPORT Data;
2289 
2290  // Owned.
2291  Data* data_;
2292 
2293  DISALLOW_COPY_AND_ASSIGN(KuduScanTokenBuilder);
2294 };
2295 
2297 class KUDU_EXPORT KuduPartitionerBuilder {
2298  public:
2303  explicit KuduPartitionerBuilder(sp::shared_ptr<KuduTable> table);
2305 
2307  KuduPartitionerBuilder* SetBuildTimeout(MonoDelta timeout);
2308 
2325  Status Build(KuduPartitioner** partitioner);
2326  private:
2327  class KUDU_NO_EXPORT Data;
2328 
2329  // Owned.
2330  Data* data_;
2331 
2332  DISALLOW_COPY_AND_ASSIGN(KuduPartitionerBuilder);
2333 };
2334 
2345 class KUDU_EXPORT KuduPartitioner {
2346  public:
2347  ~KuduPartitioner();
2348 
2352  int NumPartitions() const;
2353 
2365  Status PartitionRow(const KuduPartialRow& row, int* partition);
2366  private:
2367  class KUDU_NO_EXPORT Data;
2368  friend class KuduPartitionerBuilder;
2369 
2370  explicit KuduPartitioner(Data* data);
2371  Data* data_; // Owned.
2372 };
2373 
2374 
2375 } // namespace client
2376 } // namespace kudu
2377 #endif
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:497
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:2297
ExternalConsistencyMode
The possible external consistency modes on which Kudu operates.
Definition: client.h:1359
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:272
static const uint64_t kNoTimestamp
Definition: client.h:448
ReplicaSelection
Policy with which to choose amongst multiple replicas.
Definition: client.h:427
A single row upsert to be sent to the cluster.
Definition: write_op.h:166
Builds scan tokens for a table.
Definition: client.h:2197
ComparisonOp
Supported comparison operators.
Definition: scan_predicate.h:37
Alters an existing table based on the provided steps.
Definition: client.h:1029
OrderMode
Definition: client.h:1756
Definition: client.h:1762
Smart pointer typedefs for externally-faced code.
Definition: client.h:2345
A representation of comparison predicate for Kudu queries.
Definition: scan_predicate.h:34
An exclusive bound.
Definition: client.h:739
This class represents an error which occurred in a write operation.
Definition: client.h:1188
A handle for a connection to a cluster.
Definition: client.h:298
An inclusive bound.
Definition: client.h:740
In-memory representation of a remote tablet server.
Definition: client.h:566
The interface for all status callbacks.
Definition: callbacks.h:161
A wrapper around externally allocated data.
Definition: slice.h:47
A representation of a table on a particular cluster.
Definition: client.h:873
This class is a representation of a single scan.
Definition: client.h:1706
ReadMode
The read modes for scanners.
Definition: client.h:1709
A &quot;factory&quot; for KuduClient objects.
Definition: client.h:194
A single-row write operation to be sent to a Kudu table.
Definition: write_op.h:65
Select the LEADER replica.
Definition: client.h:428
RangePartitionBound
Range partition bound type.
Definition: client.h:738
In-memory representation of a remote tablet.
Definition: client.h:627
In-memory representation of a remote tablet&#39;s replica.
Definition: client.h:598
FlushMode
Modes of flush operations.
Definition: client.h:1298
A scan descriptor limited to a single physical contiguous location.
Definition: client.h:2137
A single row delete to be sent to the cluster.
Definition: write_op.h:220
A helper class to create a new table with the desired options.
Definition: client.h:657
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:1293
A representation of a time interval.
Definition: monotime.h:44