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 #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 ClientTest;
531  friend class KuduClientBuilder;
532  friend class KuduPartitionerBuilder;
533  friend class KuduScanner;
534  friend class KuduScanToken;
535  friend class KuduScanTokenBuilder;
536  friend class KuduSession;
537  friend class KuduTable;
538  friend class KuduTableAlterer;
539  friend class KuduTableCreator;
540  friend class ::kudu::SecurityUnknownTskTest;
541  friend class tools::LeaderMasterProxy;
542 
543  FRIEND_TEST(kudu::ClientStressTest, TestUniqueClientIds);
544  FRIEND_TEST(ClientTest, TestGetSecurityInfoFromMaster);
545  FRIEND_TEST(ClientTest, TestGetTabletServerBlacklist);
546  FRIEND_TEST(ClientTest, TestMasterDown);
547  FRIEND_TEST(ClientTest, TestMasterLookupPermits);
548  FRIEND_TEST(ClientTest, TestMetaCacheExpiry);
549  FRIEND_TEST(ClientTest, TestNonCoveringRangePartitions);
550  FRIEND_TEST(ClientTest, TestReplicatedTabletWritesWithLeaderElection);
551  FRIEND_TEST(ClientTest, TestScanFaultTolerance);
552  FRIEND_TEST(ClientTest, TestScanTimeout);
553  FRIEND_TEST(ClientTest, TestWriteWithDeadMaster);
554  FRIEND_TEST(MasterFailoverTest, TestPauseAfterCreateTableIssued);
555 
556  KuduClient();
557 
558  // Owned.
559  Data* data_;
560 
561  DISALLOW_COPY_AND_ASSIGN(KuduClient);
562 };
563 
565 class KUDU_EXPORT KuduTabletServer {
566  public:
567  ~KuduTabletServer();
568 
571  const std::string& uuid() const;
572 
575  const std::string& hostname() const;
576 
579  uint16_t port() const;
580 
581  private:
582  class KUDU_NO_EXPORT Data;
583 
584  friend class KuduClient;
585  friend class KuduScanner;
586  friend class KuduScanTokenBuilder;
587 
589 
590  // Owned.
591  Data* data_;
592 
593  DISALLOW_COPY_AND_ASSIGN(KuduTabletServer);
594 };
595 
597 class KUDU_EXPORT KuduReplica {
598  public:
599  ~KuduReplica();
600 
605  bool is_leader() const;
606 
608  const KuduTabletServer& ts() const;
609 
610  private:
611  friend class KuduClient;
612  friend class KuduScanTokenBuilder;
613  friend class internal::ReplicaController;
614 
615  class KUDU_NO_EXPORT Data;
616 
617  KuduReplica();
618 
619  // Owned.
620  Data* data_;
621 
622  DISALLOW_COPY_AND_ASSIGN(KuduReplica);
623 };
624 
626 class KUDU_EXPORT KuduTablet {
627  public:
628  ~KuduTablet();
629 
632  const std::string& id() const;
633 
639  const std::vector<const KuduReplica*>& replicas() const;
640 
641  private:
642  friend class KuduClient;
643  friend class KuduScanTokenBuilder;
644 
645  class KUDU_NO_EXPORT Data;
646 
647  KuduTablet();
648 
649  // Owned.
650  Data* data_;
651 
652  DISALLOW_COPY_AND_ASSIGN(KuduTablet);
653 };
654 
656 class KUDU_EXPORT KuduTableCreator {
657  public:
658  ~KuduTableCreator();
659 
671  KuduTableCreator& table_name(const std::string& name);
672 
683  KuduTableCreator& schema(const KuduSchema* schema);
684 
701  KuduTableCreator& add_hash_partitions(const std::vector<std::string>& columns,
702  int32_t num_buckets);
703 
719  KuduTableCreator& add_hash_partitions(const std::vector<std::string>& columns,
720  int32_t num_buckets, int32_t seed);
721 
734  KuduTableCreator& set_range_partition_columns(const std::vector<std::string>& columns);
735 
740  };
741 
770  KuduTableCreator& add_range_partition(KuduPartialRow* lower_bound,
771  KuduPartialRow* upper_bound,
772  RangePartitionBound lower_bound_type = INCLUSIVE_BOUND,
773  RangePartitionBound upper_bound_type = EXCLUSIVE_BOUND);
774 
783  KuduTableCreator& add_range_partition_split(KuduPartialRow* split_row);
784 
790  KuduTableCreator& split_rows(const std::vector<const KuduPartialRow*>& split_rows)
791  ATTRIBUTE_DEPRECATED("use add_range_partition_split() instead");
792 
802  KuduTableCreator& num_replicas(int n_replicas);
803 
813  KuduTableCreator& timeout(const MonoDelta& timeout);
814 
822  KuduTableCreator& wait(bool wait);
823 
837  Status Create();
838 
839  private:
840  class KUDU_NO_EXPORT Data;
841 
842  friend class KuduClient;
843 
844  explicit KuduTableCreator(KuduClient* client);
845 
846  // Owned.
847  Data* data_;
848 
849  DISALLOW_COPY_AND_ASSIGN(KuduTableCreator);
850 };
851 
872 class KUDU_EXPORT KuduTable : public sp::enable_shared_from_this<KuduTable> {
873  public:
874  ~KuduTable();
875 
877  const std::string& name() const;
878 
886  const std::string& id() const;
887 
889  const KuduSchema& schema() const;
890 
892  int num_replicas() const;
893 
897  KuduInsert* NewInsert();
898 
902  KuduUpsert* NewUpsert();
903 
907  KuduUpdate* NewUpdate();
908 
912  KuduDelete* NewDelete();
913 
937  KuduPredicate* NewComparisonPredicate(const Slice& col_name,
939  KuduValue* value);
940 
964  KuduPredicate* NewInListPredicate(const Slice& col_name,
965  std::vector<KuduValue*>* values);
966 
977  KuduPredicate* NewIsNotNullPredicate(const Slice& col_name);
978 
989  KuduPredicate* NewIsNullPredicate(const Slice& col_name);
990 
993  KuduClient* client() const;
994 
996  const PartitionSchema& partition_schema() const;
997 
998  private:
999  class KUDU_NO_EXPORT Data;
1000 
1001  friend class KuduClient;
1002  friend class KuduPartitioner;
1003 
1004  KuduTable(const sp::shared_ptr<KuduClient>& client,
1005  const std::string& name,
1006  const std::string& id,
1007  int num_replicas,
1008  const KuduSchema& schema,
1009  const PartitionSchema& partition_schema);
1010 
1011  // Owned.
1012  Data* data_;
1013 
1014  DISALLOW_COPY_AND_ASSIGN(KuduTable);
1015 };
1016 
1028 class KUDU_EXPORT KuduTableAlterer {
1029  public:
1030  ~KuduTableAlterer();
1031 
1037  KuduTableAlterer* RenameTo(const std::string& new_name);
1038 
1048  KuduColumnSpec* AddColumn(const std::string& name);
1049 
1058  KuduColumnSpec* AlterColumn(const std::string& name);
1059 
1067  KuduTableAlterer* DropColumn(const std::string& name);
1068 
1100  KuduTableAlterer* AddRangePartition(
1101  KuduPartialRow* lower_bound,
1102  KuduPartialRow* upper_bound,
1103  KuduTableCreator::RangePartitionBound lower_bound_type = KuduTableCreator::INCLUSIVE_BOUND,
1104  KuduTableCreator::RangePartitionBound upper_bound_type = KuduTableCreator::EXCLUSIVE_BOUND);
1105 
1133  KuduTableAlterer* DropRangePartition(
1134  KuduPartialRow* lower_bound,
1135  KuduPartialRow* upper_bound,
1136  KuduTableCreator::RangePartitionBound lower_bound_type = KuduTableCreator::INCLUSIVE_BOUND,
1137  KuduTableCreator::RangePartitionBound upper_bound_type = KuduTableCreator::EXCLUSIVE_BOUND);
1138 
1148  KuduTableAlterer* timeout(const MonoDelta& timeout);
1149 
1161  KuduTableAlterer* wait(bool wait);
1162 
1167  Status Alter();
1168 
1169  private:
1170  class KUDU_NO_EXPORT Data;
1171  friend class KuduClient;
1172 
1173  KuduTableAlterer(KuduClient* client,
1174  const std::string& name);
1175 
1176  // Owned.
1177  Data* data_;
1178 
1179  DISALLOW_COPY_AND_ASSIGN(KuduTableAlterer);
1180 };
1181 
1187 class KUDU_EXPORT KuduError {
1188  public:
1189  ~KuduError();
1190 
1192  const Status& status() const;
1193 
1195  const KuduWriteOperation& failed_op() const;
1196 
1204  KuduWriteOperation* release_failed_op();
1205 
1216  bool was_possibly_successful() const;
1217 
1218  private:
1219  class KUDU_NO_EXPORT Data;
1220 
1221  friend class internal::Batcher;
1222  friend class internal::ErrorCollector;
1223  friend class KuduSession;
1224 
1225  KuduError(KuduWriteOperation* failed_op, const Status& error);
1226 
1227  // Owned.
1228  Data* data_;
1229 
1230  DISALLOW_COPY_AND_ASSIGN(KuduError);
1231 };
1232 
1233 
1292 class KUDU_EXPORT KuduSession : public sp::enable_shared_from_this<KuduSession> {
1293  public:
1294  ~KuduSession();
1295 
1297  enum FlushMode {
1303 
1331 
1344  MANUAL_FLUSH
1345  };
1346 
1355  Status SetFlushMode(FlushMode m) WARN_UNUSED_RESULT;
1356 
1378 
1393  COMMIT_WAIT
1394  };
1395 
1401  Status SetExternalConsistencyMode(ExternalConsistencyMode m)
1402  WARN_UNUSED_RESULT;
1403 
1421  Status SetMutationBufferSpace(size_t size_bytes) WARN_UNUSED_RESULT;
1422 
1446  Status SetMutationBufferFlushWatermark(double watermark_pct)
1447  WARN_UNUSED_RESULT;
1448 
1470  Status SetMutationBufferFlushInterval(unsigned int millis) WARN_UNUSED_RESULT;
1471 
1496  Status SetMutationBufferMaxNum(unsigned int max_num) WARN_UNUSED_RESULT;
1497 
1503  void SetTimeoutMillis(int millis);
1504 
1508 
1532  Status Apply(KuduWriteOperation* write_op) WARN_UNUSED_RESULT;
1533 
1548  Status Flush() WARN_UNUSED_RESULT;
1549 
1590  void FlushAsync(KuduStatusCallback* cb);
1591 
1594  Status Close() WARN_UNUSED_RESULT;
1595 
1604  bool HasPendingOperations() const;
1605 
1627  int CountBufferedOperations() const
1628  ATTRIBUTE_DEPRECATED("this method is experimental and will disappear "
1629  "in a future release");
1630 
1655  Status SetErrorBufferSpace(size_t size_bytes);
1656 
1666  int CountPendingErrors() const;
1667 
1678  void GetPendingErrors(std::vector<KuduError*>* errors, bool* overflowed);
1679 
1681  KuduClient* client() const;
1682 
1683  private:
1684  class KUDU_NO_EXPORT Data;
1685 
1686  friend class KuduClient;
1687  friend class internal::Batcher;
1688  friend class ClientTest;
1689  FRIEND_TEST(ClientTest, TestAutoFlushBackgroundApplyBlocks);
1690  FRIEND_TEST(ClientTest, TestAutoFlushBackgroundAndErrorCollector);
1691 
1692  explicit KuduSession(const sp::shared_ptr<KuduClient>& client);
1693 
1694  // Owned.
1695  Data* data_;
1696 
1697  DISALLOW_COPY_AND_ASSIGN(KuduSession);
1698 };
1699 
1700 
1705 class KUDU_EXPORT KuduScanner {
1706  public:
1708  enum ReadMode {
1717 
1737  READ_AT_SNAPSHOT
1738  };
1739 
1743  enum OrderMode {
1750 
1755  ORDERED
1756  };
1757 
1761  enum { kScanTimeoutMillis = 30000 };
1762 
1768  explicit KuduScanner(KuduTable* table);
1769  ~KuduScanner();
1770 
1780  Status SetProjectedColumnNames(const std::vector<std::string>& col_names)
1781  WARN_UNUSED_RESULT;
1782 
1792  Status SetProjectedColumnIndexes(const std::vector<int>& col_indexes)
1793  WARN_UNUSED_RESULT;
1794 
1800  Status SetProjectedColumns(const std::vector<std::string>& col_names)
1801  WARN_UNUSED_RESULT
1802  ATTRIBUTE_DEPRECATED("use SetProjectedColumnNames() instead");
1803 
1812  Status AddConjunctPredicate(KuduPredicate* pred) WARN_UNUSED_RESULT;
1813 
1822  Status AddLowerBound(const KuduPartialRow& key);
1823 
1831  Status AddLowerBoundRaw(const Slice& key)
1832  ATTRIBUTE_DEPRECATED("use AddLowerBound() instead");
1833 
1842  Status AddExclusiveUpperBound(const KuduPartialRow& key);
1843 
1851  Status AddExclusiveUpperBoundRaw(const Slice& key)
1852  ATTRIBUTE_DEPRECATED("use AddExclusiveUpperBound() instead");
1853 
1862  Status AddLowerBoundPartitionKeyRaw(const Slice& partition_key);
1863 
1872  Status AddExclusiveUpperBoundPartitionKeyRaw(const Slice& partition_key);
1873 
1880  Status SetCacheBlocks(bool cache_blocks);
1881 
1883  Status Open();
1884 
1900  Status KeepAlive();
1901 
1910  void Close();
1911 
1920  bool HasMoreRows() const;
1921 
1933  Status NextBatch(std::vector<KuduRowResult>* rows)
1934  ATTRIBUTE_DEPRECATED("use NextBatch(KuduScanBatch*) instead");
1935 
1944  Status NextBatch(KuduScanBatch* batch);
1945 
1954  Status GetCurrentServer(KuduTabletServer** server);
1955 
1957  const ResourceMetrics& GetResourceMetrics() const;
1958 
1965  Status SetBatchSizeBytes(uint32_t batch_size);
1966 
1974  Status SetSelection(KuduClient::ReplicaSelection selection)
1975  WARN_UNUSED_RESULT;
1976 
1982  Status SetReadMode(ReadMode read_mode) WARN_UNUSED_RESULT;
1983 
1989  Status SetOrderMode(OrderMode order_mode) WARN_UNUSED_RESULT
1990  ATTRIBUTE_DEPRECATED("use SetFaultTolerant() instead");
1991 
2004  Status SetFaultTolerant() WARN_UNUSED_RESULT;
2005 
2011  Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros) WARN_UNUSED_RESULT;
2012 
2025  Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT;
2026 
2032  Status SetTimeoutMillis(int millis);
2033 
2035  KuduSchema GetProjectionSchema() const;
2036 
2038  //
2044  static const uint64_t NO_FLAGS = 0;
2052  static const uint64_t PAD_UNIXTIME_MICROS_TO_16_BYTES = 1 << 0;
2076  Status SetRowFormatFlags(uint64_t flags);
2078 
2084  std::string ToString() const;
2085 
2086  private:
2087  class KUDU_NO_EXPORT Data;
2088 
2089  friend class KuduScanToken;
2090  FRIEND_TEST(ClientTest, TestScanCloseProxy);
2091  FRIEND_TEST(ClientTest, TestScanFaultTolerance);
2092  FRIEND_TEST(ClientTest, TestScanNoBlockCaching);
2093  FRIEND_TEST(ClientTest, TestScanTimeout);
2094  FRIEND_TEST(ClientTest, TestReadAtSnapshotNoTimestampSet);
2095  FRIEND_TEST(ConsistencyITest, TestSnapshotScanTimestampReuse);
2096  FRIEND_TEST(ScanTokenTest, TestScanTokens);
2097 
2098  // Owned.
2099  Data* data_;
2100 
2101  DISALLOW_COPY_AND_ASSIGN(KuduScanner);
2102 };
2103 
2124 class KUDU_EXPORT KuduScanToken {
2125  public:
2126 
2127  ~KuduScanToken();
2128 
2139  Status IntoKuduScanner(KuduScanner** scanner) const WARN_UNUSED_RESULT;
2140 
2142  const KuduTablet& tablet() const;
2143 
2151  Status Serialize(std::string* buf) const WARN_UNUSED_RESULT;
2152 
2164  static Status DeserializeIntoScanner(KuduClient* client,
2165  const std::string& serialized_token,
2166  KuduScanner** scanner) WARN_UNUSED_RESULT;
2167 
2168  private:
2169  class KUDU_NO_EXPORT Data;
2170 
2171  friend class KuduScanTokenBuilder;
2172 
2173  KuduScanToken();
2174 
2175  // Owned.
2176  Data* data_;
2177 
2178  DISALLOW_COPY_AND_ASSIGN(KuduScanToken);
2179 };
2180 
2184 class KUDU_EXPORT KuduScanTokenBuilder {
2185  public:
2186 
2192  explicit KuduScanTokenBuilder(KuduTable* table);
2194 
2204  Status SetProjectedColumnNames(const std::vector<std::string>& col_names)
2205  WARN_UNUSED_RESULT;
2206 
2208  Status SetProjectedColumnIndexes(const std::vector<int>& col_indexes)
2209  WARN_UNUSED_RESULT;
2210 
2212  Status AddConjunctPredicate(KuduPredicate* pred) WARN_UNUSED_RESULT;
2213 
2215  Status AddLowerBound(const KuduPartialRow& key) WARN_UNUSED_RESULT;
2216 
2225  Status AddUpperBound(const KuduPartialRow& key) WARN_UNUSED_RESULT;
2226 
2228  Status SetCacheBlocks(bool cache_blocks) WARN_UNUSED_RESULT;
2229 
2236  Status SetBatchSizeBytes(uint32_t batch_size) WARN_UNUSED_RESULT;
2237 
2245  Status SetSelection(KuduClient::ReplicaSelection selection)
2246  WARN_UNUSED_RESULT;
2247 
2249  Status SetReadMode(KuduScanner::ReadMode read_mode) WARN_UNUSED_RESULT;
2250 
2252  Status SetFaultTolerant() WARN_UNUSED_RESULT;
2253 
2255  Status SetSnapshotMicros(uint64_t snapshot_timestamp_micros)
2256  WARN_UNUSED_RESULT;
2257 
2259  Status SetSnapshotRaw(uint64_t snapshot_timestamp) WARN_UNUSED_RESULT;
2260 
2262  Status SetTimeoutMillis(int millis) WARN_UNUSED_RESULT;
2263 
2272  Status Build(std::vector<KuduScanToken*>* tokens) WARN_UNUSED_RESULT;
2273 
2274  private:
2275  class KUDU_NO_EXPORT Data;
2276 
2277  // Owned.
2278  Data* data_;
2279 
2280  DISALLOW_COPY_AND_ASSIGN(KuduScanTokenBuilder);
2281 };
2282 
2284 class KUDU_EXPORT KuduPartitionerBuilder {
2285  public:
2290  explicit KuduPartitionerBuilder(sp::shared_ptr<KuduTable> table);
2292 
2294  KuduPartitionerBuilder* SetBuildTimeout(MonoDelta timeout);
2295 
2312  Status Build(KuduPartitioner** partitioner);
2313  private:
2314  class KUDU_NO_EXPORT Data;
2315 
2316  // Owned.
2317  Data* data_;
2318 
2319  DISALLOW_COPY_AND_ASSIGN(KuduPartitionerBuilder);
2320 };
2321 
2332 class KUDU_EXPORT KuduPartitioner {
2333  public:
2334  ~KuduPartitioner();
2335 
2339  int NumPartitions() const;
2340 
2352  Status PartitionRow(const KuduPartialRow& row, int* partition);
2353  private:
2354  class KUDU_NO_EXPORT Data;
2355  friend class KuduPartitionerBuilder;
2356 
2357  explicit KuduPartitioner(Data* data);
2358  Data* data_; // Owned.
2359 };
2360 
2361 
2362 } // namespace client
2363 } // namespace kudu
2364 #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:145
A constant cell value with a specific type.
Definition: value.h:34
Builder for Partitioner instances.
Definition: client.h:2284
ExternalConsistencyMode
The possible external consistency modes on which Kudu operates.
Definition: client.h:1358
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:240
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:2184
ComparisonOp
Supported comparison operators.
Definition: scan_predicate.h:37
Alters an existing table based on the provided steps.
Definition: client.h:1028
OrderMode
Definition: client.h:1743
Definition: client.h:1749
Smart pointer typedefs for externally-faced code.
Definition: client.h:2332
A representation of comparison predicate for Kudu queries.
Definition: scan_predicate.h:34
An exclusive bound.
Definition: client.h:738
This class represents an error which occurred in a write operation.
Definition: client.h:1187
A handle for a connection to a cluster.
Definition: client.h:298
An inclusive bound.
Definition: client.h:739
In-memory representation of a remote tablet server.
Definition: client.h:565
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:872
This class is a representation of a single scan.
Definition: client.h:1705
ReadMode
The read modes for scanners.
Definition: client.h:1708
A "factory" 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:737
In-memory representation of a remote tablet.
Definition: client.h:626
In-memory representation of a remote tablet&#39;s replica.
Definition: client.h:597
FlushMode
Modes of flush operations.
Definition: client.h:1297
A scan descriptor limited to a single physical contiguous location.
Definition: client.h:2124
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:656
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:1292
A batch of zero or more rows returned by a scan operation.
Definition: scan_batch.h:82
A representation of a time interval.
Definition: monotime.h:46