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 class SecurityUnknownTskTest;
56 
57 namespace tools {
58 class LeaderMasterProxy;
59 } // namespace tools
60 
61 namespace client {
62 
63 class KuduLoggingCallback;
64 class KuduPartitioner;
65 class KuduScanToken;
66 class KuduSession;
67 class KuduStatusCallback;
68 class KuduTable;
69 class KuduTableAlterer;
70 class KuduTableCreator;
71 class KuduTablet;
72 class KuduTabletServer;
73 class KuduValue;
74 class KuduWriteOperation;
75 
76 namespace internal {
77 class Batcher;
78 class ErrorCollector;
79 class GetTableSchemaRpc;
80 class LookupRpc;
81 class MetaCache;
82 class RemoteTablet;
83 class RemoteTabletServer;
84 class WriteRpc;
85 } // namespace internal
86 
99 void KUDU_EXPORT InstallLoggingCallback(KuduLoggingCallback* cb);
100 
107 void KUDU_EXPORT UninstallLoggingCallback();
108 
124 void KUDU_EXPORT SetVerboseLogLevel(int level);
125 
135 Status KUDU_EXPORT SetInternalSignalNumber(int signum);
136 
148 Status KUDU_EXPORT DisableSaslInitialization();
149 
150 
173 Status KUDU_EXPORT DisableOpenSSLInitialization();
174 
177 std::string KUDU_EXPORT GetShortVersionString();
178 
181 std::string KUDU_EXPORT GetAllVersionInfo();
182 
187 class KUDU_EXPORT KuduClientBuilder {
188  public:
191 
195  KuduClientBuilder& clear_master_server_addrs();
196 
202  KuduClientBuilder& master_server_addrs(const std::vector<std::string>& addrs);
203 
211  KuduClientBuilder& add_master_server_addr(const std::string& addr);
212 
222  KuduClientBuilder& default_admin_operation_timeout(const MonoDelta& timeout);
223 
231  KuduClientBuilder& default_rpc_timeout(const MonoDelta& timeout);
232 
240  KuduClientBuilder& import_authentication_credentials(std::string authn_creds);
241 
252  Status Build(sp::shared_ptr<KuduClient>* client);
253 
254  private:
255  class KUDU_NO_EXPORT Data;
256 
257  // Owned.
258  Data* data_;
259 
260  DISALLOW_COPY_AND_ASSIGN(KuduClientBuilder);
261 };
262 
289 class KUDU_EXPORT KuduClient : public sp::enable_shared_from_this<KuduClient> {
290  public:
291  ~KuduClient();
292 
297  KuduTableCreator* NewTableCreator();
298 
307  Status IsCreateTableInProgress(const std::string& table_name,
308  bool *create_in_progress);
309 
315  Status DeleteTable(const std::string& table_name);
316 
323  KuduTableAlterer* NewTableAlterer(const std::string& table_name);
324 
333  Status IsAlterTableInProgress(const std::string& table_name,
334  bool *alter_in_progress);
342  Status GetTableSchema(const std::string& table_name,
343  KuduSchema* schema);
344 
351  Status ListTabletServers(std::vector<KuduTabletServer*>* tablet_servers);
352 
360  Status ListTables(std::vector<std::string>* tables,
361  const std::string& filter = "");
362 
370  Status TableExists(const std::string& table_name, bool* exists);
371 
388  Status OpenTable(const std::string& table_name,
389  sp::shared_ptr<KuduTable>* table);
390 
396  sp::shared_ptr<KuduSession> NewSession();
397 
399 
412  Status KUDU_NO_EXPORT GetTablet(const std::string& tablet_id,
413  KuduTablet** tablet);
414 
416 
420 
421  CLOSEST_REPLICA,
422 
424  FIRST_REPLICA
425  };
426 
429  bool IsMultiMaster() const;
430 
432  const MonoDelta& default_admin_operation_timeout() const;
433 
435  const MonoDelta& default_rpc_timeout() const;
436 
439  static const uint64_t kNoTimestamp;
440 
480  uint64_t GetLatestObservedTimestamp() const;
481 
496  void SetLatestObservedTimestamp(uint64_t ht_timestamp);
497 
509  Status ExportAuthenticationCredentials(std::string* authn_creds) const;
510 
511  private:
512  class KUDU_NO_EXPORT Data;
513 
514  friend class internal::Batcher;
515  friend class internal::GetTableSchemaRpc;
516  friend class internal::LookupRpc;
517  friend class internal::MetaCache;
518  friend class internal::RemoteTablet;
519  friend class internal::RemoteTabletServer;
520  friend class internal::WriteRpc;
521  friend class ClientTest;
522  friend class KuduClientBuilder;
523  friend class KuduPartitionerBuilder;
524  friend class KuduScanner;
525  friend class KuduScanToken;
526  friend class KuduScanTokenBuilder;
527  friend class KuduSession;
528  friend class KuduTable;
529  friend class KuduTableAlterer;
530  friend class KuduTableCreator;
531  friend class ::kudu::SecurityUnknownTskTest;
532  friend class tools::LeaderMasterProxy;
533 
534  FRIEND_TEST(kudu::ClientStressTest, TestUniqueClientIds);
535  FRIEND_TEST(ClientTest, TestGetSecurityInfoFromMaster);
536  FRIEND_TEST(ClientTest, TestGetTabletServerBlacklist);
537  FRIEND_TEST(ClientTest, TestMasterDown);
538  FRIEND_TEST(ClientTest, TestMasterLookupPermits);
539  FRIEND_TEST(ClientTest, TestMetaCacheExpiry);
540  FRIEND_TEST(ClientTest, TestNonCoveringRangePartitions);
541  FRIEND_TEST(ClientTest, TestReplicatedTabletWritesWithLeaderElection);
542  FRIEND_TEST(ClientTest, TestScanFaultTolerance);
543  FRIEND_TEST(ClientTest, TestScanTimeout);
544  FRIEND_TEST(ClientTest, TestWriteWithDeadMaster);
545  FRIEND_TEST(MasterFailoverTest, TestPauseAfterCreateTableIssued);
546 
547  KuduClient();
548 
549  // Owned.
550  Data* data_;
551 
552  DISALLOW_COPY_AND_ASSIGN(KuduClient);
553 };
554 
556 class KUDU_EXPORT KuduTabletServer {
557  public:
558  ~KuduTabletServer();
559 
562  const std::string& uuid() const;
563 
566  const std::string& hostname() const;
567 
570  uint16_t port() const;
571 
572  private:
573  class KUDU_NO_EXPORT Data;
574 
575  friend class KuduClient;
576  friend class KuduScanner;
577  friend class KuduScanTokenBuilder;
578 
580 
581  // Owned.
582  Data* data_;
583 
584  DISALLOW_COPY_AND_ASSIGN(KuduTabletServer);
585 };
586 
588 class KUDU_EXPORT KuduReplica {
589  public:
590  ~KuduReplica();
591 
596  bool is_leader() const;
597 
599  const KuduTabletServer& ts() const;
600 
601  private:
602  friend class KuduClient;
603  friend class KuduScanTokenBuilder;
604 
605  class KUDU_NO_EXPORT Data;
606 
607  KuduReplica();
608 
609  // Owned.
610  Data* data_;
611 
612  DISALLOW_COPY_AND_ASSIGN(KuduReplica);
613 };
614 
616 class KUDU_EXPORT KuduTablet {
617  public:
618  ~KuduTablet();
619 
622  const std::string& id() const;
623 
629  const std::vector<const KuduReplica*>& replicas() const;
630 
631  private:
632  friend class KuduClient;
633  friend class KuduScanTokenBuilder;
634 
635  class KUDU_NO_EXPORT Data;
636 
637  KuduTablet();
638 
639  // Owned.
640  Data* data_;
641 
642  DISALLOW_COPY_AND_ASSIGN(KuduTablet);
643 };
644 
646 class KUDU_EXPORT KuduTableCreator {
647  public:
648  ~KuduTableCreator();
649 
661  KuduTableCreator& table_name(const std::string& name);
662 
673  KuduTableCreator& schema(const KuduSchema* schema);
674 
691  KuduTableCreator& add_hash_partitions(const std::vector<std::string>& columns,
692  int32_t num_buckets);
693 
709  KuduTableCreator& add_hash_partitions(const std::vector<std::string>& columns,
710  int32_t num_buckets, int32_t seed);
711 
724  KuduTableCreator& set_range_partition_columns(const std::vector<std::string>& columns);
725 
730  };
731 
760  KuduTableCreator& add_range_partition(KuduPartialRow* lower_bound,
761  KuduPartialRow* upper_bound,
762  RangePartitionBound lower_bound_type = INCLUSIVE_BOUND,
763  RangePartitionBound upper_bound_type = EXCLUSIVE_BOUND);
764 
773  KuduTableCreator& add_range_partition_split(KuduPartialRow* split_row);
774 
780  KuduTableCreator& split_rows(const std::vector<const KuduPartialRow*>& split_rows)
781  ATTRIBUTE_DEPRECATED("use add_range_partition_split() instead");
782 
792  KuduTableCreator& num_replicas(int n_replicas);
793 
803  KuduTableCreator& timeout(const MonoDelta& timeout);
804 
812  KuduTableCreator& wait(bool wait);
813 
827  Status Create();
828 
829  private:
830  class KUDU_NO_EXPORT Data;
831 
832  friend class KuduClient;
833 
834  explicit KuduTableCreator(KuduClient* client);
835 
836  // Owned.
837  Data* data_;
838 
839  DISALLOW_COPY_AND_ASSIGN(KuduTableCreator);
840 };
841 
862 class KUDU_EXPORT KuduTable : public sp::enable_shared_from_this<KuduTable> {
863  public:
864  ~KuduTable();
865 
867  const std::string& name() const;
868 
876  const std::string& id() const;
877 
879  const KuduSchema& schema() const;
880 
882  int num_replicas() const;
883 
887  KuduInsert* NewInsert();
888 
892  KuduUpsert* NewUpsert();
893 
897  KuduUpdate* NewUpdate();
898 
902  KuduDelete* NewDelete();
903 
927  KuduPredicate* NewComparisonPredicate(const Slice& col_name,
929  KuduValue* value);
930 
954  KuduPredicate* NewInListPredicate(const Slice& col_name,
955  std::vector<KuduValue*>* values);
956 
967  KuduPredicate* NewIsNotNullPredicate(const Slice& col_name);
968 
979  KuduPredicate* NewIsNullPredicate(const Slice& col_name);
980 
983  KuduClient* client() const;
984 
986  const PartitionSchema& partition_schema() const;
987 
988  private:
989  class KUDU_NO_EXPORT Data;
990 
991  friend class KuduClient;
992  friend class KuduPartitioner;
993 
994  KuduTable(const sp::shared_ptr<KuduClient>& client,
995  const std::string& name,
996  const std::string& id,
997  int num_replicas,
998  const KuduSchema& schema,
999  const PartitionSchema& partition_schema);
1000 
1001  // Owned.
1002  Data* data_;
1003 
1004  DISALLOW_COPY_AND_ASSIGN(KuduTable);
1005 };
1006 
1018 class KUDU_EXPORT KuduTableAlterer {
1019  public:
1020  ~KuduTableAlterer();
1021 
1027  KuduTableAlterer* RenameTo(const std::string& new_name);
1028 
1038  KuduColumnSpec* AddColumn(const std::string& name);
1039 
1048  KuduColumnSpec* AlterColumn(const std::string& name);
1049 
1057  KuduTableAlterer* DropColumn(const std::string& name);
1058 
1090  KuduTableAlterer* AddRangePartition(
1091  KuduPartialRow* lower_bound,
1092  KuduPartialRow* upper_bound,
1093  KuduTableCreator::RangePartitionBound lower_bound_type = KuduTableCreator::INCLUSIVE_BOUND,
1094  KuduTableCreator::RangePartitionBound upper_bound_type = KuduTableCreator::EXCLUSIVE_BOUND);
1095 
1123  KuduTableAlterer* DropRangePartition(
1124  KuduPartialRow* lower_bound,
1125  KuduPartialRow* upper_bound,
1126  KuduTableCreator::RangePartitionBound lower_bound_type = KuduTableCreator::INCLUSIVE_BOUND,
1127  KuduTableCreator::RangePartitionBound upper_bound_type = KuduTableCreator::EXCLUSIVE_BOUND);
1128 
1138  KuduTableAlterer* timeout(const MonoDelta& timeout);
1139 
1151  KuduTableAlterer* wait(bool wait);
1152 
1157  Status Alter();
1158 
1159  private:
1160  class KUDU_NO_EXPORT Data;
1161  friend class KuduClient;
1162 
1163  KuduTableAlterer(KuduClient* client,
1164  const std::string& name);
1165 
1166  // Owned.
1167  Data* data_;
1168 
1169  DISALLOW_COPY_AND_ASSIGN(KuduTableAlterer);
1170 };
1171 
1177 class KUDU_EXPORT KuduError {
1178  public:
1179  ~KuduError();
1180 
1182  const Status& status() const;
1183 
1185  const KuduWriteOperation& failed_op() const;
1186 
1194  KuduWriteOperation* release_failed_op();
1195 
1206  bool was_possibly_successful() const;
1207 
1208  private:
1209  class KUDU_NO_EXPORT Data;
1210 
1211  friend class internal::Batcher;
1212  friend class internal::ErrorCollector;
1213  friend class KuduSession;
1214 
1215  KuduError(KuduWriteOperation* failed_op, const Status& error);
1216 
1217  // Owned.
1218  Data* data_;
1219 
1220  DISALLOW_COPY_AND_ASSIGN(KuduError);
1221 };
1222 
1223 
1282 class KUDU_EXPORT KuduSession : public sp::enable_shared_from_this<KuduSession> {
1283  public:
1284  ~KuduSession();
1285 
1287  enum FlushMode {
1293 
1321 
1334  MANUAL_FLUSH
1335  };
1336 
1345  Status SetFlushMode(FlushMode m) WARN_UNUSED_RESULT;
1346 
1368 
1383  COMMIT_WAIT
1384  };
1385 
1391  Status SetExternalConsistencyMode(ExternalConsistencyMode m)
1392  WARN_UNUSED_RESULT;
1393 
1411  Status SetMutationBufferSpace(size_t size_bytes) WARN_UNUSED_RESULT;
1412 
1436  Status SetMutationBufferFlushWatermark(double watermark_pct)
1437  WARN_UNUSED_RESULT;
1438 
1460  Status SetMutationBufferFlushInterval(unsigned int millis) WARN_UNUSED_RESULT;
1461 
1486  Status SetMutationBufferMaxNum(unsigned int max_num) WARN_UNUSED_RESULT;
1487 
1493  void SetTimeoutMillis(int millis);
1494 
1498 
1522  Status Apply(KuduWriteOperation* write_op) WARN_UNUSED_RESULT;
1523 
1538  Status Flush() WARN_UNUSED_RESULT;
1539 
1580  void FlushAsync(KuduStatusCallback* cb);
1581 
1584  Status Close() WARN_UNUSED_RESULT;
1585 
1594  bool HasPendingOperations() const;
1595 
1617  int CountBufferedOperations() const
1618  ATTRIBUTE_DEPRECATED("this method is experimental and will disappear "
1619  "in a future release");
1620 
1645  Status SetErrorBufferSpace(size_t size_bytes);
1646 
1656  int CountPendingErrors() const;
1657 
1668  void GetPendingErrors(std::vector<KuduError*>* errors, bool* overflowed);
1669 
1671  KuduClient* client() const;
1672 
1673  private:
1674  class KUDU_NO_EXPORT Data;
1675 
1676  friend class KuduClient;
1677  friend class internal::Batcher;
1678  friend class ClientTest;
1679  FRIEND_TEST(ClientTest, TestAutoFlushBackgroundApplyBlocks);
1680  FRIEND_TEST(ClientTest, TestAutoFlushBackgroundAndErrorCollector);
1681 
1682  explicit KuduSession(const sp::shared_ptr<KuduClient>& client);
1683 
1684  // Owned.
1685  Data* data_;
1686 
1687  DISALLOW_COPY_AND_ASSIGN(KuduSession);
1688 };
1689 
1690 
1695 class KUDU_EXPORT KuduScanner {
1696  public:
1698  enum ReadMode {
1707 
1727  READ_AT_SNAPSHOT
1728  };
1729 
1733  enum OrderMode {
1740 
1745  ORDERED
1746  };
1747 
1751  enum { kScanTimeoutMillis = 30000 };
1752 
1758  explicit KuduScanner(KuduTable* table);
1759  ~KuduScanner();
1760 
1770  Status SetProjectedColumnNames(const std::vector<std::string>& col_names)
1771  WARN_UNUSED_RESULT;
1772 
1782  Status SetProjectedColumnIndexes(const std::vector<int>& col_indexes)
1783  WARN_UNUSED_RESULT;
1784 
1790  Status SetProjectedColumns(const std::vector<std::string>& col_names)
1791  WARN_UNUSED_RESULT
1792  ATTRIBUTE_DEPRECATED("use SetProjectedColumnNames() instead");
1793 
1802  Status AddConjunctPredicate(KuduPredicate* pred) WARN_UNUSED_RESULT;
1803 
1812  Status AddLowerBound(const KuduPartialRow& key);
1813 
1821  Status AddLowerBoundRaw(const Slice& key)
1822  ATTRIBUTE_DEPRECATED("use AddLowerBound() instead");
1823 
1832  Status AddExclusiveUpperBound(const KuduPartialRow& key);
1833 
1841  Status AddExclusiveUpperBoundRaw(const Slice& key)
1842  ATTRIBUTE_DEPRECATED("use AddExclusiveUpperBound() instead");
1843 
1852  Status AddLowerBoundPartitionKeyRaw(const Slice& partition_key);
1853 
1862  Status AddExclusiveUpperBoundPartitionKeyRaw(const Slice& partition_key);
1863 
1870  Status SetCacheBlocks(bool cache_blocks);
1871 
1873  Status Open();
1874 
1890  Status KeepAlive();
1891 
1900  void Close();
1901 
1910  bool HasMoreRows() const;
1911 
1923  Status NextBatch(std::vector<KuduRowResult>* rows)
1924  ATTRIBUTE_DEPRECATED("use NextBatch(KuduScanBatch*) instead");
1925 
1934  Status NextBatch(KuduScanBatch* batch);
1935 
1944  Status GetCurrentServer(KuduTabletServer** server);
1945 
1947  const ResourceMetrics& GetResourceMetrics() const;
1948 
1955  Status SetBatchSizeBytes(uint32_t batch_size);
1956 
1964  Status SetSelection(KuduClient::ReplicaSelection selection)
1965  WARN_UNUSED_RESULT;
1966 
1972  Status SetReadMode(ReadMode read_mode) WARN_UNUSED_RESULT;
1973 
1979  Status SetOrderMode(OrderMode order_mode) WARN_UNUSED_RESULT
1980  ATTRIBUTE_DEPRECATED("use SetFaultTolerant() instead");
1981 
1994  Status SetFaultTolerant() WARN_UNUSED_RESULT;
1995 
2001  Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros) WARN_UNUSED_RESULT;
2002 
2015  Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT;
2016 
2022  Status SetTimeoutMillis(int millis);
2023 
2025  KuduSchema GetProjectionSchema() const;
2026 
2028  //
2034  static const uint64_t NO_FLAGS = 0;
2042  static const uint64_t PAD_UNIXTIME_MICROS_TO_16_BYTES = 1 << 0;
2066  Status SetRowFormatFlags(uint64_t flags);
2068 
2074  std::string ToString() const;
2075 
2076  private:
2077  class KUDU_NO_EXPORT Data;
2078 
2079  friend class KuduScanToken;
2080  FRIEND_TEST(ClientTest, TestScanCloseProxy);
2081  FRIEND_TEST(ClientTest, TestScanFaultTolerance);
2082  FRIEND_TEST(ClientTest, TestScanNoBlockCaching);
2083  FRIEND_TEST(ClientTest, TestScanTimeout);
2084  FRIEND_TEST(ClientTest, TestReadAtSnapshotNoTimestampSet);
2085  FRIEND_TEST(ConsistencyITest, TestSnapshotScanTimestampReuse);
2086 
2087  // Owned.
2088  Data* data_;
2089 
2090  DISALLOW_COPY_AND_ASSIGN(KuduScanner);
2091 };
2092 
2113 class KUDU_EXPORT KuduScanToken {
2114  public:
2115 
2116  ~KuduScanToken();
2117 
2128  Status IntoKuduScanner(KuduScanner** scanner) const WARN_UNUSED_RESULT;
2129 
2131  const KuduTablet& tablet() const;
2132 
2140  Status Serialize(std::string* buf) const WARN_UNUSED_RESULT;
2141 
2153  static Status DeserializeIntoScanner(KuduClient* client,
2154  const std::string& serialized_token,
2155  KuduScanner** scanner) WARN_UNUSED_RESULT;
2156 
2157  private:
2158  class KUDU_NO_EXPORT Data;
2159 
2160  friend class KuduScanTokenBuilder;
2161 
2162  KuduScanToken();
2163 
2164  // Owned.
2165  Data* data_;
2166 
2167  DISALLOW_COPY_AND_ASSIGN(KuduScanToken);
2168 };
2169 
2173 class KUDU_EXPORT KuduScanTokenBuilder {
2174  public:
2175 
2181  explicit KuduScanTokenBuilder(KuduTable* table);
2183 
2193  Status SetProjectedColumnNames(const std::vector<std::string>& col_names)
2194  WARN_UNUSED_RESULT;
2195 
2197  Status SetProjectedColumnIndexes(const std::vector<int>& col_indexes)
2198  WARN_UNUSED_RESULT;
2199 
2201  Status AddConjunctPredicate(KuduPredicate* pred) WARN_UNUSED_RESULT;
2202 
2204  Status AddLowerBound(const KuduPartialRow& key) WARN_UNUSED_RESULT;
2205 
2214  Status AddUpperBound(const KuduPartialRow& key) WARN_UNUSED_RESULT;
2215 
2217  Status SetCacheBlocks(bool cache_blocks) WARN_UNUSED_RESULT;
2218 
2225  Status SetBatchSizeBytes(uint32_t batch_size) WARN_UNUSED_RESULT;
2226 
2234  Status SetSelection(KuduClient::ReplicaSelection selection)
2235  WARN_UNUSED_RESULT;
2236 
2238  Status SetReadMode(KuduScanner::ReadMode read_mode) WARN_UNUSED_RESULT;
2239 
2241  Status SetFaultTolerant() WARN_UNUSED_RESULT;
2242 
2244  Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros)
2245  WARN_UNUSED_RESULT;
2246 
2248  Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT;
2249 
2251  Status SetTimeoutMillis(int millis) WARN_UNUSED_RESULT;
2252 
2261  Status Build(std::vector<KuduScanToken*>* tokens) WARN_UNUSED_RESULT;
2262 
2263  private:
2264  class KUDU_NO_EXPORT Data;
2265 
2266  // Owned.
2267  Data* data_;
2268 
2269  DISALLOW_COPY_AND_ASSIGN(KuduScanTokenBuilder);
2270 };
2271 
2273 class KUDU_EXPORT KuduPartitionerBuilder {
2274  public:
2279  explicit KuduPartitionerBuilder(sp::shared_ptr<KuduTable> table);
2281 
2283  KuduPartitionerBuilder* SetBuildTimeout(MonoDelta timeout);
2284 
2301  Status Build(KuduPartitioner** partitioner);
2302  private:
2303  class KUDU_NO_EXPORT Data;
2304 
2305  // Owned.
2306  Data* data_;
2307 
2308  DISALLOW_COPY_AND_ASSIGN(KuduPartitionerBuilder);
2309 };
2310 
2321 class KUDU_EXPORT KuduPartitioner {
2322  public:
2323  ~KuduPartitioner();
2324 
2328  int NumPartitions() const;
2329 
2341  Status PartitionRow(const KuduPartialRow& row, int* partition);
2342  private:
2343  class KUDU_NO_EXPORT Data;
2344  friend class KuduPartitionerBuilder;
2345 
2346  explicit KuduPartitioner(Data* data);
2347  Data* data_; // Owned.
2348 };
2349 
2350 
2351 } // namespace client
2352 } // namespace kudu
2353 #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:423
A representation of an operation&#39;s outcome.
Definition: status.h:130
A constant cell value with a specific type.
Definition: value.h:33
Builder for Partitioner instances.
Definition: client.h:2273
ExternalConsistencyMode
The possible external consistency modes on which Kudu operates.
Definition: client.h:1348
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:232
static const uint64_t kNoTimestamp
Definition: client.h:439
ReplicaSelection
Policy with which to choose amongst multiple replicas.
Definition: client.h:418
A single row upsert to be sent to the cluster.
Definition: write_op.h:157
Builds scan tokens for a table.
Definition: client.h:2173
ComparisonOp
Supported comparison operators.
Definition: scan_predicate.h:39
Alters an existing table based on the provided steps.
Definition: client.h:1018
OrderMode
Definition: client.h:1733
Definition: client.h:1739
Smart pointer typedefs for externally-faced code.
Definition: client.h:2321
A representation of comparison predicate for Kudu queries.
Definition: scan_predicate.h:36
An exclusive bound.
Definition: client.h:728
This class represents an error which occurred in a write operation.
Definition: client.h:1177
A handle for a connection to a cluster.
Definition: client.h:289
An inclusive bound.
Definition: client.h:729
In-memory representation of a remote tablet server.
Definition: client.h:556
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:862
This class is a representation of a single scan.
Definition: client.h:1695
ReadMode
The read modes for scanners.
Definition: client.h:1698
A "factory" for KuduClient objects.
Definition: client.h:187
A single-row write operation to be sent to a Kudu table.
Definition: write_op.h:56
Select the LEADER replica.
Definition: client.h:419
RangePartitionBound
Range partition bound type.
Definition: client.h:727
In-memory representation of a remote tablet.
Definition: client.h:616
In-memory representation of a remote tablet&#39;s replica.
Definition: client.h:588
FlushMode
Modes of flush operations.
Definition: client.h:1287
A scan descriptor limited to a single physical contiguous location.
Definition: client.h:2113
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:646
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:1282
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