rippled
CassandraFactory.cpp
1 //------------------------------------------------------------------------------
2 /*
3  This file is part of rippled: https://github.com/ripple/rippled
4  Copyright (c) 2020 Ripple Labs Inc.
5 
6  Permission to use, copy, modify, and/or distribute this software for any
7  purpose with or without fee is hereby granted, provided that the above
8  copyright notice and this permission notice appear in all copies.
9 
10  THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
11  WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
12  MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
13  ANY SPECIAL , DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
14  WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
15  ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
16  OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
17 */
18 //==============================================================================
19 
20 #ifdef RIPPLED_REPORTING
21 
22 #include <cassandra.h>
23 #include <libpq-fe.h>
24 
25 #include <ripple/basics/Slice.h>
26 #include <ripple/basics/StringUtilities.h>
27 #include <ripple/basics/contract.h>
28 #include <ripple/basics/strHex.h>
29 #include <ripple/nodestore/Backend.h>
30 #include <ripple/nodestore/Factory.h>
31 #include <ripple/nodestore/Manager.h>
32 #include <ripple/nodestore/impl/DecodedBlob.h>
33 #include <ripple/nodestore/impl/EncodedBlob.h>
34 #include <ripple/nodestore/impl/codec.h>
35 #include <ripple/protocol/digest.h>
36 #include <boost/asio/steady_timer.hpp>
37 #include <boost/filesystem.hpp>
38 #include <atomic>
39 #include <cassert>
40 #include <chrono>
41 #include <cmath>
42 #include <cstdint>
43 #include <cstdio>
44 #include <cstring>
45 #include <exception>
46 #include <fstream>
47 #include <memory>
48 #include <mutex>
49 #include <nudb/nudb.hpp>
50 #include <queue>
51 #include <sstream>
52 #include <thread>
53 #include <utility>
54 #include <vector>
55 
56 namespace ripple {
57 namespace NodeStore {
58 
59 void
60 writeCallback(CassFuture* fut, void* cbData);
61 void
62 readCallback(CassFuture* fut, void* cbData);
63 
64 class CassandraBackend : public Backend
65 {
66 private:
67  // convenience function for one-off queries. For normal reads and writes,
68  // use the prepared statements insert_ and select_
69  CassStatement*
70  makeStatement(char const* query, std::size_t params)
71  {
72  CassStatement* ret = cass_statement_new(query, params);
73  CassError rc =
74  cass_statement_set_consistency(ret, CASS_CONSISTENCY_QUORUM);
75  if (rc != CASS_OK)
76  {
78  ss << "nodestore: Error setting query consistency: " << query
79  << ", result: " << rc << ", " << cass_error_desc(rc);
80  Throw<std::runtime_error>(ss.str());
81  }
82  return ret;
83  }
84 
85  beast::Journal const j_;
86  // size of a key
87  size_t const keyBytes_;
88 
89  Section const config_;
90 
91  std::atomic<bool> open_{false};
92 
93  // mutex used for open() and close()
94  std::mutex mutex_;
95 
96  std::unique_ptr<CassSession, void (*)(CassSession*)> session_{
97  nullptr,
98  [](CassSession* session) {
99  // Try to disconnect gracefully.
100  CassFuture* fut = cass_session_close(session);
101  cass_future_wait(fut);
102  cass_future_free(fut);
103  cass_session_free(session);
104  }};
105 
106  // Database statements cached server side. Using these is more efficient
107  // than making a new statement
108  const CassPrepared* insert_ = nullptr;
109  const CassPrepared* select_ = nullptr;
110 
111  // io_context used for exponential backoff for write retries
112  boost::asio::io_context ioContext_;
114  std::thread ioThread_;
115 
116  // maximum number of concurrent in flight requests. New requests will wait
117  // for earlier requests to finish if this limit is exceeded
118  uint32_t maxRequestsOutstanding = 10000000;
119  std::atomic_uint32_t numRequestsOutstanding_ = 0;
120 
121  // mutex and condition_variable to limit the number of concurrent in flight
122  // requests
123  std::mutex throttleMutex_;
124  std::condition_variable throttleCv_;
125 
126  // writes are asynchronous. This mutex and condition_variable is used to
127  // wait for all writes to finish
128  std::mutex syncMutex_;
129  std::condition_variable syncCv_;
130 
131  Counters<std::atomic<std::uint64_t>> counters_;
132 
133 public:
134  CassandraBackend(
135  size_t keyBytes,
136  Section const& keyValues,
137  beast::Journal journal)
138  : j_(journal), keyBytes_(keyBytes), config_(keyValues)
139  {
140  }
141 
142  ~CassandraBackend() override
143  {
144  close();
145  }
146 
148  getName() override
149  {
150  return "cassandra";
151  }
152 
153  bool
154  isOpen() override
155  {
156  return open_;
157  }
158 
159  // Setup all of the necessary components for talking to the database.
160  // Create the table if it doesn't exist already
161  // @param createIfMissing ignored
162  void
163  open(bool createIfMissing) override
164  {
165  if (open_)
166  {
167  assert(false);
168  JLOG(j_.error()) << "database is already open";
169  return;
170  }
171 
173  CassCluster* cluster = cass_cluster_new();
174  if (!cluster)
175  Throw<std::runtime_error>(
176  "nodestore:: Failed to create CassCluster");
177 
178  std::string secureConnectBundle =
179  get<std::string>(config_, "secure_connect_bundle");
180 
181  if (!secureConnectBundle.empty())
182  {
183  /* Setup driver to connect to the cloud using the secure connection
184  * bundle */
185  if (cass_cluster_set_cloud_secure_connection_bundle(
186  cluster, secureConnectBundle.c_str()) != CASS_OK)
187  {
188  JLOG(j_.error()) << "Unable to configure cloud using the "
189  "secure connection bundle: "
190  << secureConnectBundle;
191  Throw<std::runtime_error>(
192  "nodestore: Failed to connect using secure connection "
193  "bundle");
194  return;
195  }
196  }
197  else
198  {
199  std::string contact_points =
200  get<std::string>(config_, "contact_points");
201  if (contact_points.empty())
202  {
203  Throw<std::runtime_error>(
204  "nodestore: Missing contact_points in Cassandra config");
205  }
206  CassError rc = cass_cluster_set_contact_points(
207  cluster, contact_points.c_str());
208  if (rc != CASS_OK)
209  {
211  ss << "nodestore: Error setting Cassandra contact_points: "
212  << contact_points << ", result: " << rc << ", "
213  << cass_error_desc(rc);
214 
215  Throw<std::runtime_error>(ss.str());
216  }
217 
218  int port = get<int>(config_, "port");
219  if (port)
220  {
221  rc = cass_cluster_set_port(cluster, port);
222  if (rc != CASS_OK)
223  {
225  ss << "nodestore: Error setting Cassandra port: " << port
226  << ", result: " << rc << ", " << cass_error_desc(rc);
227 
228  Throw<std::runtime_error>(ss.str());
229  }
230  }
231  }
232  cass_cluster_set_token_aware_routing(cluster, cass_true);
233  CassError rc = cass_cluster_set_protocol_version(
234  cluster, CASS_PROTOCOL_VERSION_V4);
235  if (rc != CASS_OK)
236  {
238  ss << "nodestore: Error setting cassandra protocol version: "
239  << ", result: " << rc << ", " << cass_error_desc(rc);
240 
241  Throw<std::runtime_error>(ss.str());
242  }
243 
244  std::string username = get<std::string>(config_, "username");
245  if (username.size())
246  {
247  std::cout << "user = " << username.c_str() << " password = "
248  << get<std::string>(config_, "password").c_str()
249  << std::endl;
250  cass_cluster_set_credentials(
251  cluster,
252  username.c_str(),
253  get<std::string>(config_, "password").c_str());
254  }
255 
256  unsigned int const workers = std::thread::hardware_concurrency();
257  rc = cass_cluster_set_num_threads_io(cluster, workers);
258  if (rc != CASS_OK)
259  {
261  ss << "nodestore: Error setting Cassandra io threads to " << workers
262  << ", result: " << rc << ", " << cass_error_desc(rc);
263  Throw<std::runtime_error>(ss.str());
264  }
265 
266  cass_cluster_set_request_timeout(cluster, 2000);
267 
268  rc = cass_cluster_set_queue_size_io(
269  cluster,
270  maxRequestsOutstanding); // This number needs to scale w/ the
271  // number of request per sec
272  if (rc != CASS_OK)
273  {
275  ss << "nodestore: Error setting Cassandra max core connections per "
276  "host"
277  << ", result: " << rc << ", " << cass_error_desc(rc);
278  std::cout << ss.str() << std::endl;
279  return;
280  ;
281  }
282 
283  std::string certfile = get<std::string>(config_, "certfile");
284  if (certfile.size())
285  {
286  std::ifstream fileStream(
287  boost::filesystem::path(certfile).string(), std::ios::in);
288  if (!fileStream)
289  {
291  ss << "opening config file " << certfile;
292  Throw<std::system_error>(
293  errno, std::generic_category(), ss.str());
294  }
295  std::string cert(
296  std::istreambuf_iterator<char>{fileStream},
298  if (fileStream.bad())
299  {
301  ss << "reading config file " << certfile;
302  Throw<std::system_error>(
303  errno, std::generic_category(), ss.str());
304  }
305 
306  CassSsl* context = cass_ssl_new();
307  cass_ssl_set_verify_flags(context, CASS_SSL_VERIFY_NONE);
308  rc = cass_ssl_add_trusted_cert(context, cert.c_str());
309  if (rc != CASS_OK)
310  {
312  ss << "nodestore: Error setting Cassandra ssl context: " << rc
313  << ", " << cass_error_desc(rc);
314  Throw<std::runtime_error>(ss.str());
315  }
316 
317  cass_cluster_set_ssl(cluster, context);
318  cass_ssl_free(context);
319  }
320 
321  std::string keyspace = get<std::string>(config_, "keyspace");
322  if (keyspace.empty())
323  {
324  Throw<std::runtime_error>(
325  "nodestore: Missing keyspace in Cassandra config");
326  }
327 
328  std::string tableName = get<std::string>(config_, "table_name");
329  if (tableName.empty())
330  {
331  Throw<std::runtime_error>(
332  "nodestore: Missing table name in Cassandra config");
333  }
334 
335  cass_cluster_set_connect_timeout(cluster, 10000);
336 
337  CassStatement* statement;
338  CassFuture* fut;
339  bool setupSessionAndTable = false;
340  while (!setupSessionAndTable)
341  {
343  session_.reset(cass_session_new());
344  assert(session_);
345 
346  fut = cass_session_connect_keyspace(
347  session_.get(), cluster, keyspace.c_str());
348  rc = cass_future_error_code(fut);
349  cass_future_free(fut);
350  if (rc != CASS_OK)
351  {
353  ss << "nodestore: Error connecting Cassandra session keyspace: "
354  << rc << ", " << cass_error_desc(rc);
355  JLOG(j_.error()) << ss.str();
356  continue;
357  }
358 
359  std::stringstream query;
360  query << "CREATE TABLE IF NOT EXISTS " << tableName
361  << " ( hash blob PRIMARY KEY, object blob)";
362 
363  statement = makeStatement(query.str().c_str(), 0);
364  fut = cass_session_execute(session_.get(), statement);
365  rc = cass_future_error_code(fut);
366  cass_future_free(fut);
367  cass_statement_free(statement);
368  if (rc != CASS_OK && rc != CASS_ERROR_SERVER_INVALID_QUERY)
369  {
371  ss << "nodestore: Error creating Cassandra table: " << rc
372  << ", " << cass_error_desc(rc);
373  JLOG(j_.error()) << ss.str();
374  continue;
375  }
376 
377  query = {};
378  query << "SELECT * FROM " << tableName << " LIMIT 1";
379  statement = makeStatement(query.str().c_str(), 0);
380  fut = cass_session_execute(session_.get(), statement);
381  rc = cass_future_error_code(fut);
382  cass_future_free(fut);
383  cass_statement_free(statement);
384  if (rc != CASS_OK)
385  {
386  if (rc == CASS_ERROR_SERVER_INVALID_QUERY)
387  {
388  JLOG(j_.warn()) << "table not here yet, sleeping 1s to "
389  "see if table creation propagates";
390  continue;
391  }
392  else
393  {
395  ss << "nodestore: Error checking for table: " << rc << ", "
396  << cass_error_desc(rc);
397  JLOG(j_.error()) << ss.str();
398  continue;
399  }
400  }
401 
402  setupSessionAndTable = true;
403  }
404 
405  cass_cluster_free(cluster);
406 
407  bool setupPreparedStatements = false;
408  while (!setupPreparedStatements)
409  {
411  std::stringstream query;
412  query << "INSERT INTO " << tableName
413  << " (hash, object) VALUES (?, ?)";
414  CassFuture* prepare_future =
415  cass_session_prepare(session_.get(), query.str().c_str());
416 
417  /* Wait for the statement to prepare and get the result */
418  rc = cass_future_error_code(prepare_future);
419 
420  if (rc != CASS_OK)
421  {
422  /* Handle error */
423  cass_future_free(prepare_future);
424 
426  ss << "nodestore: Error preparing insert : " << rc << ", "
427  << cass_error_desc(rc);
428  JLOG(j_.error()) << ss.str();
429  continue;
430  }
431 
432  /* Get the prepared object from the future */
433  insert_ = cass_future_get_prepared(prepare_future);
434 
435  /* The future can be freed immediately after getting the prepared
436  * object
437  */
438  cass_future_free(prepare_future);
439 
440  query = {};
441  query << "SELECT object FROM " << tableName << " WHERE hash = ?";
442  prepare_future =
443  cass_session_prepare(session_.get(), query.str().c_str());
444 
445  /* Wait for the statement to prepare and get the result */
446  rc = cass_future_error_code(prepare_future);
447 
448  if (rc != CASS_OK)
449  {
450  /* Handle error */
451  cass_future_free(prepare_future);
452 
454  ss << "nodestore: Error preparing select : " << rc << ", "
455  << cass_error_desc(rc);
456  JLOG(j_.error()) << ss.str();
457  continue;
458  }
459 
460  /* Get the prepared object from the future */
461  select_ = cass_future_get_prepared(prepare_future);
462 
463  /* The future can be freed immediately after getting the prepared
464  * object
465  */
466  cass_future_free(prepare_future);
467  setupPreparedStatements = true;
468  }
469 
470  work_.emplace(ioContext_);
471  ioThread_ = std::thread{[this]() { ioContext_.run(); }};
472  open_ = true;
473 
474  if (config_.exists("max_requests_outstanding"))
475  {
476  maxRequestsOutstanding =
477  get<int>(config_, "max_requests_outstanding");
478  }
479  }
480 
481  // Close the connection to the database
482  void
483  close() override
484  {
485  {
487  if (insert_)
488  {
489  cass_prepared_free(insert_);
490  insert_ = nullptr;
491  }
492  if (select_)
493  {
494  cass_prepared_free(select_);
495  select_ = nullptr;
496  }
497  work_.reset();
498  ioThread_.join();
499  }
500  open_ = false;
501  }
502 
503  // Synchronously fetch the object with key key and store the result in pno
504  // @param key the key of the object
505  // @param pno object in which to store the result
506  // @return result status of query
507  Status
508  fetch(void const* key, std::shared_ptr<NodeObject>* pno) override
509  {
510  JLOG(j_.trace()) << "Fetching from cassandra";
511  CassStatement* statement = cass_prepared_bind(select_);
512  cass_statement_set_consistency(statement, CASS_CONSISTENCY_QUORUM);
513  CassError rc = cass_statement_bind_bytes(
514  statement, 0, static_cast<cass_byte_t const*>(key), keyBytes_);
515  if (rc != CASS_OK)
516  {
517  cass_statement_free(statement);
518  JLOG(j_.error()) << "Binding Cassandra fetch query: " << rc << ", "
519  << cass_error_desc(rc);
520  pno->reset();
521  return backendError;
522  }
523  CassFuture* fut;
524  do
525  {
526  fut = cass_session_execute(session_.get(), statement);
527  rc = cass_future_error_code(fut);
528  if (rc != CASS_OK)
529  {
531  ss << "Cassandra fetch error";
532  ss << ", retrying";
533  ++counters_.readRetries;
534  ss << ": " << cass_error_desc(rc);
535  JLOG(j_.warn()) << ss.str();
536  }
537  } while (rc != CASS_OK);
538 
539  CassResult const* res = cass_future_get_result(fut);
540  cass_statement_free(statement);
541  cass_future_free(fut);
542 
543  CassRow const* row = cass_result_first_row(res);
544  if (!row)
545  {
546  cass_result_free(res);
547  pno->reset();
548  return notFound;
549  }
550  cass_byte_t const* buf;
551  std::size_t bufSize;
552  rc = cass_value_get_bytes(cass_row_get_column(row, 0), &buf, &bufSize);
553  if (rc != CASS_OK)
554  {
555  cass_result_free(res);
556  pno->reset();
557  JLOG(j_.error()) << "Cassandra fetch result error: " << rc << ", "
558  << cass_error_desc(rc);
559  ++counters_.readErrors;
560  return backendError;
561  }
562 
563  nudb::detail::buffer bf;
565  nodeobject_decompress(buf, bufSize, bf);
566  DecodedBlob decoded(key, uncompressed.first, uncompressed.second);
567  cass_result_free(res);
568 
569  if (!decoded.wasOk())
570  {
571  pno->reset();
572  JLOG(j_.error()) << "Cassandra error decoding result: " << rc
573  << ", " << cass_error_desc(rc);
574  ++counters_.readErrors;
575  return dataCorrupt;
576  }
577  *pno = decoded.createObject();
578  return ok;
579  }
580 
581  struct ReadCallbackData
582  {
583  CassandraBackend& backend;
584  const void* const key;
587 
588  std::atomic_uint32_t& numFinished;
589  size_t batchSize;
590 
591  ReadCallbackData(
592  CassandraBackend& backend,
593  const void* const key,
596  std::atomic_uint32_t& numFinished,
597  size_t batchSize)
598  : backend(backend)
599  , key(key)
600  , result(result)
601  , cv(cv)
602  , numFinished(numFinished)
603  , batchSize(batchSize)
604  {
605  }
606 
607  ReadCallbackData(ReadCallbackData const& other) = default;
608  };
609 
611  fetchBatch(std::vector<uint256 const*> const& hashes) override
612  {
613  std::size_t const numHashes = hashes.size();
614  JLOG(j_.trace()) << "Fetching " << numHashes
615  << " records from Cassandra";
616  std::atomic_uint32_t numFinished = 0;
618  std::mutex mtx;
619  std::vector<std::shared_ptr<NodeObject>> results{numHashes};
621  cbs.reserve(numHashes);
622  for (std::size_t i = 0; i < hashes.size(); ++i)
623  {
624  cbs.push_back(std::make_shared<ReadCallbackData>(
625  *this,
626  static_cast<void const*>(hashes[i]),
627  results[i],
628  cv,
629  numFinished,
630  numHashes));
631  read(*cbs[i]);
632  }
633  assert(results.size() == cbs.size());
634 
636  cv.wait(lck, [&numFinished, &numHashes]() {
637  return numFinished == numHashes;
638  });
639 
640  JLOG(j_.trace()) << "Fetched " << numHashes
641  << " records from Cassandra";
642  return {results, ok};
643  }
644 
645  void
646  read(ReadCallbackData& data)
647  {
648  CassStatement* statement = cass_prepared_bind(select_);
649  cass_statement_set_consistency(statement, CASS_CONSISTENCY_QUORUM);
650  CassError rc = cass_statement_bind_bytes(
651  statement, 0, static_cast<cass_byte_t const*>(data.key), keyBytes_);
652  if (rc != CASS_OK)
653  {
654  size_t batchSize = data.batchSize;
655  if (++(data.numFinished) == batchSize)
656  data.cv.notify_all();
657  cass_statement_free(statement);
658  JLOG(j_.error()) << "Binding Cassandra fetch query: " << rc << ", "
659  << cass_error_desc(rc);
660  return;
661  }
662 
663  CassFuture* fut = cass_session_execute(session_.get(), statement);
664 
665  cass_statement_free(statement);
666 
667  cass_future_set_callback(fut, readCallback, static_cast<void*>(&data));
668  cass_future_free(fut);
669  }
670 
671  struct WriteCallbackData
672  {
673  CassandraBackend* backend;
674  // The shared pointer to the node object must exist until it's
675  // confirmed persisted. Otherwise, it can become deleted
676  // prematurely if other copies are removed from caches.
678  NodeStore::EncodedBlob e;
680  std::chrono::steady_clock::time_point begin;
681  // The data is stored in this buffer. The void* in the above member
682  // is a pointer into the below buffer
683  nudb::detail::buffer bf;
684  std::atomic<std::uint64_t>& totalWriteRetries;
685 
686  uint32_t currentRetries = 0;
687 
688  WriteCallbackData(
689  CassandraBackend* f,
690  std::shared_ptr<NodeObject> const& nobj,
692  : backend(f), no(nobj), totalWriteRetries(retries)
693  {
694  e.prepare(no);
695 
696  compressed =
697  NodeStore::nodeobject_compress(e.getData(), e.getSize(), bf);
698  }
699  };
700 
701  void
702  write(WriteCallbackData& data, bool isRetry)
703  {
704  {
705  // We limit the total number of concurrent inflight writes. This is
706  // a client side throttling to prevent overloading the database.
707  // This is mostly useful when the very first ledger is being written
708  // in full, which is several millions records. On sufficiently large
709  // Cassandra clusters, this throttling is not needed; the default
710  // value of maxRequestsOutstanding is 10 million, which is more
711  // records than are present in any single ledger
712  std::unique_lock<std::mutex> lck(throttleMutex_);
713  if (!isRetry && numRequestsOutstanding_ > maxRequestsOutstanding)
714  {
715  JLOG(j_.trace()) << __func__ << " : "
716  << "Max outstanding requests reached. "
717  << "Waiting for other requests to finish";
718  ++counters_.writesDelayed;
719  throttleCv_.wait(lck, [this]() {
720  return numRequestsOutstanding_ < maxRequestsOutstanding;
721  });
722  }
723  }
724 
725  CassStatement* statement = cass_prepared_bind(insert_);
726  cass_statement_set_consistency(statement, CASS_CONSISTENCY_QUORUM);
727  CassError rc = cass_statement_bind_bytes(
728  statement,
729  0,
730  static_cast<cass_byte_t const*>(data.e.getKey()),
731  keyBytes_);
732  if (rc != CASS_OK)
733  {
734  cass_statement_free(statement);
736  ss << "Binding cassandra insert hash: " << rc << ", "
737  << cass_error_desc(rc);
738  JLOG(j_.error()) << __func__ << " : " << ss.str();
739  Throw<std::runtime_error>(ss.str());
740  }
741  rc = cass_statement_bind_bytes(
742  statement,
743  1,
744  static_cast<cass_byte_t const*>(data.compressed.first),
745  data.compressed.second);
746  if (rc != CASS_OK)
747  {
748  cass_statement_free(statement);
750  ss << "Binding cassandra insert object: " << rc << ", "
751  << cass_error_desc(rc);
752  JLOG(j_.error()) << __func__ << " : " << ss.str();
753  Throw<std::runtime_error>(ss.str());
754  }
756  CassFuture* fut = cass_session_execute(session_.get(), statement);
757  cass_statement_free(statement);
758 
759  cass_future_set_callback(fut, writeCallback, static_cast<void*>(&data));
760  cass_future_free(fut);
761  }
762 
763  void
764  store(std::shared_ptr<NodeObject> const& no) override
765  {
766  JLOG(j_.trace()) << "Writing to cassandra";
767  WriteCallbackData* data =
768  new WriteCallbackData(this, no, counters_.writeRetries);
769 
770  ++numRequestsOutstanding_;
771  write(*data, false);
772  }
773 
774  void
775  storeBatch(Batch const& batch) override
776  {
777  for (auto const& no : batch)
778  {
779  store(no);
780  }
781  }
782 
783  void
784  sync() override
785  {
786  std::unique_lock<std::mutex> lck(syncMutex_);
787 
788  syncCv_.wait(lck, [this]() { return numRequestsOutstanding_ == 0; });
789  }
790 
791  // Iterate through entire table and execute f(). Used for import only,
792  // with database not being written to, so safe to paginate through
793  // objects table with LIMIT x OFFSET y.
794  void
796  {
797  assert(false);
798  Throw<std::runtime_error>("not implemented");
799  }
800 
801  int
802  getWriteLoad() override
803  {
804  return 0;
805  }
806 
807  void
808  setDeletePath() override
809  {
810  }
811 
812  int
813  fdRequired() const override
814  {
815  return 0;
816  }
817 
819  counters() const override
820  {
821  return counters_;
822  }
823 
824  friend void
825  writeCallback(CassFuture* fut, void* cbData);
826 
827  friend void
828  readCallback(CassFuture* fut, void* cbData);
829 };
830 
831 // Process the result of an asynchronous read. Retry on error
832 // @param fut cassandra future associated with the read
833 // @param cbData struct that holds the request parameters
834 void
835 readCallback(CassFuture* fut, void* cbData)
836 {
837  CassandraBackend::ReadCallbackData& requestParams =
838  *static_cast<CassandraBackend::ReadCallbackData*>(cbData);
839 
840  CassError rc = cass_future_error_code(fut);
841 
842  if (rc != CASS_OK)
843  {
844  ++(requestParams.backend.counters_.readRetries);
845  JLOG(requestParams.backend.j_.warn())
846  << "Cassandra fetch error : " << rc << " : " << cass_error_desc(rc)
847  << " - retrying";
848  // Retry right away. The only time the cluster should ever be overloaded
849  // is when the very first ledger is being written in full (millions of
850  // writes at once), during which no reads should be occurring. If reads
851  // are timing out, the code/architecture should be modified to handle
852  // greater read load, as opposed to just exponential backoff
853  requestParams.backend.read(requestParams);
854  }
855  else
856  {
857  auto finish = [&requestParams]() {
858  size_t batchSize = requestParams.batchSize;
859  if (++(requestParams.numFinished) == batchSize)
860  requestParams.cv.notify_all();
861  };
862  CassResult const* res = cass_future_get_result(fut);
863 
864  CassRow const* row = cass_result_first_row(res);
865  if (!row)
866  {
867  cass_result_free(res);
868  JLOG(requestParams.backend.j_.error())
869  << "Cassandra fetch get row error : " << rc << ", "
870  << cass_error_desc(rc);
871  finish();
872  return;
873  }
874  cass_byte_t const* buf;
875  std::size_t bufSize;
876  rc = cass_value_get_bytes(cass_row_get_column(row, 0), &buf, &bufSize);
877  if (rc != CASS_OK)
878  {
879  cass_result_free(res);
880  JLOG(requestParams.backend.j_.error())
881  << "Cassandra fetch get bytes error : " << rc << ", "
882  << cass_error_desc(rc);
883  ++requestParams.backend.counters_.readErrors;
884  finish();
885  return;
886  }
887  nudb::detail::buffer bf;
889  nodeobject_decompress(buf, bufSize, bf);
890  DecodedBlob decoded(
891  requestParams.key, uncompressed.first, uncompressed.second);
892  cass_result_free(res);
893 
894  if (!decoded.wasOk())
895  {
896  JLOG(requestParams.backend.j_.fatal())
897  << "Cassandra fetch error - data corruption : " << rc << ", "
898  << cass_error_desc(rc);
899  ++requestParams.backend.counters_.readErrors;
900  finish();
901  return;
902  }
903  requestParams.result = decoded.createObject();
904  finish();
905  }
906 }
907 
908 // Process the result of an asynchronous write. Retry on error
909 // @param fut cassandra future associated with the write
910 // @param cbData struct that holds the request parameters
911 void
912 writeCallback(CassFuture* fut, void* cbData)
913 {
914  CassandraBackend::WriteCallbackData& requestParams =
915  *static_cast<CassandraBackend::WriteCallbackData*>(cbData);
916  CassandraBackend& backend = *requestParams.backend;
917  auto rc = cass_future_error_code(fut);
918  if (rc != CASS_OK)
919  {
920  JLOG(backend.j_.error())
921  << "ERROR!!! Cassandra insert error: " << rc << ", "
922  << cass_error_desc(rc) << ", retrying ";
923  ++requestParams.totalWriteRetries;
924  // exponential backoff with a max wait of 2^10 ms (about 1 second)
925  auto wait = std::chrono::milliseconds(
926  lround(std::pow(2, std::min(10u, requestParams.currentRetries))));
927  ++requestParams.currentRetries;
929  std::make_shared<boost::asio::steady_timer>(
930  backend.ioContext_, std::chrono::steady_clock::now() + wait);
931  timer->async_wait([timer, &requestParams, &backend](
932  const boost::system::error_code& error) {
933  backend.write(requestParams, true);
934  });
935  }
936  else
937  {
938  backend.counters_.writeDurationUs +=
939  std::chrono::duration_cast<std::chrono::microseconds>(
940  std::chrono::steady_clock::now() - requestParams.begin)
941  .count();
942  --(backend.numRequestsOutstanding_);
943 
944  backend.throttleCv_.notify_all();
945  if (backend.numRequestsOutstanding_ == 0)
946  backend.syncCv_.notify_all();
947  delete &requestParams;
948  }
949 }
950 
951 //------------------------------------------------------------------------------
952 
953 class CassandraFactory : public Factory
954 {
955 public:
956  CassandraFactory()
957  {
958  Manager::instance().insert(*this);
959  }
960 
961  ~CassandraFactory() override
962  {
963  Manager::instance().erase(*this);
964  }
965 
967  getName() const override
968  {
969  return "cassandra";
970  }
971 
973  createInstance(
974  size_t keyBytes,
975  Section const& keyValues,
976  std::size_t burstSize,
977  Scheduler& scheduler,
978  beast::Journal journal) override
979  {
980  return std::make_unique<CassandraBackend>(keyBytes, keyValues, journal);
981  }
982 };
983 
984 static CassandraFactory cassandraFactory;
985 
986 } // namespace NodeStore
987 } // namespace ripple
988 #endif
ripple::NodeStore::nodeobject_decompress
std::pair< void const *, std::size_t > nodeobject_decompress(void const *in, std::size_t in_size, BufferFactory &&bf)
Definition: codec.h:100
sstream
std::this_thread::sleep_for
T sleep_for(T... args)
std::lock
T lock(T... args)
std::for_each
T for_each(T... args)
fstream
ripple::NodeStore::read
void read(nudb::detail::istream &is, std::size_t &u)
Definition: varint.h:120
std::string
STL class.
std::shared_ptr< NodeObject >
utility
exception
cstring
beast::Journal::trace
Stream trace() const
Severity stream access functions.
Definition: Journal.h:309
ripple::NodeStore::ok
@ ok
Definition: nodestore/Types.h:45
ripple::NodeStore::Manager::erase
virtual void erase(Factory &factory)=0
Remove a factory.
std::pair
std::vector::reserve
T reserve(T... args)
vector
std::string::size
T size(T... args)
std::chrono::seconds
std::optional::emplace
T emplace(T... args)
std::stringstream
STL class.
beast::Journal::warn
Stream warn() const
Definition: Journal.h:327
std::lock_guard
STL class.
std::function
ripple::NodeStore::backendError
@ backendError
Definition: nodestore/Types.h:49
queue
cmath
ripple::NodeStore::write
void write(nudb::detail::ostream &os, std::size_t t)
Definition: varint.h:133
std::optional::reset
T reset(T... args)
std::vector::push_back
T push_back(T... args)
ripple::NodeStore::notFound
@ notFound
Definition: nodestore/Types.h:46
std::cout
ripple::NodeStore::Batch
std::vector< std::shared_ptr< NodeObject > > Batch
A batch of NodeObjects to write at once.
Definition: nodestore/Types.h:55
ripple::NodeStore::Manager::insert
virtual void insert(Factory &factory)=0
Add a factory.
thread
std::thread::hardware_concurrency
T hardware_concurrency(T... args)
std::generic_category
T generic_category(T... args)
chrono
std::string::c_str
T c_str(T... args)
std::unique_lock
STL class.
beast::Journal::error
Stream error() const
Definition: Journal.h:333
cstdint
beast::Journal
A generic endpoint for log messages.
Definition: Journal.h:58
ripple::NodeStore::dataCorrupt
@ dataCorrupt
Definition: nodestore/Types.h:47
std::condition_variable::wait
T wait(T... args)
atomic
std::istreambuf_iterator
memory
std::min
T min(T... args)
ripple::NodeStore::Status
Status
Return codes from Backend operations.
Definition: nodestore/Types.h:44
ripple
Use hash_* containers for keys that do not need a cryptographically secure hashing algorithm.
Definition: RCLCensorshipDetector.h:29
std::lround
T lround(T... args)
std::endl
T endl(T... args)
std::begin
T begin(T... args)
cassert
std::condition_variable
std::string::empty
T empty(T... args)
std::optional
mutex
std::stringstream::str
T str(T... args)
std::size_t
ripple::NodeStore::nodeobject_compress
std::pair< void const *, std::size_t > nodeobject_compress(void const *in, std::size_t in_size, BufferFactory &&bf)
Definition: codec.h:211
ripple::NodeStore::Manager::instance
static Manager & instance()
Returns the instance of the manager singleton.
Definition: ManagerImp.cpp:120
std::unique_ptr
STL class.
std::data
T data(T... args)
cstdio
std::thread::join
T join(T... args)
ripple::open
void open(soci::session &s, BasicConfig const &config, std::string const &dbName)
Open a soci session.
Definition: SociDB.cpp:98
std::ifstream
STL class.
std::pow
T pow(T... args)
std::chrono::steady_clock::now
T now(T... args)