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 WriteRpc;
91 } // namespace internal
92 
105 void KUDU_EXPORT InstallLoggingCallback(KuduLoggingCallback* cb);
106 
113 void KUDU_EXPORT UninstallLoggingCallback();
114 
130 void KUDU_EXPORT SetVerboseLogLevel(int level);
131 
141 Status KUDU_EXPORT SetInternalSignalNumber(int signum);
142 
154 Status KUDU_EXPORT DisableSaslInitialization();
155 
156 
179 Status KUDU_EXPORT DisableOpenSSLInitialization();
180 
183 std::string KUDU_EXPORT GetShortVersionString();
184 
187 std::string KUDU_EXPORT GetAllVersionInfo();
188 
193 class KUDU_EXPORT KuduClientBuilder {
194  public:
197 
201  KuduClientBuilder& clear_master_server_addrs();
202 
208  KuduClientBuilder& master_server_addrs(const std::vector<std::string>& addrs);
209 
217  KuduClientBuilder& add_master_server_addr(const std::string& addr);
218 
228  KuduClientBuilder& default_admin_operation_timeout(const MonoDelta& timeout);
229 
237  KuduClientBuilder& default_rpc_timeout(const MonoDelta& timeout);
238 
246  KuduClientBuilder& import_authentication_credentials(std::string authn_creds);
247 
258  Status Build(sp::shared_ptr<KuduClient>* client);
259 
260  private:
261  class KUDU_NO_EXPORT Data;
262 
263  // Owned.
264  Data* data_;
265 
266  DISALLOW_COPY_AND_ASSIGN(KuduClientBuilder);
267 };
268 
295 class KUDU_EXPORT KuduClient : public sp::enable_shared_from_this<KuduClient> {
296  public:
297  ~KuduClient();
298 
303  KuduTableCreator* NewTableCreator();
304 
313  Status IsCreateTableInProgress(const std::string& table_name,
314  bool *create_in_progress);
315 
321  Status DeleteTable(const std::string& table_name);
322 
329  KuduTableAlterer* NewTableAlterer(const std::string& table_name);
330 
339  Status IsAlterTableInProgress(const std::string& table_name,
340  bool *alter_in_progress);
348  Status GetTableSchema(const std::string& table_name,
349  KuduSchema* schema);
350 
357  Status ListTabletServers(std::vector<KuduTabletServer*>* tablet_servers);
358 
366  Status ListTables(std::vector<std::string>* tables,
367  const std::string& filter = "");
368 
376  Status TableExists(const std::string& table_name, bool* exists);
377 
394  Status OpenTable(const std::string& table_name,
395  sp::shared_ptr<KuduTable>* table);
396 
402  sp::shared_ptr<KuduSession> NewSession();
403 
405 
418  Status KUDU_NO_EXPORT GetTablet(const std::string& tablet_id,
419  KuduTablet** tablet);
420 
422 
426 
427  CLOSEST_REPLICA,
428 
430  FIRST_REPLICA
431  };
432 
435  bool IsMultiMaster() const;
436 
438  const MonoDelta& default_admin_operation_timeout() const;
439 
441  const MonoDelta& default_rpc_timeout() const;
442 
445  static const uint64_t kNoTimestamp;
446 
486  uint64_t GetLatestObservedTimestamp() const;
487 
502  void SetLatestObservedTimestamp(uint64_t ht_timestamp);
503 
515  Status ExportAuthenticationCredentials(std::string* authn_creds) const;
516 
517  private:
518  class KUDU_NO_EXPORT Data;
519 
520  friend class internal::Batcher;
521  friend class internal::GetTableSchemaRpc;
522  friend class internal::LookupRpc;
523  friend class internal::MetaCache;
524  friend class internal::RemoteTablet;
525  friend class internal::RemoteTabletServer;
526  friend class internal::WriteRpc;
527  friend class ClientTest;
528  friend class KuduClientBuilder;
529  friend class KuduPartitionerBuilder;
530  friend class KuduScanner;
531  friend class KuduScanToken;
532  friend class KuduScanTokenBuilder;
533  friend class KuduSession;
534  friend class KuduTable;
535  friend class KuduTableAlterer;
536  friend class KuduTableCreator;
537  friend class ::kudu::SecurityUnknownTskTest;
538  friend class tools::LeaderMasterProxy;
539 
540  FRIEND_TEST(kudu::ClientStressTest, TestUniqueClientIds);
541  FRIEND_TEST(ClientTest, TestGetSecurityInfoFromMaster);
542  FRIEND_TEST(ClientTest, TestGetTabletServerBlacklist);
543  FRIEND_TEST(ClientTest, TestMasterDown);
544  FRIEND_TEST(ClientTest, TestMasterLookupPermits);
545  FRIEND_TEST(ClientTest, TestMetaCacheExpiry);
546  FRIEND_TEST(ClientTest, TestNonCoveringRangePartitions);
547  FRIEND_TEST(ClientTest, TestReplicatedTabletWritesWithLeaderElection);
548  FRIEND_TEST(ClientTest, TestScanFaultTolerance);
549  FRIEND_TEST(ClientTest, TestScanTimeout);
550  FRIEND_TEST(ClientTest, TestWriteWithDeadMaster);
551  FRIEND_TEST(MasterFailoverTest, TestPauseAfterCreateTableIssued);
552 
553  KuduClient();
554 
555  // Owned.
556  Data* data_;
557 
558  DISALLOW_COPY_AND_ASSIGN(KuduClient);
559 };
560 
562 class KUDU_EXPORT KuduTabletServer {
563  public:
564  ~KuduTabletServer();
565 
568  const std::string& uuid() const;
569 
572  const std::string& hostname() const;
573 
576  uint16_t port() const;
577 
578  private:
579  class KUDU_NO_EXPORT Data;
580 
581  friend class KuduClient;
582  friend class KuduScanner;
583  friend class KuduScanTokenBuilder;
584 
586 
587  // Owned.
588  Data* data_;
589 
590  DISALLOW_COPY_AND_ASSIGN(KuduTabletServer);
591 };
592 
594 class KUDU_EXPORT KuduReplica {
595  public:
596  ~KuduReplica();
597 
602  bool is_leader() const;
603 
605  const KuduTabletServer& ts() const;
606 
607  private:
608  friend class KuduClient;
609  friend class KuduScanTokenBuilder;
610 
611  class KUDU_NO_EXPORT Data;
612 
613  KuduReplica();
614 
615  // Owned.
616  Data* data_;
617 
618  DISALLOW_COPY_AND_ASSIGN(KuduReplica);
619 };
620 
622 class KUDU_EXPORT KuduTablet {
623  public:
624  ~KuduTablet();
625 
628  const std::string& id() const;
629 
635  const std::vector<const KuduReplica*>& replicas() const;
636 
637  private:
638  friend class KuduClient;
639  friend class KuduScanTokenBuilder;
640 
641  class KUDU_NO_EXPORT Data;
642 
643  KuduTablet();
644 
645  // Owned.
646  Data* data_;
647 
648  DISALLOW_COPY_AND_ASSIGN(KuduTablet);
649 };
650 
652 class KUDU_EXPORT KuduTableCreator {
653  public:
654  ~KuduTableCreator();
655 
667  KuduTableCreator& table_name(const std::string& name);
668 
679  KuduTableCreator& schema(const KuduSchema* schema);
680 
697  KuduTableCreator& add_hash_partitions(const std::vector<std::string>& columns,
698  int32_t num_buckets);
699 
715  KuduTableCreator& add_hash_partitions(const std::vector<std::string>& columns,
716  int32_t num_buckets, int32_t seed);
717 
730  KuduTableCreator& set_range_partition_columns(const std::vector<std::string>& columns);
731 
736  };
737 
766  KuduTableCreator& add_range_partition(KuduPartialRow* lower_bound,
767  KuduPartialRow* upper_bound,
768  RangePartitionBound lower_bound_type = INCLUSIVE_BOUND,
769  RangePartitionBound upper_bound_type = EXCLUSIVE_BOUND);
770 
779  KuduTableCreator& add_range_partition_split(KuduPartialRow* split_row);
780 
786  KuduTableCreator& split_rows(const std::vector<const KuduPartialRow*>& split_rows)
787  ATTRIBUTE_DEPRECATED("use add_range_partition_split() instead");
788 
798  KuduTableCreator& num_replicas(int n_replicas);
799 
809  KuduTableCreator& timeout(const MonoDelta& timeout);
810 
818  KuduTableCreator& wait(bool wait);
819 
833  Status Create();
834 
835  private:
836  class KUDU_NO_EXPORT Data;
837 
838  friend class KuduClient;
839 
840  explicit KuduTableCreator(KuduClient* client);
841 
842  // Owned.
843  Data* data_;
844 
845  DISALLOW_COPY_AND_ASSIGN(KuduTableCreator);
846 };
847 
868 class KUDU_EXPORT KuduTable : public sp::enable_shared_from_this<KuduTable> {
869  public:
870  ~KuduTable();
871 
873  const std::string& name() const;
874 
882  const std::string& id() const;
883 
885  const KuduSchema& schema() const;
886 
888  int num_replicas() const;
889 
893  KuduInsert* NewInsert();
894 
898  KuduUpsert* NewUpsert();
899 
903  KuduUpdate* NewUpdate();
904 
908  KuduDelete* NewDelete();
909 
933  KuduPredicate* NewComparisonPredicate(const Slice& col_name,
935  KuduValue* value);
936 
960  KuduPredicate* NewInListPredicate(const Slice& col_name,
961  std::vector<KuduValue*>* values);
962 
973  KuduPredicate* NewIsNotNullPredicate(const Slice& col_name);
974 
985  KuduPredicate* NewIsNullPredicate(const Slice& col_name);
986 
989  KuduClient* client() const;
990 
992  const PartitionSchema& partition_schema() const;
993 
994  private:
995  class KUDU_NO_EXPORT Data;
996 
997  friend class KuduClient;
998  friend class KuduPartitioner;
999 
1000  KuduTable(const sp::shared_ptr<KuduClient>& client,
1001  const std::string& name,
1002  const std::string& id,
1003  int num_replicas,
1004  const KuduSchema& schema,
1005  const PartitionSchema& partition_schema);
1006 
1007  // Owned.
1008  Data* data_;
1009 
1010  DISALLOW_COPY_AND_ASSIGN(KuduTable);
1011 };
1012 
1024 class KUDU_EXPORT KuduTableAlterer {
1025  public:
1026  ~KuduTableAlterer();
1027 
1033  KuduTableAlterer* RenameTo(const std::string& new_name);
1034 
1044  KuduColumnSpec* AddColumn(const std::string& name);
1045 
1054  KuduColumnSpec* AlterColumn(const std::string& name);
1055 
1063  KuduTableAlterer* DropColumn(const std::string& name);
1064 
1096  KuduTableAlterer* AddRangePartition(
1097  KuduPartialRow* lower_bound,
1098  KuduPartialRow* upper_bound,
1101 
1129  KuduTableAlterer* DropRangePartition(
1130  KuduPartialRow* lower_bound,
1131  KuduPartialRow* upper_bound,
1134 
1144  KuduTableAlterer* timeout(const MonoDelta& timeout);
1145 
1157  KuduTableAlterer* wait(bool wait);
1158 
1163  Status Alter();
1164 
1165  private:
1166  class KUDU_NO_EXPORT Data;
1167  friend class KuduClient;
1168 
1169  KuduTableAlterer(KuduClient* client,
1170  const std::string& name);
1171 
1172  // Owned.
1173  Data* data_;
1174 
1175  DISALLOW_COPY_AND_ASSIGN(KuduTableAlterer);
1176 };
1177 
1183 class KUDU_EXPORT KuduError {
1184  public:
1185  ~KuduError();
1186 
1188  const Status& status() const;
1189 
1191  const KuduWriteOperation& failed_op() const;
1192 
1200  KuduWriteOperation* release_failed_op();
1201 
1212  bool was_possibly_successful() const;
1213 
1214  private:
1215  class KUDU_NO_EXPORT Data;
1216 
1217  friend class internal::Batcher;
1218  friend class internal::ErrorCollector;
1219  friend class KuduSession;
1220 
1221  KuduError(KuduWriteOperation* failed_op, const Status& error);
1222 
1223  // Owned.
1224  Data* data_;
1225 
1226  DISALLOW_COPY_AND_ASSIGN(KuduError);
1227 };
1228 
1229 
1288 class KUDU_EXPORT KuduSession : public sp::enable_shared_from_this<KuduSession> {
1289  public:
1290  ~KuduSession();
1291 
1293  enum FlushMode {
1299 
1327 
1340  MANUAL_FLUSH
1341  };
1342 
1351  Status SetFlushMode(FlushMode m) WARN_UNUSED_RESULT;
1352 
1374 
1389  COMMIT_WAIT
1390  };
1391 
1397  Status SetExternalConsistencyMode(ExternalConsistencyMode m)
1398  WARN_UNUSED_RESULT;
1399 
1417  Status SetMutationBufferSpace(size_t size_bytes) WARN_UNUSED_RESULT;
1418 
1442  Status SetMutationBufferFlushWatermark(double watermark_pct)
1443  WARN_UNUSED_RESULT;
1444 
1466  Status SetMutationBufferFlushInterval(unsigned int millis) WARN_UNUSED_RESULT;
1467 
1492  Status SetMutationBufferMaxNum(unsigned int max_num) WARN_UNUSED_RESULT;
1493 
1499  void SetTimeoutMillis(int millis);
1500 
1504 
1528  Status Apply(KuduWriteOperation* write_op) WARN_UNUSED_RESULT;
1529 
1544  Status Flush() WARN_UNUSED_RESULT;
1545 
1586  void FlushAsync(KuduStatusCallback* cb);
1587 
1590  Status Close() WARN_UNUSED_RESULT;
1591 
1600  bool HasPendingOperations() const;
1601 
1623  int CountBufferedOperations() const
1624  ATTRIBUTE_DEPRECATED("this method is experimental and will disappear "
1625  "in a future release");
1626 
1651  Status SetErrorBufferSpace(size_t size_bytes);
1652 
1662  int CountPendingErrors() const;
1663 
1674  void GetPendingErrors(std::vector<KuduError*>* errors, bool* overflowed);
1675 
1677  KuduClient* client() const;
1678 
1679  private:
1680  class KUDU_NO_EXPORT Data;
1681 
1682  friend class KuduClient;
1683  friend class internal::Batcher;
1684  friend class ClientTest;
1685  FRIEND_TEST(ClientTest, TestAutoFlushBackgroundApplyBlocks);
1686  FRIEND_TEST(ClientTest, TestAutoFlushBackgroundAndErrorCollector);
1687 
1688  explicit KuduSession(const sp::shared_ptr<KuduClient>& client);
1689 
1690  // Owned.
1691  Data* data_;
1692 
1693  DISALLOW_COPY_AND_ASSIGN(KuduSession);
1694 };
1695 
1696 
1701 class KUDU_EXPORT KuduScanner {
1702  public:
1704  enum ReadMode {
1713 
1733  READ_AT_SNAPSHOT
1734  };
1735 
1739  enum OrderMode {
1746 
1751  ORDERED
1752  };
1753 
1757  enum { kScanTimeoutMillis = 30000 };
1758 
1764  explicit KuduScanner(KuduTable* table);
1765  ~KuduScanner();
1766 
1776  Status SetProjectedColumnNames(const std::vector<std::string>& col_names)
1777  WARN_UNUSED_RESULT;
1778 
1788  Status SetProjectedColumnIndexes(const std::vector<int>& col_indexes)
1789  WARN_UNUSED_RESULT;
1790 
1796  Status SetProjectedColumns(const std::vector<std::string>& col_names)
1797  WARN_UNUSED_RESULT
1798  ATTRIBUTE_DEPRECATED("use SetProjectedColumnNames() instead");
1799 
1808  Status AddConjunctPredicate(KuduPredicate* pred) WARN_UNUSED_RESULT;
1809 
1818  Status AddLowerBound(const KuduPartialRow& key);
1819 
1827  Status AddLowerBoundRaw(const Slice& key)
1828  ATTRIBUTE_DEPRECATED("use AddLowerBound() instead");
1829 
1838  Status AddExclusiveUpperBound(const KuduPartialRow& key);
1839 
1847  Status AddExclusiveUpperBoundRaw(const Slice& key)
1848  ATTRIBUTE_DEPRECATED("use AddExclusiveUpperBound() instead");
1849 
1858  Status AddLowerBoundPartitionKeyRaw(const Slice& partition_key);
1859 
1868  Status AddExclusiveUpperBoundPartitionKeyRaw(const Slice& partition_key);
1869 
1876  Status SetCacheBlocks(bool cache_blocks);
1877 
1879  Status Open();
1880 
1896  Status KeepAlive();
1897 
1906  void Close();
1907 
1916  bool HasMoreRows() const;
1917 
1929  Status NextBatch(std::vector<KuduRowResult>* rows)
1930  ATTRIBUTE_DEPRECATED("use NextBatch(KuduScanBatch*) instead");
1931 
1940  Status NextBatch(KuduScanBatch* batch);
1941 
1950  Status GetCurrentServer(KuduTabletServer** server);
1951 
1953  const ResourceMetrics& GetResourceMetrics() const;
1954 
1961  Status SetBatchSizeBytes(uint32_t batch_size);
1962 
1970  Status SetSelection(KuduClient::ReplicaSelection selection)
1971  WARN_UNUSED_RESULT;
1972 
1978  Status SetReadMode(ReadMode read_mode) WARN_UNUSED_RESULT;
1979 
1985  Status SetOrderMode(OrderMode order_mode) WARN_UNUSED_RESULT
1986  ATTRIBUTE_DEPRECATED("use SetFaultTolerant() instead");
1987 
2000  Status SetFaultTolerant() WARN_UNUSED_RESULT;
2001 
2007  Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros) WARN_UNUSED_RESULT;
2008 
2021  Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT;
2022 
2028  Status SetTimeoutMillis(int millis);
2029 
2031  KuduSchema GetProjectionSchema() const;
2032 
2034  //
2040  static const uint64_t NO_FLAGS = 0;
2048  static const uint64_t PAD_UNIXTIME_MICROS_TO_16_BYTES = 1 << 0;
2072  Status SetRowFormatFlags(uint64_t flags);
2074 
2080  std::string ToString() const;
2081 
2082  private:
2083  class KUDU_NO_EXPORT Data;
2084 
2085  friend class KuduScanToken;
2086  FRIEND_TEST(ClientTest, TestScanCloseProxy);
2087  FRIEND_TEST(ClientTest, TestScanFaultTolerance);
2088  FRIEND_TEST(ClientTest, TestScanNoBlockCaching);
2089  FRIEND_TEST(ClientTest, TestScanTimeout);
2090  FRIEND_TEST(ClientTest, TestReadAtSnapshotNoTimestampSet);
2091  FRIEND_TEST(ConsistencyITest, TestSnapshotScanTimestampReuse);
2092 
2093  // Owned.
2094  Data* data_;
2095 
2096  DISALLOW_COPY_AND_ASSIGN(KuduScanner);
2097 };
2098 
2119 class KUDU_EXPORT KuduScanToken {
2120  public:
2121 
2122  ~KuduScanToken();
2123 
2134  Status IntoKuduScanner(KuduScanner** scanner) const WARN_UNUSED_RESULT;
2135 
2137  const KuduTablet& tablet() const;
2138 
2146  Status Serialize(std::string* buf) const WARN_UNUSED_RESULT;
2147 
2159  static Status DeserializeIntoScanner(KuduClient* client,
2160  const std::string& serialized_token,
2161  KuduScanner** scanner) WARN_UNUSED_RESULT;
2162 
2163  private:
2164  class KUDU_NO_EXPORT Data;
2165 
2166  friend class KuduScanTokenBuilder;
2167 
2168  KuduScanToken();
2169 
2170  // Owned.
2171  Data* data_;
2172 
2173  DISALLOW_COPY_AND_ASSIGN(KuduScanToken);
2174 };
2175 
2179 class KUDU_EXPORT KuduScanTokenBuilder {
2180  public:
2181 
2187  explicit KuduScanTokenBuilder(KuduTable* table);
2189 
2199  Status SetProjectedColumnNames(const std::vector<std::string>& col_names)
2200  WARN_UNUSED_RESULT;
2201 
2203  Status SetProjectedColumnIndexes(const std::vector<int>& col_indexes)
2204  WARN_UNUSED_RESULT;
2205 
2207  Status AddConjunctPredicate(KuduPredicate* pred) WARN_UNUSED_RESULT;
2208 
2210  Status AddLowerBound(const KuduPartialRow& key) WARN_UNUSED_RESULT;
2211 
2220  Status AddUpperBound(const KuduPartialRow& key) WARN_UNUSED_RESULT;
2221 
2223  Status SetCacheBlocks(bool cache_blocks) WARN_UNUSED_RESULT;
2224 
2231  Status SetBatchSizeBytes(uint32_t batch_size) WARN_UNUSED_RESULT;
2232 
2240  Status SetSelection(KuduClient::ReplicaSelection selection)
2241  WARN_UNUSED_RESULT;
2242 
2244  Status SetReadMode(KuduScanner::ReadMode read_mode) WARN_UNUSED_RESULT;
2245 
2247  Status SetFaultTolerant() WARN_UNUSED_RESULT;
2248 
2250  Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros)
2251  WARN_UNUSED_RESULT;
2252 
2254  Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT;
2255 
2257  Status SetTimeoutMillis(int millis) WARN_UNUSED_RESULT;
2258 
2267  Status Build(std::vector<KuduScanToken*>* tokens) WARN_UNUSED_RESULT;
2268 
2269  private:
2270  class KUDU_NO_EXPORT Data;
2271 
2272  // Owned.
2273  Data* data_;
2274 
2275  DISALLOW_COPY_AND_ASSIGN(KuduScanTokenBuilder);
2276 };
2277 
2279 class KUDU_EXPORT KuduPartitionerBuilder {
2280  public:
2285  explicit KuduPartitionerBuilder(sp::shared_ptr<KuduTable> table);
2287 
2289  KuduPartitionerBuilder* SetBuildTimeout(MonoDelta timeout);
2290 
2307  Status Build(KuduPartitioner** partitioner);
2308  private:
2309  class KUDU_NO_EXPORT Data;
2310 
2311  // Owned.
2312  Data* data_;
2313 
2314  DISALLOW_COPY_AND_ASSIGN(KuduPartitionerBuilder);
2315 };
2316 
2327 class KUDU_EXPORT KuduPartitioner {
2328  public:
2329  ~KuduPartitioner();
2330 
2334  int NumPartitions() const;
2335 
2347  Status PartitionRow(const KuduPartialRow& row, int* partition);
2348  private:
2349  class KUDU_NO_EXPORT Data;
2350  friend class KuduPartitionerBuilder;
2351 
2352  explicit KuduPartitioner(Data* data);
2353  Data* data_; // Owned.
2354 };
2355 
2356 
2357 } // namespace client
2358 } // namespace kudu
2359 #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:431
A representation of an operation&#39;s outcome.
Definition: status.h:135
A constant cell value with a specific type.
Definition: value.h:34
Builder for Partitioner instances.
Definition: client.h:2279
ExternalConsistencyMode
The possible external consistency modes on which Kudu operates.
Definition: client.h:1354
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:240
static const uint64_t kNoTimestamp
Definition: client.h:445
ReplicaSelection
Policy with which to choose amongst multiple replicas.
Definition: client.h:424
A single row upsert to be sent to the cluster.
Definition: write_op.h:166
Builds scan tokens for a table.
Definition: client.h:2179
ComparisonOp
Supported comparison operators.
Definition: scan_predicate.h:37
Alters an existing table based on the provided steps.
Definition: client.h:1024
OrderMode
Definition: client.h:1739
Definition: client.h:1745
Smart pointer typedefs for externally-faced code.
Definition: client.h:2327
A representation of comparison predicate for Kudu queries.
Definition: scan_predicate.h:34
An exclusive bound.
Definition: client.h:734
This class represents an error which occurred in a write operation.
Definition: client.h:1183
A handle for a connection to a cluster.
Definition: client.h:295
An inclusive bound.
Definition: client.h:735
In-memory representation of a remote tablet server.
Definition: client.h:562
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:868
This class is a representation of a single scan.
Definition: client.h:1701
ReadMode
The read modes for scanners.
Definition: client.h:1704
A &quot;factory&quot; for KuduClient objects.
Definition: client.h:193
A single-row write operation to be sent to a Kudu table.
Definition: write_op.h:65
Select the LEADER replica.
Definition: client.h:425
RangePartitionBound
Range partition bound type.
Definition: client.h:733
In-memory representation of a remote tablet.
Definition: client.h:622
In-memory representation of a remote tablet&#39;s replica.
Definition: client.h:594
FlushMode
Modes of flush operations.
Definition: client.h:1293
A scan descriptor limited to a single physical contiguous location.
Definition: client.h:2119
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:652
A row which may only contain values for a subset of the columns.
Definition: partial_row.h:63
Representation of a Kudu client session.
Definition: client.h:1288
A representation of a time interval.
Definition: monotime.h:46