write_prepared_txn_db.h 24.7 KB
Newer Older
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16
//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
//  This source code is licensed under both the GPLv2 (found in the
//  COPYING file in the root directory) and Apache 2.0 License
//  (found in the LICENSE.Apache file in the root directory).

#pragma once
#ifndef ROCKSDB_LITE

#include <mutex>
#include <queue>
#include <set>
#include <string>
#include <unordered_map>
#include <vector>

#include "db/db_iter.h"
17
#include "db/pre_release_callback.h"
18 19 20 21 22
#include "db/read_callback.h"
#include "db/snapshot_checker.h"
#include "rocksdb/db.h"
#include "rocksdb/options.h"
#include "rocksdb/utilities/transaction_db.h"
23
#include "util/set_comparator.h"
24
#include "util/string_util.h"
25 26 27 28 29 30 31
#include "utilities/transactions/pessimistic_transaction.h"
#include "utilities/transactions/pessimistic_transaction_db.h"
#include "utilities/transactions/transaction_lock_mgr.h"
#include "utilities/transactions/write_prepared_txn.h"

namespace rocksdb {

32 33 34 35
#define ROCKS_LOG_DETAILS(LGR, FMT, ...) \
  ;  // due to overhead by default skip such lines
// ROCKS_LOG_DEBUG(LGR, FMT, ##__VA_ARGS__)

36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76
// A PessimisticTransactionDB that writes data to DB after prepare phase of 2PC.
// In this way some data in the DB might not be committed. The DB provides
// mechanisms to tell such data apart from committed data.
class WritePreparedTxnDB : public PessimisticTransactionDB {
 public:
  explicit WritePreparedTxnDB(
      DB* db, const TransactionDBOptions& txn_db_options,
      size_t snapshot_cache_bits = DEF_SNAPSHOT_CACHE_BITS,
      size_t commit_cache_bits = DEF_COMMIT_CACHE_BITS)
      : PessimisticTransactionDB(db, txn_db_options),
        SNAPSHOT_CACHE_BITS(snapshot_cache_bits),
        SNAPSHOT_CACHE_SIZE(static_cast<size_t>(1ull << SNAPSHOT_CACHE_BITS)),
        COMMIT_CACHE_BITS(commit_cache_bits),
        COMMIT_CACHE_SIZE(static_cast<size_t>(1ull << COMMIT_CACHE_BITS)),
        FORMAT(COMMIT_CACHE_BITS) {
    Init(txn_db_options);
  }

  explicit WritePreparedTxnDB(
      StackableDB* db, const TransactionDBOptions& txn_db_options,
      size_t snapshot_cache_bits = DEF_SNAPSHOT_CACHE_BITS,
      size_t commit_cache_bits = DEF_COMMIT_CACHE_BITS)
      : PessimisticTransactionDB(db, txn_db_options),
        SNAPSHOT_CACHE_BITS(snapshot_cache_bits),
        SNAPSHOT_CACHE_SIZE(static_cast<size_t>(1ull << SNAPSHOT_CACHE_BITS)),
        COMMIT_CACHE_BITS(commit_cache_bits),
        COMMIT_CACHE_SIZE(static_cast<size_t>(1ull << COMMIT_CACHE_BITS)),
        FORMAT(COMMIT_CACHE_BITS) {
    Init(txn_db_options);
  }

  virtual ~WritePreparedTxnDB();

  virtual Status Initialize(
      const std::vector<size_t>& compaction_enabled_cf_indices,
      const std::vector<ColumnFamilyHandle*>& handles) override;

  Transaction* BeginTransaction(const WriteOptions& write_options,
                                const TransactionOptions& txn_options,
                                Transaction* old_txn) override;

77 78
  // Optimized version of ::Write that receives more optimization request such
  // as skip_concurrency_control.
79 80 81 82 83 84 85 86
  using PessimisticTransactionDB::Write;
  Status Write(const WriteOptions& opts, const TransactionDBWriteOptimizations&,
               WriteBatch* updates) override;

  // Write the batch to the underlying DB and mark it as committed. Could be
  // used by both directly from TxnDB or through a transaction.
  Status WriteInternal(const WriteOptions& write_options, WriteBatch* batch,
                       size_t batch_cnt, WritePreparedTxn* txn);
87

88 89 90 91 92
  using DB::Get;
  virtual Status Get(const ReadOptions& options,
                     ColumnFamilyHandle* column_family, const Slice& key,
                     PinnableSlice* value) override;

93 94 95 96 97 98 99
  using DB::MultiGet;
  virtual std::vector<Status> MultiGet(
      const ReadOptions& options,
      const std::vector<ColumnFamilyHandle*>& column_family,
      const std::vector<Slice>& keys,
      std::vector<std::string>* values) override;

100 101 102 103 104 105 106 107 108 109
  using DB::NewIterator;
  virtual Iterator* NewIterator(const ReadOptions& options,
                                ColumnFamilyHandle* column_family) override;

  using DB::NewIterators;
  virtual Status NewIterators(
      const ReadOptions& options,
      const std::vector<ColumnFamilyHandle*>& column_families,
      std::vector<Iterator*>* iterators) override;

110 111
  virtual void ReleaseSnapshot(const Snapshot* snapshot) override;

112
  // Check whether the transaction that wrote the value with sequence number seq
113 114
  // is visible to the snapshot with sequence number snapshot_seq
  bool IsInSnapshot(uint64_t seq, uint64_t snapshot_seq) const;
115
  // Add the transaction with prepare sequence seq to the prepared list
116 117 118
  void AddPrepared(uint64_t seq);
  // Rollback a prepared txn identified with prep_seq. rollback_seq is the seq
  // with which the additional data is written to cancel the txn effect. It can
119
  // be used to identify the snapshots that overlap with the rolled back txn.
120 121
  void RollbackPrepared(uint64_t prep_seq, uint64_t rollback_seq);
  // Add the transaction with prepare sequence prepare_seq and commit sequence
122
  // commit_seq to the commit map. prepare_skipped is set if the prepare phase
123 124 125
  // is skipped for this commit. loop_cnt is to detect infinite loops.
  void AddCommitted(uint64_t prepare_seq, uint64_t commit_seq,
                    bool prepare_skipped = false, uint8_t loop_cnt = 0);
126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141

  struct CommitEntry {
    uint64_t prep_seq;
    uint64_t commit_seq;
    CommitEntry() : prep_seq(0), commit_seq(0) {}
    CommitEntry(uint64_t ps, uint64_t cs) : prep_seq(ps), commit_seq(cs) {}
    bool operator==(const CommitEntry& rhs) const {
      return prep_seq == rhs.prep_seq && commit_seq == rhs.commit_seq;
    }
  };

  struct CommitEntry64bFormat {
    explicit CommitEntry64bFormat(size_t index_bits)
        : INDEX_BITS(index_bits),
          PREP_BITS(static_cast<size_t>(64 - PAD_BITS - INDEX_BITS)),
          COMMIT_BITS(static_cast<size_t>(64 - PREP_BITS)),
142 143
          COMMIT_FILTER(static_cast<uint64_t>((1ull << COMMIT_BITS) - 1)),
          DELTA_UPPERBOUND(static_cast<uint64_t>((1ull << COMMIT_BITS))) {}
144 145 146 147 148 149 150 151 152 153 154 155
    // Number of higher bits of a sequence number that is not used. They are
    // used to encode the value type, ...
    const size_t PAD_BITS = static_cast<size_t>(8);
    // Number of lower bits from prepare seq that can be skipped as they are
    // implied by the index of the entry in the array
    const size_t INDEX_BITS;
    // Number of bits we use to encode the prepare seq
    const size_t PREP_BITS;
    // Number of bits we use to encode the commit seq.
    const size_t COMMIT_BITS;
    // Filter to encode/decode commit seq
    const uint64_t COMMIT_FILTER;
156 157
    // The value of commit_seq - prepare_seq + 1 must be less than this bound
    const uint64_t DELTA_UPPERBOUND;
158 159 160
  };

  // Prepare Seq (64 bits) = PAD ... PAD PREP PREP ... PREP INDEX INDEX ...
161
  // INDEX Delta Seq (64 bits)   = 0 0 0 0 0 0 0 0 0  0 0 0 DELTA DELTA ...
162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181
  // DELTA DELTA Encoded Value         = PREP PREP .... PREP PREP DELTA DELTA
  // ... DELTA DELTA PAD: first bits of a seq that is reserved for tagging and
  // hence ignored PREP/INDEX: the used bits in a prepare seq number INDEX: the
  // bits that do not have to be encoded (will be provided externally) DELTA:
  // prep seq - commit seq + 1 Number of DELTA bits should be equal to number of
  // index bits + PADs
  struct CommitEntry64b {
    constexpr CommitEntry64b() noexcept : rep_(0) {}

    CommitEntry64b(const CommitEntry& entry, const CommitEntry64bFormat& format)
        : CommitEntry64b(entry.prep_seq, entry.commit_seq, format) {}

    CommitEntry64b(const uint64_t ps, const uint64_t cs,
                   const CommitEntry64bFormat& format) {
      assert(ps < static_cast<uint64_t>(
                      (1ull << (format.PREP_BITS + format.INDEX_BITS))));
      assert(ps <= cs);
      uint64_t delta = cs - ps + 1;  // make initialized delta always >= 1
      // zero is reserved for uninitialized entries
      assert(0 < delta);
182 183 184 185 186 187 188
      assert(delta < format.DELTA_UPPERBOUND);
      if (delta >= format.DELTA_UPPERBOUND) {
        throw std::runtime_error(
            "commit_seq >> prepare_seq. The allowed distance is " +
            ToString(format.DELTA_UPPERBOUND) + " commit_seq is " +
            ToString(cs) + " prepare_seq is " + ToString(ps));
      }
189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219
      rep_ = (ps << format.PAD_BITS) & ~format.COMMIT_FILTER;
      rep_ = rep_ | delta;
    }

    // Return false if the entry is empty
    bool Parse(const uint64_t indexed_seq, CommitEntry* entry,
               const CommitEntry64bFormat& format) {
      uint64_t delta = rep_ & format.COMMIT_FILTER;
      // zero is reserved for uninitialized entries
      assert(delta < static_cast<uint64_t>((1ull << format.COMMIT_BITS)));
      if (delta == 0) {
        return false;  // initialized entry would have non-zero delta
      }

      assert(indexed_seq < static_cast<uint64_t>((1ull << format.INDEX_BITS)));
      uint64_t prep_up = rep_ & ~format.COMMIT_FILTER;
      prep_up >>= format.PAD_BITS;
      const uint64_t& prep_low = indexed_seq;
      entry->prep_seq = prep_up | prep_low;

      entry->commit_seq = entry->prep_seq + delta - 1;
      return true;
    }

   private:
    uint64_t rep_;
  };

  // Struct to hold ownership of snapshot and read callback for cleanup.
  struct IteratorState;

220 221 222 223 224 225 226
  std::map<uint32_t, const Comparator*>* GetCFComparatorMap() {
    return cf_map_.load();
  }
  void UpdateCFComparatorMap(
      const std::vector<ColumnFamilyHandle*>& handles) override;
  void UpdateCFComparatorMap(const ColumnFamilyHandle* handle) override;

227 228 229 230
 protected:
  virtual Status VerifyCFOptions(
      const ColumnFamilyOptions& cf_options) override;

231 232 233 234
 private:
  friend class WritePreparedTransactionTest_IsInSnapshotTest_Test;
  friend class WritePreparedTransactionTest_CheckAgainstSnapshotsTest_Test;
  friend class WritePreparedTransactionTest_CommitMapTest_Test;
235 236
  friend class
      WritePreparedTransactionTest_ConflictDetectionAfterRecoveryTest_Test;
237 238
  friend class SnapshotConcurrentAccessTest_SnapshotConcurrentAccessTest_Test;
  friend class WritePreparedTransactionTestBase;
239
  friend class PreparedHeap_BasicsTest_Test;
240 241
  friend class PreparedHeap_EmptyAtTheEnd_Test;
  friend class PreparedHeap_Concurrent_Test;
242 243
  friend class WritePreparedTxnDBMock;
  friend class WritePreparedTransactionTest_AdvanceMaxEvictedSeqBasicTest_Test;
244 245
  friend class
      WritePreparedTransactionTest_AdvanceMaxEvictedSeqWithDuplicatesTest_Test;
246 247
  friend class WritePreparedTransactionTest_BasicRecoveryTest_Test;
  friend class WritePreparedTransactionTest_IsInSnapshotEmptyMapTest_Test;
248
  friend class WritePreparedTransactionTest_OldCommitMapGC_Test;
249 250 251 252 253 254 255 256 257 258 259
  friend class WritePreparedTransactionTest_RollbackTest_Test;

  void Init(const TransactionDBOptions& /* unused */);

  // A heap with the amortized O(1) complexity for erase. It uses one extra heap
  // to keep track of erased entries that are not yet on top of the main heap.
  class PreparedHeap {
    std::priority_queue<uint64_t, std::vector<uint64_t>, std::greater<uint64_t>>
        heap_;
    std::priority_queue<uint64_t, std::vector<uint64_t>, std::greater<uint64_t>>
        erased_heap_;
260 261 262
    // True when testing crash recovery
    bool TEST_CRASH_ = false;
    friend class WritePreparedTxnDB;
263 264

   public:
265 266 267 268 269 270
    ~PreparedHeap() {
      if (!TEST_CRASH_) {
        assert(heap_.empty());
        assert(erased_heap_.empty());
      }
    }
271 272 273 274 275 276
    bool empty() { return heap_.empty(); }
    uint64_t top() { return heap_.top(); }
    void push(uint64_t v) { heap_.push(v); }
    void pop() {
      heap_.pop();
      while (!heap_.empty() && !erased_heap_.empty() &&
277 278
             // heap_.top() > erased_heap_.top() could happen if we have erased
             // a non-existent entry. Ideally the user should not do that but we
279
             // should be resilient against it.
280 281 282 283
             heap_.top() >= erased_heap_.top()) {
        if (heap_.top() == erased_heap_.top()) {
          heap_.pop();
        }
D
Dmitri Smirnov 已提交
284 285
        uint64_t erased __attribute__((__unused__));
        erased = erased_heap_.top();
286
        erased_heap_.pop();
287 288
        // No duplicate prepare sequence numbers
        assert(erased_heap_.empty() || erased_heap_.top() != erased);
289 290 291 292 293 294 295 296 297 298 299
      }
      while (heap_.empty() && !erased_heap_.empty()) {
        erased_heap_.pop();
      }
    }
    void erase(uint64_t seq) {
      if (!heap_.empty()) {
        if (seq < heap_.top()) {
          // Already popped, ignore it.
        } else if (heap_.top() == seq) {
          pop();
300
          assert(heap_.empty() || heap_.top() != seq);
301 302 303 304 305 306 307 308
        } else {  // (heap_.top() > seq)
          // Down the heap, remember to pop it later
          erased_heap_.push(seq);
        }
      }
    }
  };

309 310
  void TEST_Crash() override { prepared_txns_.TEST_CRASH_ = true; }

311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334
  // Get the commit entry with index indexed_seq from the commit table. It
  // returns true if such entry exists.
  bool GetCommitEntry(const uint64_t indexed_seq, CommitEntry64b* entry_64b,
                      CommitEntry* entry) const;

  // Rewrite the entry with the index indexed_seq in the commit table with the
  // commit entry <prep_seq, commit_seq>. If the rewrite results into eviction,
  // sets the evicted_entry and returns true.
  bool AddCommitEntry(const uint64_t indexed_seq, const CommitEntry& new_entry,
                      CommitEntry* evicted_entry);

  // Rewrite the entry with the index indexed_seq in the commit table with the
  // commit entry new_entry only if the existing entry matches the
  // expected_entry. Returns false otherwise.
  bool ExchangeCommitEntry(const uint64_t indexed_seq,
                           CommitEntry64b& expected_entry,
                           const CommitEntry& new_entry);

  // Increase max_evicted_seq_ from the previous value prev_max to the new
  // value. This also involves taking care of prepared txns that are not
  // committed before new_max, as well as updating the list of live snapshots at
  // the time of updating the max. Thread-safety: this function can be called
  // concurrently. The concurrent invocations of this function is equivalent to
  // a serial invocation in which the last invocation is the one with the
335
  // largest new_max value.
336 337
  void AdvanceMaxEvictedSeq(const SequenceNumber& prev_max,
                            const SequenceNumber& new_max);
338 339 340 341

  virtual const std::vector<SequenceNumber> GetSnapshotListFromDB(
      SequenceNumber max);

342 343 344 345
  // Will be called by the public ReleaseSnapshot method. Does the maintenance
  // internal to WritePreparedTxnDB
  void ReleaseSnapshotInternal(const SequenceNumber snap_seq);

346
  // Update the list of snapshots corresponding to the soon-to-be-updated
347
  // max_evicted_seq_. Thread-safety: this function can be called concurrently.
348
  // The concurrent invocations of this function is equivalent to a serial
349
  // invocation in which the last invocation is the one with the largest
350 351 352 353 354 355 356 357 358 359 360 361 362 363
  // version value.
  void UpdateSnapshots(const std::vector<SequenceNumber>& snapshots,
                       const SequenceNumber& version);

  // Check an evicted entry against live snapshots to see if it should be kept
  // around or it can be safely discarded (and hence assume committed for all
  // snapshots). Thread-safety: this function can be called concurrently. If it
  // is called concurrently with multiple UpdateSnapshots, the result is the
  // same as checking the intersection of the snapshot list before updates with
  // the snapshot list of all the concurrent updates.
  void CheckAgainstSnapshots(const CommitEntry& evicted);

  // Add a new entry to old_commit_map_ if prep_seq <= snapshot_seq <
  // commit_seq. Return false if checking the next snapshot(s) is not needed.
364 365
  // This is the case if none of the next snapshots could satisfy the condition.
  // next_is_larger: the next snapshot will be a larger value
366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387
  bool MaybeUpdateOldCommitMap(const uint64_t& prep_seq,
                               const uint64_t& commit_seq,
                               const uint64_t& snapshot_seq,
                               const bool next_is_larger);

  // The list of live snapshots at the last time that max_evicted_seq_ advanced.
  // The list stored into two data structures: in snapshot_cache_ that is
  // efficient for concurrent reads, and in snapshots_ if the data does not fit
  // into snapshot_cache_. The total number of snapshots in the two lists
  std::atomic<size_t> snapshots_total_ = {};
  // The list sorted in ascending order. Thread-safety for writes is provided
  // with snapshots_mutex_ and concurrent reads are safe due to std::atomic for
  // each entry. In x86_64 architecture such reads are compiled to simple read
  // instructions. 128 entries
  static const size_t DEF_SNAPSHOT_CACHE_BITS = static_cast<size_t>(7);
  const size_t SNAPSHOT_CACHE_BITS;
  const size_t SNAPSHOT_CACHE_SIZE;
  unique_ptr<std::atomic<SequenceNumber>[]> snapshot_cache_;
  // 2nd list for storing snapshots. The list sorted in ascending order.
  // Thread-safety is provided with snapshots_mutex_.
  std::vector<SequenceNumber> snapshots_;
  // The version of the latest list of snapshots. This can be used to avoid
388
  // rewriting a list that is concurrently updated with a more recent version.
389 390 391 392 393
  SequenceNumber snapshots_version_ = 0;

  // A heap of prepared transactions. Thread-safety is provided with
  // prepared_mutex_.
  PreparedHeap prepared_txns_;
394 395
  // 8m entry, 64MB size
  static const size_t DEF_COMMIT_CACHE_BITS = static_cast<size_t>(23);
396 397 398 399 400 401
  const size_t COMMIT_CACHE_BITS;
  const size_t COMMIT_CACHE_SIZE;
  const CommitEntry64bFormat FORMAT;
  // commit_cache_ must be initialized to zero to tell apart an empty index from
  // a filled one. Thread-safety is provided with commit_cache_mutex_.
  unique_ptr<std::atomic<CommitEntry64b>[]> commit_cache_;
402 403 404 405
  // The largest evicted *commit* sequence number from the commit_cache_. If a
  // seq is smaller than max_evicted_seq_ is might or might not be present in
  // commit_cache_. So commit_cache_ must first be checked before consulting
  // with max_evicted_seq_.
406 407 408 409 410 411
  std::atomic<uint64_t> max_evicted_seq_ = {};
  // Advance max_evicted_seq_ by this value each time it needs an update. The
  // larger the value, the less frequent advances we would have. We do not want
  // it to be too large either as it would cause stalls by doing too much
  // maintenance work under the lock.
  size_t INC_STEP_FOR_MAX_EVICTED = 1;
412
  // A map from old snapshots (expected to be used by a few read-only txns) to
413
  // prepared sequence number of the evicted entries from commit_cache_ that
414 415 416 417
  // overlaps with such snapshot. These are the prepared sequence numbers that
  // the snapshot, to which they are mapped, cannot assume to be committed just
  // because it is no longer in the commit_cache_. The vector must be sorted
  // after each update.
418
  // Thread-safety is provided with old_commit_map_mutex_.
419
  std::map<SequenceNumber, std::vector<SequenceNumber>> old_commit_map_;
420 421 422 423 424 425 426 427 428 429 430 431 432
  // A set of long-running prepared transactions that are not finished by the
  // time max_evicted_seq_ advances their sequence number. This is expected to
  // be empty normally. Thread-safety is provided with prepared_mutex_.
  std::set<uint64_t> delayed_prepared_;
  // Update when delayed_prepared_.empty() changes. Expected to be true
  // normally.
  std::atomic<bool> delayed_prepared_empty_ = {true};
  // Update when old_commit_map_.empty() changes. Expected to be true normally.
  std::atomic<bool> old_commit_map_empty_ = {true};
  mutable port::RWMutex prepared_mutex_;
  mutable port::RWMutex old_commit_map_mutex_;
  mutable port::RWMutex commit_cache_mutex_;
  mutable port::RWMutex snapshots_mutex_;
433 434 435 436
  // A cache of the cf comparators
  std::atomic<std::map<uint32_t, const Comparator*>*> cf_map_;
  // GC of the object above
  std::unique_ptr<std::map<uint32_t, const Comparator*>> cf_map_gc_;
437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454
};

class WritePreparedTxnReadCallback : public ReadCallback {
 public:
  WritePreparedTxnReadCallback(WritePreparedTxnDB* db, SequenceNumber snapshot)
      : db_(db), snapshot_(snapshot) {}

  // Will be called to see if the seq number accepted; if not it moves on to the
  // next seq number.
  virtual bool IsCommitted(SequenceNumber seq) override {
    return db_->IsInSnapshot(seq, snapshot_);
  }

 private:
  WritePreparedTxnDB* db_;
  SequenceNumber snapshot_;
};

455 456 457 458
class WritePreparedCommitEntryPreReleaseCallback : public PreReleaseCallback {
 public:
  // includes_data indicates that the commit also writes non-empty
  // CommitTimeWriteBatch to memtable, which needs to be committed separately.
459 460 461 462
  WritePreparedCommitEntryPreReleaseCallback(WritePreparedTxnDB* db,
                                             DBImpl* db_impl,
                                             SequenceNumber prep_seq,
                                             size_t prep_batch_cnt,
463
                                             size_t data_batch_cnt = 0,
464 465
                                             bool prep_heap_skipped = false,
                                             bool publish_seq = true)
466 467 468
      : db_(db),
        db_impl_(db_impl),
        prep_seq_(prep_seq),
469 470
        prep_batch_cnt_(prep_batch_cnt),
        data_batch_cnt_(data_batch_cnt),
471
        prep_heap_skipped_(prep_heap_skipped),
472 473
        includes_data_(data_batch_cnt_ > 0),
        publish_seq_(publish_seq) {
474 475 476
    assert((prep_batch_cnt_ > 0) != (prep_seq == kMaxSequenceNumber));  // xor
    assert(prep_batch_cnt_ > 0 || data_batch_cnt_ > 0);
  }
477

478 479
  virtual Status Callback(SequenceNumber commit_seq,
                          bool is_mem_disabled) override {
480
    assert(includes_data_ || prep_seq_ != kMaxSequenceNumber);
481 482 483
    const uint64_t last_commit_seq = LIKELY(data_batch_cnt_ <= 1)
                                         ? commit_seq
                                         : commit_seq + data_batch_cnt_ - 1;
484
    if (prep_seq_ != kMaxSequenceNumber) {
485
      for (size_t i = 0; i < prep_batch_cnt_; i++) {
486
        db_->AddCommitted(prep_seq_ + i, last_commit_seq, prep_heap_skipped_);
487
      }
488
    }  // else there was no prepare phase
489
    if (includes_data_) {
490
      assert(data_batch_cnt_);
491
      // Commit the data that is accompanied with the commit request
492
      const bool PREPARE_SKIPPED = true;
493 494 495 496 497 498
      for (size_t i = 0; i < data_batch_cnt_; i++) {
        // For commit seq of each batch use the commit seq of the last batch.
        // This would make debugging easier by having all the batches having
        // the same sequence number.
        db_->AddCommitted(commit_seq + i, last_commit_seq, PREPARE_SKIPPED);
      }
499
    }
500 501
    if (db_impl_->immutable_db_options().two_write_queues && publish_seq_) {
      assert(is_mem_disabled);  // implies the 2nd queue
502 503 504 505
      // Publish the sequence number. We can do that here assuming the callback
      // is invoked only from one write queue, which would guarantee that the
      // publish sequence numbers will be in order, i.e., once a seq is
      // published all the seq prior to that are also publishable.
506
      db_impl_->SetLastPublishedSequence(last_commit_seq);
507 508 509
    }
    // else SequenceNumber that is updated as part of the write already does the
    // publishing
510 511 512 513 514 515
    return Status::OK();
  }

 private:
  WritePreparedTxnDB* db_;
  DBImpl* db_impl_;
516
  // kMaxSequenceNumber if there was no prepare phase
517
  SequenceNumber prep_seq_;
518 519
  size_t prep_batch_cnt_;
  size_t data_batch_cnt_;
520 521 522
  // An optimization that indicates that there is no need to update the prepare
  // heap since the prepare sequence number was not added to it.
  bool prep_heap_skipped_;
523 524
  // Either because it is commit without prepare or it has a
  // CommitTimeWriteBatch
525
  bool includes_data_;
526 527
  // Should the callback also publishes the commit seq number
  bool publish_seq_;
528 529
};

530 531 532 533 534 535 536 537 538 539 540
// Count the number of sub-batches inside a batch. A sub-batch does not have
// duplicate keys.
struct SubBatchCounter : public WriteBatch::Handler {
  explicit SubBatchCounter(std::map<uint32_t, const Comparator*>& comparators)
      : comparators_(comparators), batches_(1) {}
  std::map<uint32_t, const Comparator*>& comparators_;
  using CFKeys = std::set<Slice, SetComparator>;
  std::map<uint32_t, CFKeys> keys_;
  size_t batches_;
  size_t BatchCount() { return batches_; }
  void AddKey(const uint32_t cf, const Slice& key);
541
  void InitWithComp(const uint32_t cf);
542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565
  Status MarkNoop(bool) override { return Status::OK(); }
  Status MarkEndPrepare(const Slice&) override { return Status::OK(); }
  Status MarkCommit(const Slice&) override { return Status::OK(); }
  Status PutCF(uint32_t cf, const Slice& key, const Slice&) override {
    AddKey(cf, key);
    return Status::OK();
  }
  Status DeleteCF(uint32_t cf, const Slice& key) override {
    AddKey(cf, key);
    return Status::OK();
  }
  Status SingleDeleteCF(uint32_t cf, const Slice& key) override {
    AddKey(cf, key);
    return Status::OK();
  }
  Status MergeCF(uint32_t cf, const Slice& key, const Slice&) override {
    AddKey(cf, key);
    return Status::OK();
  }
  Status MarkBeginPrepare() override { return Status::OK(); }
  Status MarkRollback(const Slice&) override { return Status::OK(); }
  bool WriteAfterCommit() const override { return false; }
};

566 567
}  //  namespace rocksdb
#endif  // ROCKSDB_LITE