db_impl_write.cc 69.9 KB
Newer Older
S
Siying Dong 已提交
1
//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
S
Siying Dong 已提交
2 3 4
//  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).
S
Siying Dong 已提交
5 6 7 8
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
9
#include <cinttypes>
10 11

#include "db/db_impl/db_impl.h"
12
#include "db/error_handler.h"
13
#include "db/event_helpers.h"
14
#include "monitoring/perf_context_imp.h"
S
Siying Dong 已提交
15
#include "options/options_helper.h"
16
#include "test_util/sync_point.h"
17
#include "util/cast_util.h"
S
Siying Dong 已提交
18

19
namespace ROCKSDB_NAMESPACE {
S
Siying Dong 已提交
20 21 22 23 24 25 26 27
// Convenience methods
Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family,
                   const Slice& key, const Slice& val) {
  return DB::Put(o, column_family, key, val);
}

Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family,
                     const Slice& key, const Slice& val) {
28
  auto cfh = static_cast_with_check<ColumnFamilyHandleImpl>(column_family);
S
Siying Dong 已提交
29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46
  if (!cfh->cfd()->ioptions()->merge_operator) {
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
    return DB::Merge(o, column_family, key, val);
  }
}

Status DBImpl::Delete(const WriteOptions& write_options,
                      ColumnFamilyHandle* column_family, const Slice& key) {
  return DB::Delete(write_options, column_family, key);
}

Status DBImpl::SingleDelete(const WriteOptions& write_options,
                            ColumnFamilyHandle* column_family,
                            const Slice& key) {
  return DB::SingleDelete(write_options, column_family, key);
}

47 48 49 50 51
void DBImpl::SetRecoverableStatePreReleaseCallback(
    PreReleaseCallback* callback) {
  recoverable_state_pre_release_callback_.reset(callback);
}

S
Siying Dong 已提交
52 53 54 55 56 57 58 59 60 61 62 63
Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) {
  return WriteImpl(write_options, my_batch, nullptr, nullptr);
}

#ifndef ROCKSDB_LITE
Status DBImpl::WriteWithCallback(const WriteOptions& write_options,
                                 WriteBatch* my_batch,
                                 WriteCallback* callback) {
  return WriteImpl(write_options, my_batch, callback, nullptr);
}
#endif  // ROCKSDB_LITE

64 65 66
// The main write queue. This is the only write queue that updates LastSequence.
// When using one write queue, the same sequence also indicates the last
// published sequence.
S
Siying Dong 已提交
67 68 69
Status DBImpl::WriteImpl(const WriteOptions& write_options,
                         WriteBatch* my_batch, WriteCallback* callback,
                         uint64_t* log_used, uint64_t log_ref,
70
                         bool disable_memtable, uint64_t* seq_used,
71
                         size_t batch_cnt,
72
                         PreReleaseCallback* pre_release_callback) {
73
  assert(!seq_per_batch_ || batch_cnt != 0);
S
Siying Dong 已提交
74 75 76
  if (my_batch == nullptr) {
    return Status::Corruption("Batch is nullptr!");
  }
77 78 79 80 81 82
  if (tracer_) {
    InstrumentedMutexLock lock(&trace_mutex_);
    if (tracer_) {
      tracer_->Write(my_batch);
    }
  }
83 84 85
  if (write_options.sync && write_options.disableWAL) {
    return Status::InvalidArgument("Sync writes has to enable WAL.");
  }
86
  if (two_write_queues_ && immutable_db_options_.enable_pipelined_write) {
87 88 89
    return Status::NotSupported(
        "pipelined_writes is not compatible with concurrent prepares");
  }
90
  if (seq_per_batch_ && immutable_db_options_.enable_pipelined_write) {
91
    // TODO(yiwu): update pipeline write with seq_per_batch and batch_cnt
92 93 94
    return Status::NotSupported(
        "pipelined_writes is not compatible with seq_per_batch");
  }
M
Maysam Yabandeh 已提交
95 96 97 98 99
  if (immutable_db_options_.unordered_write &&
      immutable_db_options_.enable_pipelined_write) {
    return Status::NotSupported(
        "pipelined_writes is not compatible with unordered_write");
  }
100 101 102
  // Otherwise IsLatestPersistentState optimization does not make sense
  assert(!WriteBatchInternal::IsLatestPersistentState(my_batch) ||
         disable_memtable);
S
Siying Dong 已提交
103

104
  Status status;
105
  IOStatus io_s;
106 107 108 109 110 111 112
  if (write_options.low_pri) {
    status = ThrottleLowPriWritesIfNeeded(write_options, my_batch);
    if (!status.ok()) {
      return status;
    }
  }

113
  if (two_write_queues_ && disable_memtable) {
M
Maysam Yabandeh 已提交
114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135
    AssignOrder assign_order =
        seq_per_batch_ ? kDoAssignOrder : kDontAssignOrder;
    // Otherwise it is WAL-only Prepare batches in WriteCommitted policy and
    // they don't consume sequence.
    return WriteImplWALOnly(&nonmem_write_thread_, write_options, my_batch,
                            callback, log_used, log_ref, seq_used, batch_cnt,
                            pre_release_callback, assign_order,
                            kDontPublishLastSeq, disable_memtable);
  }

  if (immutable_db_options_.unordered_write) {
    const size_t sub_batch_cnt = batch_cnt != 0
                                     ? batch_cnt
                                     // every key is a sub-batch consuming a seq
                                     : WriteBatchInternal::Count(my_batch);
    uint64_t seq;
    // Use a write thread to i) optimize for WAL write, ii) publish last
    // sequence in in increasing order, iii) call pre_release_callback serially
    status = WriteImplWALOnly(&write_thread_, write_options, my_batch, callback,
                              log_used, log_ref, &seq, sub_batch_cnt,
                              pre_release_callback, kDoAssignOrder,
                              kDoPublishLastSeq, disable_memtable);
136
    TEST_SYNC_POINT("DBImpl::WriteImpl:UnorderedWriteAfterWriteWAL");
M
Maysam Yabandeh 已提交
137 138 139 140 141 142 143
    if (!status.ok()) {
      return status;
    }
    if (seq_used) {
      *seq_used = seq;
    }
    if (!disable_memtable) {
144
      TEST_SYNC_POINT("DBImpl::WriteImpl:BeforeUnorderedWriteMemtable");
M
Maysam Yabandeh 已提交
145 146 147 148
      status = UnorderedWriteMemtable(write_options, my_batch, callback,
                                      log_ref, seq, sub_batch_cnt);
    }
    return status;
149 150
  }

151 152
  if (immutable_db_options_.enable_pipelined_write) {
    return PipelinedWriteImpl(write_options, my_batch, callback, log_used,
153
                              log_ref, disable_memtable, seq_used);
154 155
  }

S
Siying Dong 已提交
156 157
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
  WriteThread::Writer w(write_options, my_batch, callback, log_ref,
158
                        disable_memtable, batch_cnt, pre_release_callback);
S
Siying Dong 已提交
159 160 161 162 163 164 165 166

  if (!write_options.disableWAL) {
    RecordTick(stats_, WRITE_WITH_WAL);
  }

  StopWatch write_sw(env_, immutable_db_options_.statistics.get(), DB_WRITE);

  write_thread_.JoinBatchGroup(&w);
167
  if (w.state == WriteThread::STATE_PARALLEL_MEMTABLE_WRITER) {
S
Siying Dong 已提交
168 169 170
    // we are a non-leader in a parallel group

    if (w.ShouldWriteToMemtable()) {
171 172 173
      PERF_TIMER_STOP(write_pre_and_post_process_time);
      PERF_TIMER_GUARD(write_memtable_time);

S
Siying Dong 已提交
174 175 176
      ColumnFamilyMemTablesImpl column_family_memtables(
          versions_->GetColumnFamilySet());
      w.status = WriteBatchInternal::InsertInto(
Y
Yi Wu 已提交
177
          &w, w.sequence, &column_family_memtables, &flush_scheduler_,
178
          &trim_history_scheduler_,
S
Siying Dong 已提交
179
          write_options.ignore_missing_column_families, 0 /*log_number*/, this,
180
          true /*concurrent_memtable_writes*/, seq_per_batch_, w.batch_cnt,
181
          batch_per_txn_, write_options.memtable_insert_hint_per_batch);
182 183

      PERF_TIMER_START(write_pre_and_post_process_time);
S
Siying Dong 已提交
184 185
    }

186
    if (write_thread_.CompleteParallelMemTableWriter(&w)) {
Y
Yi Wu 已提交
187
      // we're responsible for exit batch group
188
      // TODO(myabandeh): propagate status to write_group
189
      auto last_sequence = w.write_group->last_sequence;
S
Siying Dong 已提交
190
      versions_->SetLastSequence(last_sequence);
191
      MemTableInsertStatusCheck(w.status);
Y
Yi Wu 已提交
192
      write_thread_.ExitAsBatchGroupFollower(&w);
S
Siying Dong 已提交
193 194 195 196 197 198 199 200 201 202
    }
    assert(w.state == WriteThread::STATE_COMPLETED);
    // STATE_COMPLETED conditional below handles exit

    status = w.FinalStatus();
  }
  if (w.state == WriteThread::STATE_COMPLETED) {
    if (log_used != nullptr) {
      *log_used = w.log_used;
    }
203 204 205
    if (seq_used != nullptr) {
      *seq_used = w.sequence;
    }
S
Siying Dong 已提交
206 207 208 209 210 211 212 213 214 215 216
    // write is complete and leader has updated sequence
    return w.FinalStatus();
  }
  // else we are the leader of the write batch group
  assert(w.state == WriteThread::STATE_GROUP_LEADER);

  // Once reaches this point, the current writer "w" will try to do its write
  // job.  It may also pick up some of the remaining writers in the "writers_"
  // when it finds suitable, and finish them in the same write batch.
  // This is how a write job could be done by the other writer.
  WriteContext write_context;
217
  WriteThread::WriteGroup write_group;
Y
Yi Wu 已提交
218
  bool in_parallel_group = false;
S
Siying Dong 已提交
219
  uint64_t last_sequence = kMaxSequenceNumber;
S
Siying Dong 已提交
220 221 222

  mutex_.Lock();

223
  bool need_log_sync = write_options.sync;
S
Siying Dong 已提交
224
  bool need_log_dir_sync = need_log_sync && !log_dir_synced_;
225
  if (!two_write_queues_ || !disable_memtable) {
226 227 228
    // With concurrent writes we do preprocess only in the write thread that
    // also does write to memtable to avoid sync issue on shared data structure
    // with the other thread
229 230 231 232

    // PreprocessWrite does its own perf timing.
    PERF_TIMER_STOP(write_pre_and_post_process_time);

233
    status = PreprocessWrite(write_options, &need_log_sync, &write_context);
234 235 236 237 238
    if (!two_write_queues_) {
      // Assign it after ::PreprocessWrite since the sequence might advance
      // inside it by WriteRecoverableState
      last_sequence = versions_->LastSequence();
    }
239 240

    PERF_TIMER_START(write_pre_and_post_process_time);
241 242
  }
  log::Writer* log_writer = logs_.back().writer;
S
Siying Dong 已提交
243 244 245 246 247 248 249 250

  mutex_.Unlock();

  // Add to log and apply to memtable.  We can release the lock
  // during this phase since &w is currently responsible for logging
  // and protects against concurrent loggers and concurrent writes
  // into memtables

251
  TEST_SYNC_POINT("DBImpl::WriteImpl:BeforeLeaderEnters");
S
Siying Dong 已提交
252
  last_batch_group_size_ =
253
      write_thread_.EnterAsBatchGroupLeader(&w, &write_group);
S
Siying Dong 已提交
254 255 256 257 258

  if (status.ok()) {
    // Rules for when we can update the memtable concurrently
    // 1. supported by memtable
    // 2. Puts are not okay if inplace_update_support
259
    // 3. Merges are not okay
S
Siying Dong 已提交
260
    //
261
    // Rules 1..2 are enforced by checking the options
S
Siying Dong 已提交
262 263
    // during startup (CheckConcurrentWritesSupported), so if
    // options.allow_concurrent_memtable_write is true then they can be
264 265
    // assumed to be true.  Rule 3 is checked for each batch.  We could
    // relax rules 2 if we could prevent write batches from referring
S
Siying Dong 已提交
266 267
    // more than once to a particular key.
    bool parallel = immutable_db_options_.allow_concurrent_memtable_write &&
268
                    write_group.size > 1;
269
    size_t total_count = 0;
270
    size_t valid_batches = 0;
271
    size_t total_byte_size = 0;
272
    size_t pre_release_callback_cnt = 0;
273
    for (auto* writer : write_group) {
S
Siying Dong 已提交
274
      if (writer->CheckCallback(this)) {
275
        valid_batches += writer->batch_cnt;
S
Siying Dong 已提交
276 277 278 279
        if (writer->ShouldWriteToMemtable()) {
          total_count += WriteBatchInternal::Count(writer->batch);
          parallel = parallel && !writer->batch->HasMerge();
        }
I
Igor Canadi 已提交
280 281
        total_byte_size = WriteBatchInternal::AppendedByteSize(
            total_byte_size, WriteBatchInternal::ByteSize(writer->batch));
282 283 284
        if (writer->pre_release_callback) {
          pre_release_callback_cnt++;
        }
S
Siying Dong 已提交
285 286
      }
    }
287 288 289 290 291 292 293
    // Note about seq_per_batch_: either disableWAL is set for the entire write
    // group or not. In either case we inc seq for each write batch with no
    // failed callback. This means that there could be a batch with
    // disalbe_memtable in between; although we do not write this batch to
    // memtable it still consumes a seq. Otherwise, if !seq_per_batch_, we inc
    // the seq per valid written key to mem.
    size_t seq_inc = seq_per_batch_ ? valid_batches : total_count;
S
Siying Dong 已提交
294

295
    const bool concurrent_update = two_write_queues_;
S
Siying Dong 已提交
296 297 298
    // Update stats while we are an exclusive group leader, so we know
    // that nobody else can be writing to these particular stats.
    // We're optimistic, updating the stats before we successfully
Y
Yi Wu 已提交
299
    // commit.  That lets us release our leader status early.
S
Siying Dong 已提交
300
    auto stats = default_cf_internal_stats_;
301
    stats->AddDBStats(InternalStats::kIntStatsNumKeysWritten, total_count,
302
                      concurrent_update);
S
Siying Dong 已提交
303
    RecordTick(stats_, NUMBER_KEYS_WRITTEN, total_count);
304
    stats->AddDBStats(InternalStats::kIntStatsBytesWritten, total_byte_size,
305
                      concurrent_update);
S
Siying Dong 已提交
306
    RecordTick(stats_, BYTES_WRITTEN, total_byte_size);
307 308
    stats->AddDBStats(InternalStats::kIntStatsWriteDoneBySelf, 1,
                      concurrent_update);
S
Siying Dong 已提交
309
    RecordTick(stats_, WRITE_DONE_BY_SELF);
310
    auto write_done_by_other = write_group.size - 1;
S
Siying Dong 已提交
311
    if (write_done_by_other > 0) {
312 313
      stats->AddDBStats(InternalStats::kIntStatsWriteDoneByOther,
                        write_done_by_other, concurrent_update);
S
Siying Dong 已提交
314 315
      RecordTick(stats_, WRITE_DONE_BY_OTHER, write_done_by_other);
    }
S
Siying Dong 已提交
316
    RecordInHistogram(stats_, BYTES_PER_WRITE, total_byte_size);
S
Siying Dong 已提交
317 318 319 320 321 322 323

    if (write_options.disableWAL) {
      has_unpersisted_data_.store(true, std::memory_order_relaxed);
    }

    PERF_TIMER_STOP(write_pre_and_post_process_time);

324
    if (!two_write_queues_) {
325 326
      if (status.ok() && !write_options.disableWAL) {
        PERF_TIMER_GUARD(write_wal_time);
327 328
        io_s = WriteToWAL(write_group, log_writer, log_used, need_log_sync,
                          need_log_dir_sync, last_sequence + 1);
329 330 331 332
      }
    } else {
      if (status.ok() && !write_options.disableWAL) {
        PERF_TIMER_GUARD(write_wal_time);
333
        // LastAllocatedSequence is increased inside WriteToWAL under
334
        // wal_write_mutex_ to ensure ordered events in WAL
335 336
        io_s = ConcurrentWriteToWAL(write_group, log_used, &last_sequence,
                                    seq_inc);
337 338
      } else {
        // Otherwise we inc seq number for memtable writes
339
        last_sequence = versions_->FetchAddLastAllocatedSequence(seq_inc);
S
Siying Dong 已提交
340 341
      }
    }
342
    status = io_s;
S
Siying Dong 已提交
343
    assert(last_sequence != kMaxSequenceNumber);
344
    const SequenceNumber current_sequence = last_sequence + 1;
345
    last_sequence += seq_inc;
S
Siying Dong 已提交
346

347 348 349
    // PreReleaseCallback is called after WAL write and before memtable write
    if (status.ok()) {
      SequenceNumber next_sequence = current_sequence;
350
      size_t index = 0;
351 352 353 354 355 356 357 358 359 360
      // Note: the logic for advancing seq here must be consistent with the
      // logic in WriteBatchInternal::InsertInto(write_group...) as well as
      // with WriteBatchInternal::InsertInto(write_batch...) that is called on
      // the merged batch during recovery from the WAL.
      for (auto* writer : write_group) {
        if (writer->CallbackFailed()) {
          continue;
        }
        writer->sequence = next_sequence;
        if (writer->pre_release_callback) {
361
          Status ws = writer->pre_release_callback->Callback(
362 363
              writer->sequence, disable_memtable, writer->log_used, index++,
              pre_release_callback_cnt);
364 365 366 367 368 369 370 371 372 373 374 375 376 377
          if (!ws.ok()) {
            status = ws;
            break;
          }
        }
        if (seq_per_batch_) {
          assert(writer->batch_cnt);
          next_sequence += writer->batch_cnt;
        } else if (writer->ShouldWriteToMemtable()) {
          next_sequence += WriteBatchInternal::Count(writer->batch);
        }
      }
    }

S
Siying Dong 已提交
378 379 380 381
    if (status.ok()) {
      PERF_TIMER_GUARD(write_memtable_time);

      if (!parallel) {
382
        // w.sequence will be set inside InsertInto
Y
Yi Wu 已提交
383
        w.status = WriteBatchInternal::InsertInto(
S
Siying Dong 已提交
384
            write_group, current_sequence, column_family_memtables_.get(),
385 386
            &flush_scheduler_, &trim_history_scheduler_,
            write_options.ignore_missing_column_families,
387 388
            0 /*recovery_log_number*/, this, parallel, seq_per_batch_,
            batch_per_txn_);
S
Siying Dong 已提交
389
      } else {
390 391
        write_group.last_sequence = last_sequence;
        write_thread_.LaunchParallelMemTableWriters(&write_group);
Y
Yi Wu 已提交
392
        in_parallel_group = true;
S
Siying Dong 已提交
393

394 395
        // Each parallel follower is doing each own writes. The leader should
        // also do its own.
S
Siying Dong 已提交
396 397 398 399 400
        if (w.ShouldWriteToMemtable()) {
          ColumnFamilyMemTablesImpl column_family_memtables(
              versions_->GetColumnFamilySet());
          assert(w.sequence == current_sequence);
          w.status = WriteBatchInternal::InsertInto(
Y
Yi Wu 已提交
401
              &w, w.sequence, &column_family_memtables, &flush_scheduler_,
402
              &trim_history_scheduler_,
S
Siying Dong 已提交
403
              write_options.ignore_missing_column_families, 0 /*log_number*/,
404
              this, true /*concurrent_memtable_writes*/, seq_per_batch_,
405 406
              w.batch_cnt, batch_per_txn_,
              write_options.memtable_insert_hint_per_batch);
S
Siying Dong 已提交
407
        }
408 409 410
      }
      if (seq_used != nullptr) {
        *seq_used = w.sequence;
S
Siying Dong 已提交
411 412 413 414 415
      }
    }
  }
  PERF_TIMER_START(write_pre_and_post_process_time);

416
  if (!w.CallbackFailed()) {
417 418 419 420 421
    if (!io_s.ok()) {
      IOStatusCheck(io_s);
    } else {
      WriteStatusCheck(status);
    }
S
Siying Dong 已提交
422 423
  }

424
  if (need_log_sync) {
S
Siying Dong 已提交
425 426 427
    mutex_.Lock();
    MarkLogsSynced(logfile_number_, need_log_dir_sync, status);
    mutex_.Unlock();
428
    // Requesting sync with two_write_queues_ is expected to be very rare. We
429
    // hence provide a simple implementation that is not necessarily efficient.
430
    if (two_write_queues_) {
431 432 433 434 435 436
      if (manual_wal_flush_) {
        status = FlushWAL(true);
      } else {
        status = SyncWAL();
      }
    }
S
Siying Dong 已提交
437 438
  }

Y
Yi Wu 已提交
439 440 441 442
  bool should_exit_batch_group = true;
  if (in_parallel_group) {
    // CompleteParallelWorker returns true if this thread should
    // handle exit, false means somebody else did
443
    should_exit_batch_group = write_thread_.CompleteParallelMemTableWriter(&w);
Y
Yi Wu 已提交
444 445
  }
  if (should_exit_batch_group) {
446
    if (status.ok()) {
447 448
      // Note: if we are to resume after non-OK statuses we need to revisit how
      // we reacts to non-OK statuses here.
449 450
      versions_->SetLastSequence(last_sequence);
    }
451
    MemTableInsertStatusCheck(w.status);
452
    write_thread_.ExitAsBatchGroupLeader(write_group, status);
S
Siying Dong 已提交
453 454
  }

Y
Yi Wu 已提交
455 456 457
  if (status.ok()) {
    status = w.FinalStatus();
  }
S
Siying Dong 已提交
458 459 460
  return status;
}

461 462 463
Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options,
                                  WriteBatch* my_batch, WriteCallback* callback,
                                  uint64_t* log_used, uint64_t log_ref,
464
                                  bool disable_memtable, uint64_t* seq_used) {
465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
  StopWatch write_sw(env_, immutable_db_options_.statistics.get(), DB_WRITE);

  WriteContext write_context;

  WriteThread::Writer w(write_options, my_batch, callback, log_ref,
                        disable_memtable);
  write_thread_.JoinBatchGroup(&w);
  if (w.state == WriteThread::STATE_GROUP_LEADER) {
    WriteThread::WriteGroup wal_write_group;
    if (w.callback && !w.callback->AllowWriteBatching()) {
      write_thread_.WaitForMemTableWriters();
    }
    mutex_.Lock();
    bool need_log_sync = !write_options.disableWAL && write_options.sync;
    bool need_log_dir_sync = need_log_sync && !log_dir_synced_;
481 482
    // PreprocessWrite does its own perf timing.
    PERF_TIMER_STOP(write_pre_and_post_process_time);
483
    w.status = PreprocessWrite(write_options, &need_log_sync, &write_context);
484
    PERF_TIMER_START(write_pre_and_post_process_time);
485
    log::Writer* log_writer = logs_.back().writer;
486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516
    mutex_.Unlock();

    // This can set non-OK status if callback fail.
    last_batch_group_size_ =
        write_thread_.EnterAsBatchGroupLeader(&w, &wal_write_group);
    const SequenceNumber current_sequence =
        write_thread_.UpdateLastSequence(versions_->LastSequence()) + 1;
    size_t total_count = 0;
    size_t total_byte_size = 0;

    if (w.status.ok()) {
      SequenceNumber next_sequence = current_sequence;
      for (auto writer : wal_write_group) {
        if (writer->CheckCallback(this)) {
          if (writer->ShouldWriteToMemtable()) {
            writer->sequence = next_sequence;
            size_t count = WriteBatchInternal::Count(writer->batch);
            next_sequence += count;
            total_count += count;
          }
          total_byte_size = WriteBatchInternal::AppendedByteSize(
              total_byte_size, WriteBatchInternal::ByteSize(writer->batch));
        }
      }
      if (w.disable_wal) {
        has_unpersisted_data_.store(true, std::memory_order_relaxed);
      }
      write_thread_.UpdateLastSequence(current_sequence + total_count - 1);
    }

    auto stats = default_cf_internal_stats_;
517
    stats->AddDBStats(InternalStats::kIntStatsNumKeysWritten, total_count);
518
    RecordTick(stats_, NUMBER_KEYS_WRITTEN, total_count);
519
    stats->AddDBStats(InternalStats::kIntStatsBytesWritten, total_byte_size);
520
    RecordTick(stats_, BYTES_WRITTEN, total_byte_size);
S
Siying Dong 已提交
521
    RecordInHistogram(stats_, BYTES_PER_WRITE, total_byte_size);
522 523 524

    PERF_TIMER_STOP(write_pre_and_post_process_time);

525
    IOStatus io_s;
526
    if (w.status.ok() && !write_options.disableWAL) {
527
      PERF_TIMER_GUARD(write_wal_time);
528
      stats->AddDBStats(InternalStats::kIntStatsWriteDoneBySelf, 1);
529 530
      RecordTick(stats_, WRITE_DONE_BY_SELF, 1);
      if (wal_write_group.size > 1) {
531
        stats->AddDBStats(InternalStats::kIntStatsWriteDoneByOther,
532 533 534
                          wal_write_group.size - 1);
        RecordTick(stats_, WRITE_DONE_BY_OTHER, wal_write_group.size - 1);
      }
535 536 537
      io_s = WriteToWAL(wal_write_group, log_writer, log_used, need_log_sync,
                        need_log_dir_sync, current_sequence);
      w.status = io_s;
538 539 540
    }

    if (!w.CallbackFailed()) {
541 542 543 544 545
      if (!io_s.ok()) {
        IOStatusCheck(io_s);
      } else {
        WriteStatusCheck(w.status);
      }
546 547 548 549 550 551 552 553 554 555 556 557 558 559
    }

    if (need_log_sync) {
      mutex_.Lock();
      MarkLogsSynced(logfile_number_, need_log_dir_sync, w.status);
      mutex_.Unlock();
    }

    write_thread_.ExitAsBatchGroupLeader(wal_write_group, w.status);
  }

  WriteThread::WriteGroup memtable_write_group;
  if (w.state == WriteThread::STATE_MEMTABLE_WRITER_LEADER) {
    PERF_TIMER_GUARD(write_memtable_time);
560
    assert(w.ShouldWriteToMemtable());
561 562 563 564 565 566 567
    write_thread_.EnterAsMemTableWriter(&w, &memtable_write_group);
    if (memtable_write_group.size > 1 &&
        immutable_db_options_.allow_concurrent_memtable_write) {
      write_thread_.LaunchParallelMemTableWriters(&memtable_write_group);
    } else {
      memtable_write_group.status = WriteBatchInternal::InsertInto(
          memtable_write_group, w.sequence, column_family_memtables_.get(),
568 569 570
          &flush_scheduler_, &trim_history_scheduler_,
          write_options.ignore_missing_column_families, 0 /*log_number*/, this,
          false /*concurrent_memtable_writes*/, seq_per_batch_, batch_per_txn_);
571 572 573 574 575 576 577 578 579 580
      versions_->SetLastSequence(memtable_write_group.last_sequence);
      write_thread_.ExitAsMemTableWriter(&w, memtable_write_group);
    }
  }

  if (w.state == WriteThread::STATE_PARALLEL_MEMTABLE_WRITER) {
    assert(w.ShouldWriteToMemtable());
    ColumnFamilyMemTablesImpl column_family_memtables(
        versions_->GetColumnFamilySet());
    w.status = WriteBatchInternal::InsertInto(
Y
Yi Wu 已提交
581
        &w, w.sequence, &column_family_memtables, &flush_scheduler_,
582
        &trim_history_scheduler_, write_options.ignore_missing_column_families,
583 584 585
        0 /*log_number*/, this, true /*concurrent_memtable_writes*/,
        false /*seq_per_batch*/, 0 /*batch_cnt*/, true /*batch_per_txn*/,
        write_options.memtable_insert_hint_per_batch);
586 587 588 589 590 591
    if (write_thread_.CompleteParallelMemTableWriter(&w)) {
      MemTableInsertStatusCheck(w.status);
      versions_->SetLastSequence(w.write_group->last_sequence);
      write_thread_.ExitAsMemTableWriter(&w, *w.write_group);
    }
  }
592 593 594
  if (seq_used != nullptr) {
    *seq_used = w.sequence;
  }
595 596

  assert(w.state == WriteThread::STATE_COMPLETED);
597 598 599
  return w.FinalStatus();
}

M
Maysam Yabandeh 已提交
600 601 602 603 604 605 606 607 608 609 610 611 612 613 614
Status DBImpl::UnorderedWriteMemtable(const WriteOptions& write_options,
                                      WriteBatch* my_batch,
                                      WriteCallback* callback, uint64_t log_ref,
                                      SequenceNumber seq,
                                      const size_t sub_batch_cnt) {
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
  StopWatch write_sw(env_, immutable_db_options_.statistics.get(), DB_WRITE);

  WriteThread::Writer w(write_options, my_batch, callback, log_ref,
                        false /*disable_memtable*/);

  if (w.CheckCallback(this) && w.ShouldWriteToMemtable()) {
    w.sequence = seq;
    size_t total_count = WriteBatchInternal::Count(my_batch);
    InternalStats* stats = default_cf_internal_stats_;
615
    stats->AddDBStats(InternalStats::kIntStatsNumKeysWritten, total_count);
M
Maysam Yabandeh 已提交
616 617 618 619 620 621
    RecordTick(stats_, NUMBER_KEYS_WRITTEN, total_count);

    ColumnFamilyMemTablesImpl column_family_memtables(
        versions_->GetColumnFamilySet());
    w.status = WriteBatchInternal::InsertInto(
        &w, w.sequence, &column_family_memtables, &flush_scheduler_,
622 623
        &trim_history_scheduler_, write_options.ignore_missing_column_families,
        0 /*log_number*/, this, true /*concurrent_memtable_writes*/,
624 625
        seq_per_batch_, sub_batch_cnt, true /*batch_per_txn*/,
        write_options.memtable_insert_hint_per_batch);
M
Maysam Yabandeh 已提交
626 627 628 629 630 631 632
    if (write_options.disableWAL) {
      has_unpersisted_data_.store(true, std::memory_order_relaxed);
    }
  }

  size_t pending_cnt = pending_memtable_writes_.fetch_sub(1) - 1;
  if (pending_cnt == 0) {
633 634 635 636 637
    // switch_cv_ waits until pending_memtable_writes_ = 0. Locking its mutex
    // before notify ensures that cv is in waiting state when it is notified
    // thus not missing the update to pending_memtable_writes_ even though it is
    // not modified under the mutex.
    std::lock_guard<std::mutex> lck(switch_mutex_);
M
Maysam Yabandeh 已提交
638 639
    switch_cv_.notify_all();
  }
640
  WriteStatusCheck(w.status);
M
Maysam Yabandeh 已提交
641 642 643 644 645 646 647

  if (!w.FinalStatus().ok()) {
    return w.FinalStatus();
  }
  return Status::OK();
}

648 649 650
// The 2nd write queue. If enabled it will be used only for WAL-only writes.
// This is the only queue that updates LastPublishedSequence which is only
// applicable in a two-queue setting.
M
Maysam Yabandeh 已提交
651 652 653 654 655 656
Status DBImpl::WriteImplWALOnly(
    WriteThread* write_thread, const WriteOptions& write_options,
    WriteBatch* my_batch, WriteCallback* callback, uint64_t* log_used,
    const uint64_t log_ref, uint64_t* seq_used, const size_t sub_batch_cnt,
    PreReleaseCallback* pre_release_callback, const AssignOrder assign_order,
    const PublishLastSeq publish_last_seq, const bool disable_memtable) {
657 658 659
  Status status;
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
  WriteThread::Writer w(write_options, my_batch, callback, log_ref,
M
Maysam Yabandeh 已提交
660
                        disable_memtable, sub_batch_cnt, pre_release_callback);
661 662
  RecordTick(stats_, WRITE_WITH_WAL);
  StopWatch write_sw(env_, immutable_db_options_.statistics.get(), DB_WRITE);
663

M
Maysam Yabandeh 已提交
664
  write_thread->JoinBatchGroup(&w);
665 666 667 668 669
  assert(w.state != WriteThread::STATE_PARALLEL_MEMTABLE_WRITER);
  if (w.state == WriteThread::STATE_COMPLETED) {
    if (log_used != nullptr) {
      *log_used = w.log_used;
    }
670 671 672
    if (seq_used != nullptr) {
      *seq_used = w.sequence;
    }
673 674 675 676
    return w.FinalStatus();
  }
  // else we are the leader of the write batch group
  assert(w.state == WriteThread::STATE_GROUP_LEADER);
M
Maysam Yabandeh 已提交
677 678 679 680 681 682 683 684 685 686 687 688 689 690

  if (publish_last_seq == kDoPublishLastSeq) {
    // Currently we only use kDoPublishLastSeq in unordered_write
    assert(immutable_db_options_.unordered_write);
    WriteContext write_context;
    if (error_handler_.IsDBStopped()) {
      status = error_handler_.GetBGError();
    }
    // TODO(myabandeh): Make preliminary checks thread-safe so we could do them
    // without paying the cost of obtaining the mutex.
    if (status.ok()) {
      InstrumentedMutexLock l(&mutex_);
      bool need_log_sync = false;
      status = PreprocessWrite(write_options, &need_log_sync, &write_context);
691
      WriteStatusCheckOnLocked(status);
M
Maysam Yabandeh 已提交
692 693 694 695 696 697 698 699 700
    }
    if (!status.ok()) {
      WriteThread::WriteGroup write_group;
      write_thread->EnterAsBatchGroupLeader(&w, &write_group);
      write_thread->ExitAsBatchGroupLeader(write_group, status);
      return status;
    }
  }

701 702
  WriteThread::WriteGroup write_group;
  uint64_t last_sequence;
M
Maysam Yabandeh 已提交
703
  write_thread->EnterAsBatchGroupLeader(&w, &write_group);
704 705 706
  // Note: no need to update last_batch_group_size_ here since the batch writes
  // to WAL only

707
  size_t pre_release_callback_cnt = 0;
708
  size_t total_byte_size = 0;
709 710 711 712
  for (auto* writer : write_group) {
    if (writer->CheckCallback(this)) {
      total_byte_size = WriteBatchInternal::AppendedByteSize(
          total_byte_size, WriteBatchInternal::ByteSize(writer->batch));
713 714 715
      if (writer->pre_release_callback) {
        pre_release_callback_cnt++;
      }
716 717 718
    }
  }

719
  const bool concurrent_update = true;
720 721 722 723 724
  // Update stats while we are an exclusive group leader, so we know
  // that nobody else can be writing to these particular stats.
  // We're optimistic, updating the stats before we successfully
  // commit.  That lets us release our leader status early.
  auto stats = default_cf_internal_stats_;
725
  stats->AddDBStats(InternalStats::kIntStatsBytesWritten, total_byte_size,
726
                    concurrent_update);
727
  RecordTick(stats_, BYTES_WRITTEN, total_byte_size);
728 729
  stats->AddDBStats(InternalStats::kIntStatsWriteDoneBySelf, 1,
                    concurrent_update);
730 731 732
  RecordTick(stats_, WRITE_DONE_BY_SELF);
  auto write_done_by_other = write_group.size - 1;
  if (write_done_by_other > 0) {
733 734
    stats->AddDBStats(InternalStats::kIntStatsWriteDoneByOther,
                      write_done_by_other, concurrent_update);
735 736
    RecordTick(stats_, WRITE_DONE_BY_OTHER, write_done_by_other);
  }
S
Siying Dong 已提交
737
  RecordInHistogram(stats_, BYTES_PER_WRITE, total_byte_size);
738 739 740 741

  PERF_TIMER_STOP(write_pre_and_post_process_time);

  PERF_TIMER_GUARD(write_wal_time);
742
  // LastAllocatedSequence is increased inside WriteToWAL under
743
  // wal_write_mutex_ to ensure ordered events in WAL
744
  size_t seq_inc = 0 /* total_count */;
M
Maysam Yabandeh 已提交
745
  if (assign_order == kDoAssignOrder) {
746 747
    size_t total_batch_cnt = 0;
    for (auto* writer : write_group) {
M
Maysam Yabandeh 已提交
748 749 750 751
      assert(writer->batch_cnt || !seq_per_batch_);
      if (!writer->CallbackFailed()) {
        total_batch_cnt += writer->batch_cnt;
      }
752 753 754
    }
    seq_inc = total_batch_cnt;
  }
755
  IOStatus io_s;
M
Maysam Yabandeh 已提交
756
  if (!write_options.disableWAL) {
757 758
    io_s = ConcurrentWriteToWAL(write_group, log_used, &last_sequence, seq_inc);
    status = io_s;
M
Maysam Yabandeh 已提交
759 760 761 762
  } else {
    // Otherwise we inc seq number to do solely the seq allocation
    last_sequence = versions_->FetchAddLastAllocatedSequence(seq_inc);
  }
M
Maysam Yabandeh 已提交
763 764

  size_t memtable_write_cnt = 0;
765 766
  auto curr_seq = last_sequence + 1;
  for (auto* writer : write_group) {
767 768
    if (writer->CallbackFailed()) {
      continue;
769
    }
770
    writer->sequence = curr_seq;
M
Maysam Yabandeh 已提交
771 772
    if (assign_order == kDoAssignOrder) {
      assert(writer->batch_cnt || !seq_per_batch_);
773
      curr_seq += writer->batch_cnt;
774
    }
M
Maysam Yabandeh 已提交
775 776 777
    if (!writer->disable_memtable) {
      memtable_write_cnt++;
    }
778
    // else seq advances only by memtable writes
779
  }
780
  if (status.ok() && write_options.sync) {
M
Maysam Yabandeh 已提交
781
    assert(!write_options.disableWAL);
782
    // Requesting sync with two_write_queues_ is expected to be very rare. We
783 784 785 786 787 788 789 790 791 792
    // hance provide a simple implementation that is not necessarily efficient.
    if (manual_wal_flush_) {
      status = FlushWAL(true);
    } else {
      status = SyncWAL();
    }
  }
  PERF_TIMER_START(write_pre_and_post_process_time);

  if (!w.CallbackFailed()) {
793 794 795 796 797
    if (!io_s.ok()) {
      IOStatusCheck(io_s);
    } else {
      WriteStatusCheck(status);
    }
798
  }
799
  if (status.ok()) {
800
    size_t index = 0;
801 802 803
    for (auto* writer : write_group) {
      if (!writer->CallbackFailed() && writer->pre_release_callback) {
        assert(writer->sequence != kMaxSequenceNumber);
804
        Status ws = writer->pre_release_callback->Callback(
805 806
            writer->sequence, disable_memtable, writer->log_used, index++,
            pre_release_callback_cnt);
807 808 809 810 811 812 813
        if (!ws.ok()) {
          status = ws;
          break;
        }
      }
    }
  }
M
Maysam Yabandeh 已提交
814 815 816 817 818 819 820 821 822
  if (publish_last_seq == kDoPublishLastSeq) {
    versions_->SetLastSequence(last_sequence + seq_inc);
    // Currently we only use kDoPublishLastSeq in unordered_write
    assert(immutable_db_options_.unordered_write);
  }
  if (immutable_db_options_.unordered_write && status.ok()) {
    pending_memtable_writes_ += memtable_write_cnt;
  }
  write_thread->ExitAsBatchGroupLeader(write_group, status);
823 824 825
  if (status.ok()) {
    status = w.FinalStatus();
  }
826 827 828
  if (seq_used != nullptr) {
    *seq_used = w.sequence;
  }
829
  return status;
830 831
}

832 833 834 835 836 837 838 839 840 841 842
void DBImpl::WriteStatusCheckOnLocked(const Status& status) {
  // Is setting bg_error_ enough here?  This will at least stop
  // compaction and fail any further writes.
  // Caller must hold mutex_.
  mutex_.AssertHeld();
  if (immutable_db_options_.paranoid_checks && !status.ok() &&
      !status.IsBusy() && !status.IsIncomplete()) {
    error_handler_.SetBGError(status, BackgroundErrorReason::kWriteCallback);
  }
}

843
void DBImpl::WriteStatusCheck(const Status& status) {
844 845 846 847 848
  // Is setting bg_error_ enough here?  This will at least stop
  // compaction and fail any further writes.
  if (immutable_db_options_.paranoid_checks && !status.ok() &&
      !status.IsBusy() && !status.IsIncomplete()) {
    mutex_.Lock();
849
    error_handler_.SetBGError(status, BackgroundErrorReason::kWriteCallback);
850 851 852 853
    mutex_.Unlock();
  }
}

854 855 856 857 858 859 860 861 862 863 864
void DBImpl::IOStatusCheck(const IOStatus& io_status) {
  // Is setting bg_error_ enough here?  This will at least stop
  // compaction and fail any further writes.
  if (immutable_db_options_.paranoid_checks && !io_status.ok() &&
      !io_status.IsBusy() && !io_status.IsIncomplete()) {
    mutex_.Lock();
    error_handler_.SetBGError(io_status, BackgroundErrorReason::kWriteCallback);
    mutex_.Unlock();
  }
}

865
void DBImpl::MemTableInsertStatusCheck(const Status& status) {
Y
Yi Wu 已提交
866 867 868 869 870
  // A non-OK status here indicates that the state implied by the
  // WAL has diverged from the in-memory state.  This could be
  // because of a corrupt write_batch (very bad), or because the
  // client specified an invalid column family and didn't specify
  // ignore_missing_column_families.
871
  if (!status.ok()) {
Y
Yi Wu 已提交
872
    mutex_.Lock();
873 874
    assert(!error_handler_.IsBGWorkStopped());
    error_handler_.SetBGError(status, BackgroundErrorReason::kMemTable);
Y
Yi Wu 已提交
875 876 877 878
    mutex_.Unlock();
  }
}

S
Siying Dong 已提交
879
Status DBImpl::PreprocessWrite(const WriteOptions& write_options,
880
                               bool* need_log_sync,
S
Siying Dong 已提交
881 882
                               WriteContext* write_context) {
  mutex_.AssertHeld();
883
  assert(write_context != nullptr && need_log_sync != nullptr);
S
Siying Dong 已提交
884 885
  Status status;

886 887 888 889
  if (error_handler_.IsDBStopped()) {
    status = error_handler_.GetBGError();
  }

890 891
  PERF_TIMER_GUARD(write_scheduling_flushes_compactions_time);

S
Siying Dong 已提交
892 893 894 895
  assert(!single_column_family_mode_ ||
         versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1);
  if (UNLIKELY(status.ok() && !single_column_family_mode_ &&
               total_log_size_ > GetMaxTotalWalSize())) {
M
Maysam Yabandeh 已提交
896
    WaitForPendingWrites();
897
    status = SwitchWAL(write_context);
S
Siying Dong 已提交
898 899 900 901 902 903 904 905
  }

  if (UNLIKELY(status.ok() && write_buffer_manager_->ShouldFlush())) {
    // Before a new memtable is added in SwitchMemtable(),
    // write_buffer_manager_->ShouldFlush() will keep returning true. If another
    // thread is writing to another DB with the same write buffer, they may also
    // be flushed. We may end up with flushing much more DBs than needed. It's
    // suboptimal but still correct.
M
Maysam Yabandeh 已提交
906
    WaitForPendingWrites();
S
Siying Dong 已提交
907 908 909
    status = HandleWriteBufferFull(write_context);
  }

910 911 912 913
  if (UNLIKELY(status.ok() && !trim_history_scheduler_.Empty())) {
    status = TrimMemtableHistory(write_context);
  }

S
Siying Dong 已提交
914
  if (UNLIKELY(status.ok() && !flush_scheduler_.Empty())) {
M
Maysam Yabandeh 已提交
915
    WaitForPendingWrites();
S
Siying Dong 已提交
916 917 918
    status = ScheduleFlushes(write_context);
  }

919 920 921
  PERF_TIMER_STOP(write_scheduling_flushes_compactions_time);
  PERF_TIMER_GUARD(write_pre_and_post_process_time);

S
Siying Dong 已提交
922 923
  if (UNLIKELY(status.ok() && (write_controller_.IsStopped() ||
                               write_controller_.NeedsDelay()))) {
924
    PERF_TIMER_STOP(write_pre_and_post_process_time);
S
Siying Dong 已提交
925 926 927 928 929 930
    PERF_TIMER_GUARD(write_delay_time);
    // We don't know size of curent batch so that we always use the size
    // for previous one. It might create a fairness issue that expiration
    // might happen for smaller writes but larger writes can go through.
    // Can optimize it if it is an issue.
    status = DelayWrite(last_batch_group_size_, write_options);
931
    PERF_TIMER_START(write_pre_and_post_process_time);
S
Siying Dong 已提交
932 933
  }

934
  if (status.ok() && *need_log_sync) {
935 936 937 938 939 940 941
    // Wait until the parallel syncs are finished. Any sync process has to sync
    // the front log too so it is enough to check the status of front()
    // We do a while loop since log_sync_cv_ is signalled when any sync is
    // finished
    // Note: there does not seem to be a reason to wait for parallel sync at
    // this early step but it is not important since parallel sync (SyncWAL) and
    // need_log_sync are usually not used together.
S
Siying Dong 已提交
942 943 944 945 946
    while (logs_.front().getting_synced) {
      log_sync_cv_.Wait();
    }
    for (auto& log : logs_) {
      assert(!log.getting_synced);
947 948 949 950 951
      // This is just to prevent the logs to be synced by a parallel SyncWAL
      // call. We will do the actual syncing later after we will write to the
      // WAL.
      // Note: there does not seem to be a reason to set this early before we
      // actually write to the WAL
S
Siying Dong 已提交
952 953
      log.getting_synced = true;
    }
954 955
  } else {
    *need_log_sync = false;
S
Siying Dong 已提交
956 957 958 959 960
  }

  return status;
}

961
WriteBatch* DBImpl::MergeBatch(const WriteThread::WriteGroup& write_group,
962 963
                               WriteBatch* tmp_batch, size_t* write_with_wal,
                               WriteBatch** to_be_cached_state) {
964 965
  assert(write_with_wal != nullptr);
  assert(tmp_batch != nullptr);
966
  assert(*to_be_cached_state == nullptr);
S
Siying Dong 已提交
967
  WriteBatch* merged_batch = nullptr;
968
  *write_with_wal = 0;
969
  auto* leader = write_group.leader;
970 971
  assert(!leader->disable_wal);  // Same holds for all in the batch group
  if (write_group.size == 1 && !leader->CallbackFailed() &&
972
      leader->batch->GetWalTerminationPoint().is_cleared()) {
S
Siying Dong 已提交
973 974 975
    // we simply write the first WriteBatch to WAL if the group only
    // contains one batch, that batch should be written to the WAL,
    // and the batch is not wanting to be truncated
976
    merged_batch = leader->batch;
977 978 979
    if (WriteBatchInternal::IsLatestPersistentState(merged_batch)) {
      *to_be_cached_state = merged_batch;
    }
980
    *write_with_wal = 1;
S
Siying Dong 已提交
981 982 983 984
  } else {
    // WAL needs all of the batches flattened into a single batch.
    // We could avoid copying here with an iov-like AddRecord
    // interface
985
    merged_batch = tmp_batch;
S
Siying Dong 已提交
986
    for (auto writer : write_group) {
987
      if (!writer->CallbackFailed()) {
S
Siying Dong 已提交
988 989
        WriteBatchInternal::Append(merged_batch, writer->batch,
                                   /*WAL_only*/ true);
990 991 992 993
        if (WriteBatchInternal::IsLatestPersistentState(writer->batch)) {
          // We only need to cache the last of such write batch
          *to_be_cached_state = writer->batch;
        }
994
        (*write_with_wal)++;
S
Siying Dong 已提交
995 996 997
      }
    }
  }
998 999
  return merged_batch;
}
S
Siying Dong 已提交
1000

1001
// When two_write_queues_ is disabled, this function is called from the only
1002
// write thread. Otherwise this must be called holding log_write_mutex_.
1003 1004 1005
IOStatus DBImpl::WriteToWAL(const WriteBatch& merged_batch,
                            log::Writer* log_writer, uint64_t* log_used,
                            uint64_t* log_size) {
1006 1007 1008
  assert(log_size != nullptr);
  Slice log_entry = WriteBatchInternal::Contents(&merged_batch);
  *log_size = log_entry.size();
1009 1010 1011 1012 1013 1014 1015 1016 1017 1018 1019
  // When two_write_queues_ WriteToWAL has to be protected from concurretn calls
  // from the two queues anyway and log_write_mutex_ is already held. Otherwise
  // if manual_wal_flush_ is enabled we need to protect log_writer->AddRecord
  // from possible concurrent calls via the FlushWAL by the application.
  const bool needs_locking = manual_wal_flush_ && !two_write_queues_;
  // Due to performance cocerns of missed branch prediction penalize the new
  // manual_wal_flush_ feature (by UNLIKELY) instead of the more common case
  // when we do not need any locking.
  if (UNLIKELY(needs_locking)) {
    log_write_mutex_.Lock();
  }
1020 1021
  IOStatus io_s = log_writer->AddRecord(log_entry);

1022 1023 1024
  if (UNLIKELY(needs_locking)) {
    log_write_mutex_.Unlock();
  }
1025 1026 1027
  if (log_used != nullptr) {
    *log_used = logfile_number_;
  }
S
Siying Dong 已提交
1028
  total_log_size_ += log_entry.size();
1029 1030
  // TODO(myabandeh): it might be unsafe to access alive_log_files_.back() here
  // since alive_log_files_ might be modified concurrently
S
Siying Dong 已提交
1031 1032
  alive_log_files_.back().AddSize(log_entry.size());
  log_empty_ = false;
1033
  return io_s;
1034 1035
}

1036 1037 1038 1039 1040
IOStatus DBImpl::WriteToWAL(const WriteThread::WriteGroup& write_group,
                            log::Writer* log_writer, uint64_t* log_used,
                            bool need_log_sync, bool need_log_dir_sync,
                            SequenceNumber sequence) {
  IOStatus io_s;
1041 1042
  assert(!write_group.leader->disable_wal);
  // Same holds for all in the batch group
1043
  size_t write_with_wal = 0;
1044 1045 1046
  WriteBatch* to_be_cached_state = nullptr;
  WriteBatch* merged_batch = MergeBatch(write_group, &tmp_batch_,
                                        &write_with_wal, &to_be_cached_state);
M
Maysam Yabandeh 已提交
1047 1048 1049 1050 1051 1052 1053
  if (merged_batch == write_group.leader->batch) {
    write_group.leader->log_used = logfile_number_;
  } else if (write_with_wal > 1) {
    for (auto writer : write_group) {
      writer->log_used = logfile_number_;
    }
  }
1054 1055 1056 1057

  WriteBatchInternal::SetSequence(merged_batch, sequence);

  uint64_t log_size;
1058
  io_s = WriteToWAL(*merged_batch, log_writer, log_used, &log_size);
1059 1060
  if (to_be_cached_state) {
    cached_recoverable_state_ = *to_be_cached_state;
1061
    cached_recoverable_state_empty_ = false;
1062
  }
S
Siying Dong 已提交
1063

1064
  if (io_s.ok() && need_log_sync) {
S
Siying Dong 已提交
1065 1066 1067 1068 1069 1070 1071 1072 1073
    StopWatch sw(env_, stats_, WAL_FILE_SYNC_MICROS);
    // It's safe to access logs_ with unlocked mutex_ here because:
    //  - we've set getting_synced=true for all logs,
    //    so other threads won't pop from logs_ while we're here,
    //  - only writer thread can push to logs_, and we're in
    //    writer thread, so no one will push to logs_,
    //  - as long as other threads don't modify it, it's safe to read
    //    from std::deque from multiple threads concurrently.
    for (auto& log : logs_) {
1074 1075
      io_s = log.writer->file()->Sync(immutable_db_options_.use_fsync);
      if (!io_s.ok()) {
S
Siying Dong 已提交
1076 1077 1078
        break;
      }
    }
1079 1080

    if (io_s.ok() && need_log_dir_sync) {
S
Siying Dong 已提交
1081 1082 1083
      // We only sync WAL directory the first time WAL syncing is
      // requested, so that in case users never turn on WAL sync,
      // we can avoid the disk I/O in the write code path.
1084
      io_s = directories_.GetWalDir()->Fsync(IOOptions(), nullptr);
S
Siying Dong 已提交
1085 1086 1087 1088 1089 1090
    }
  }

  if (merged_batch == &tmp_batch_) {
    tmp_batch_.Clear();
  }
1091
  if (io_s.ok()) {
S
Siying Dong 已提交
1092 1093
    auto stats = default_cf_internal_stats_;
    if (need_log_sync) {
1094
      stats->AddDBStats(InternalStats::kIntStatsWalFileSynced, 1);
S
Siying Dong 已提交
1095 1096
      RecordTick(stats_, WAL_FILE_SYNCED);
    }
1097
    stats->AddDBStats(InternalStats::kIntStatsWalFileBytes, log_size);
S
Siying Dong 已提交
1098
    RecordTick(stats_, WAL_FILE_BYTES, log_size);
1099
    stats->AddDBStats(InternalStats::kIntStatsWriteWithWal, write_with_wal);
S
Siying Dong 已提交
1100 1101
    RecordTick(stats_, WRITE_WITH_WAL, write_with_wal);
  }
1102
  return io_s;
S
Siying Dong 已提交
1103 1104
}

1105 1106 1107 1108
IOStatus DBImpl::ConcurrentWriteToWAL(
    const WriteThread::WriteGroup& write_group, uint64_t* log_used,
    SequenceNumber* last_sequence, size_t seq_inc) {
  IOStatus io_s;
1109

1110 1111
  assert(!write_group.leader->disable_wal);
  // Same holds for all in the batch group
1112 1113
  WriteBatch tmp_batch;
  size_t write_with_wal = 0;
1114
  WriteBatch* to_be_cached_state = nullptr;
1115
  WriteBatch* merged_batch =
1116
      MergeBatch(write_group, &tmp_batch, &write_with_wal, &to_be_cached_state);
1117 1118 1119 1120

  // We need to lock log_write_mutex_ since logs_ and alive_log_files might be
  // pushed back concurrently
  log_write_mutex_.Lock();
M
Maysam Yabandeh 已提交
1121 1122 1123 1124 1125 1126 1127
  if (merged_batch == write_group.leader->batch) {
    write_group.leader->log_used = logfile_number_;
  } else if (write_with_wal > 1) {
    for (auto writer : write_group) {
      writer->log_used = logfile_number_;
    }
  }
1128
  *last_sequence = versions_->FetchAddLastAllocatedSequence(seq_inc);
1129 1130 1131 1132 1133
  auto sequence = *last_sequence + 1;
  WriteBatchInternal::SetSequence(merged_batch, sequence);

  log::Writer* log_writer = logs_.back().writer;
  uint64_t log_size;
1134
  io_s = WriteToWAL(*merged_batch, log_writer, log_used, &log_size);
1135 1136
  if (to_be_cached_state) {
    cached_recoverable_state_ = *to_be_cached_state;
1137
    cached_recoverable_state_empty_ = false;
1138
  }
1139 1140
  log_write_mutex_.Unlock();

1141
  if (io_s.ok()) {
1142
    const bool concurrent = true;
1143
    auto stats = default_cf_internal_stats_;
1144 1145
    stats->AddDBStats(InternalStats::kIntStatsWalFileBytes, log_size,
                      concurrent);
1146
    RecordTick(stats_, WAL_FILE_BYTES, log_size);
1147
    stats->AddDBStats(InternalStats::kIntStatsWriteWithWal, write_with_wal,
1148
                      concurrent);
1149 1150
    RecordTick(stats_, WRITE_WITH_WAL, write_with_wal);
  }
1151
  return io_s;
1152 1153
}

1154 1155 1156 1157 1158
Status DBImpl::WriteRecoverableState() {
  mutex_.AssertHeld();
  if (!cached_recoverable_state_empty_) {
    bool dont_care_bool;
    SequenceNumber next_seq;
1159
    if (two_write_queues_) {
1160 1161
      log_write_mutex_.Lock();
    }
1162 1163 1164 1165 1166 1167
    SequenceNumber seq;
    if (two_write_queues_) {
      seq = versions_->FetchAddLastAllocatedSequence(0);
    } else {
      seq = versions_->LastSequence();
    }
1168
    WriteBatchInternal::SetSequence(&cached_recoverable_state_, seq + 1);
1169 1170
    auto status = WriteBatchInternal::InsertInto(
        &cached_recoverable_state_, column_family_memtables_.get(),
1171 1172 1173
        &flush_scheduler_, &trim_history_scheduler_, true,
        0 /*recovery_log_number*/, this, false /* concurrent_memtable_writes */,
        &next_seq, &dont_care_bool, seq_per_batch_);
1174 1175 1176
    auto last_seq = next_seq - 1;
    if (two_write_queues_) {
      versions_->FetchAddLastAllocatedSequence(last_seq - seq);
1177
      versions_->SetLastPublishedSequence(last_seq);
1178 1179
    }
    versions_->SetLastSequence(last_seq);
1180
    if (two_write_queues_) {
1181 1182
      log_write_mutex_.Unlock();
    }
1183 1184 1185 1186
    if (status.ok() && recoverable_state_pre_release_callback_) {
      const bool DISABLE_MEMTABLE = true;
      for (uint64_t sub_batch_seq = seq + 1;
           sub_batch_seq < next_seq && status.ok(); sub_batch_seq++) {
1187
        uint64_t const no_log_num = 0;
1188 1189 1190
        // Unlock it since the callback might end up locking mutex. e.g.,
        // AddCommitted -> AdvanceMaxEvictedSeq -> GetSnapshotListFromDB
        mutex_.Unlock();
1191
        status = recoverable_state_pre_release_callback_->Callback(
1192
            sub_batch_seq, !DISABLE_MEMTABLE, no_log_num, 0, 1);
1193
        mutex_.Lock();
1194 1195
      }
    }
1196 1197 1198 1199 1200 1201 1202 1203 1204
    if (status.ok()) {
      cached_recoverable_state_.Clear();
      cached_recoverable_state_empty_ = true;
    }
    return status;
  }
  return Status::OK();
}

Y
Yanqin Jin 已提交
1205 1206 1207 1208 1209 1210 1211 1212 1213 1214 1215 1216 1217 1218 1219
void DBImpl::SelectColumnFamiliesForAtomicFlush(
    autovector<ColumnFamilyData*>* cfds) {
  for (ColumnFamilyData* cfd : *versions_->GetColumnFamilySet()) {
    if (cfd->IsDropped()) {
      continue;
    }
    if (cfd->imm()->NumNotFlushed() != 0 || !cfd->mem()->IsEmpty() ||
        !cached_recoverable_state_empty_.load()) {
      cfds->push_back(cfd);
    }
  }
}

// Assign sequence number for atomic flush.
void DBImpl::AssignAtomicFlushSeq(const autovector<ColumnFamilyData*>& cfds) {
1220
  assert(immutable_db_options_.atomic_flush);
Y
Yanqin Jin 已提交
1221 1222 1223 1224 1225 1226
  auto seq = versions_->LastSequence();
  for (auto cfd : cfds) {
    cfd->imm()->AssignAtomicFlushSeq(seq);
  }
}

1227
Status DBImpl::SwitchWAL(WriteContext* write_context) {
S
Siying Dong 已提交
1228 1229 1230 1231 1232 1233 1234 1235 1236
  mutex_.AssertHeld();
  assert(write_context != nullptr);
  Status status;

  if (alive_log_files_.begin()->getting_flushed) {
    return status;
  }

  auto oldest_alive_log = alive_log_files_.begin()->number;
S
Siying Dong 已提交
1237 1238
  bool flush_wont_release_oldest_log = false;
  if (allow_2pc()) {
F
Faustin Lammler 已提交
1239
    auto oldest_log_with_uncommitted_prep =
S
Siying Dong 已提交
1240 1241
        logs_with_prep_tracker_.FindMinLogContainingOutstandingPrep();

F
Faustin Lammler 已提交
1242 1243 1244 1245
    assert(oldest_log_with_uncommitted_prep == 0 ||
           oldest_log_with_uncommitted_prep >= oldest_alive_log);
    if (oldest_log_with_uncommitted_prep > 0 &&
        oldest_log_with_uncommitted_prep == oldest_alive_log) {
S
Siying Dong 已提交
1246
      if (unable_to_release_oldest_log_) {
S
Siying Dong 已提交
1247
        // we already attempted to flush all column families dependent on
F
Faustin Lammler 已提交
1248
        // the oldest alive log but the log still contained uncommitted
S
Siying Dong 已提交
1249
        // transactions so there is still nothing that we can do.
S
Siying Dong 已提交
1250
        return status;
S
Siying Dong 已提交
1251 1252 1253
      } else {
        ROCKS_LOG_WARN(
            immutable_db_options_.info_log,
F
Faustin Lammler 已提交
1254
            "Unable to release oldest log due to uncommitted transaction");
S
Siying Dong 已提交
1255 1256 1257
        unable_to_release_oldest_log_ = true;
        flush_wont_release_oldest_log = true;
      }
S
Siying Dong 已提交
1258
    }
S
Siying Dong 已提交
1259 1260
  }
  if (!flush_wont_release_oldest_log) {
S
Siying Dong 已提交
1261 1262
    // we only mark this log as getting flushed if we have successfully
    // flushed all data in this log. If this log contains outstanding prepared
1263 1264
    // transactions then we cannot flush this log until those transactions are
    // commited.
S
Siying Dong 已提交
1265
    unable_to_release_oldest_log_ = false;
S
Siying Dong 已提交
1266 1267 1268
    alive_log_files_.begin()->getting_flushed = true;
  }

1269 1270 1271 1272 1273
  ROCKS_LOG_INFO(
      immutable_db_options_.info_log,
      "Flushing all column families with data in WAL number %" PRIu64
      ". Total log size is %" PRIu64 " while max_total_wal_size is %" PRIu64,
      oldest_alive_log, total_log_size_.load(), GetMaxTotalWalSize());
S
Siying Dong 已提交
1274 1275
  // no need to refcount because drop is happening in write thread, so can't
  // happen while we're in the write thread
Y
Yanqin Jin 已提交
1276
  autovector<ColumnFamilyData*> cfds;
1277
  if (immutable_db_options_.atomic_flush) {
Y
Yanqin Jin 已提交
1278 1279 1280 1281 1282
    SelectColumnFamiliesForAtomicFlush(&cfds);
  } else {
    for (auto cfd : *versions_->GetColumnFamilySet()) {
      if (cfd->IsDropped()) {
        continue;
S
Siying Dong 已提交
1283
      }
Y
Yanqin Jin 已提交
1284 1285 1286 1287
      if (cfd->OldestLogToKeep() <= oldest_alive_log) {
        cfds.push_back(cfd);
      }
    }
1288
    MaybeFlushStatsCF(&cfds);
Y
Yanqin Jin 已提交
1289
  }
1290 1291 1292 1293 1294
  WriteThread::Writer nonmem_w;
  if (two_write_queues_) {
    nonmem_write_thread_.EnterUnbatched(&nonmem_w, &mutex_);
  }

Y
Yanqin Jin 已提交
1295 1296 1297
  for (const auto cfd : cfds) {
    cfd->Ref();
    status = SwitchMemtable(cfd, write_context);
1298
    cfd->UnrefAndTryDelete();
Y
Yanqin Jin 已提交
1299 1300
    if (!status.ok()) {
      break;
S
Siying Dong 已提交
1301 1302
    }
  }
1303 1304 1305 1306
  if (two_write_queues_) {
    nonmem_write_thread_.ExitUnbatched(&nonmem_w);
  }

1307
  if (status.ok()) {
1308
    if (immutable_db_options_.atomic_flush) {
Y
Yanqin Jin 已提交
1309 1310 1311 1312 1313 1314 1315
      AssignAtomicFlushSeq(cfds);
    }
    for (auto cfd : cfds) {
      cfd->imm()->FlushRequested();
    }
    FlushRequest flush_req;
    GenerateFlushRequest(cfds, &flush_req);
1316 1317 1318
    SchedulePendingFlush(flush_req, FlushReason::kWriteBufferManager);
    MaybeScheduleFlushOrCompaction();
  }
S
Siying Dong 已提交
1319 1320 1321 1322 1323 1324 1325 1326 1327 1328 1329 1330 1331 1332 1333
  return status;
}

Status DBImpl::HandleWriteBufferFull(WriteContext* write_context) {
  mutex_.AssertHeld();
  assert(write_context != nullptr);
  Status status;

  // Before a new memtable is added in SwitchMemtable(),
  // write_buffer_manager_->ShouldFlush() will keep returning true. If another
  // thread is writing to another DB with the same write buffer, they may also
  // be flushed. We may end up with flushing much more DBs than needed. It's
  // suboptimal but still correct.
  ROCKS_LOG_INFO(
      immutable_db_options_.info_log,
1334
      "Flushing column family with oldest memtable entry. Write buffer is "
1335
      "using %" ROCKSDB_PRIszt " bytes out of a total of %" ROCKSDB_PRIszt ".",
S
Siying Dong 已提交
1336 1337 1338 1339
      write_buffer_manager_->memory_usage(),
      write_buffer_manager_->buffer_size());
  // no need to refcount because drop is happening in write thread, so can't
  // happen while we're in the write thread
Y
Yanqin Jin 已提交
1340
  autovector<ColumnFamilyData*> cfds;
1341
  if (immutable_db_options_.atomic_flush) {
Y
Yanqin Jin 已提交
1342 1343 1344 1345
    SelectColumnFamiliesForAtomicFlush(&cfds);
  } else {
    ColumnFamilyData* cfd_picked = nullptr;
    SequenceNumber seq_num_for_cf_picked = kMaxSequenceNumber;
S
Siying Dong 已提交
1346

Y
Yanqin Jin 已提交
1347 1348 1349
    for (auto cfd : *versions_->GetColumnFamilySet()) {
      if (cfd->IsDropped()) {
        continue;
S
Siying Dong 已提交
1350
      }
Y
Yanqin Jin 已提交
1351 1352 1353 1354 1355 1356 1357 1358 1359 1360 1361 1362
      if (!cfd->mem()->IsEmpty()) {
        // We only consider active mem table, hoping immutable memtable is
        // already in the process of flushing.
        uint64_t seq = cfd->mem()->GetCreationSeq();
        if (cfd_picked == nullptr || seq < seq_num_for_cf_picked) {
          cfd_picked = cfd;
          seq_num_for_cf_picked = seq;
        }
      }
    }
    if (cfd_picked != nullptr) {
      cfds.push_back(cfd_picked);
S
Siying Dong 已提交
1363
    }
1364
    MaybeFlushStatsCF(&cfds);
S
Siying Dong 已提交
1365
  }
1366

1367 1368 1369 1370
  WriteThread::Writer nonmem_w;
  if (two_write_queues_) {
    nonmem_write_thread_.EnterUnbatched(&nonmem_w, &mutex_);
  }
1371
  for (const auto cfd : cfds) {
1372 1373 1374
    if (cfd->mem()->IsEmpty()) {
      continue;
    }
1375 1376
    cfd->Ref();
    status = SwitchMemtable(cfd, write_context);
1377
    cfd->UnrefAndTryDelete();
1378 1379
    if (!status.ok()) {
      break;
S
Siying Dong 已提交
1380
    }
1381
  }
1382 1383 1384 1385
  if (two_write_queues_) {
    nonmem_write_thread_.ExitUnbatched(&nonmem_w);
  }

1386
  if (status.ok()) {
1387
    if (immutable_db_options_.atomic_flush) {
Y
Yanqin Jin 已提交
1388 1389 1390 1391 1392 1393 1394
      AssignAtomicFlushSeq(cfds);
    }
    for (const auto cfd : cfds) {
      cfd->imm()->FlushRequested();
    }
    FlushRequest flush_req;
    GenerateFlushRequest(cfds, &flush_req);
1395 1396
    SchedulePendingFlush(flush_req, FlushReason::kWriteBufferFull);
    MaybeScheduleFlushOrCompaction();
S
Siying Dong 已提交
1397 1398 1399 1400 1401 1402 1403 1404 1405 1406 1407 1408 1409 1410 1411 1412 1413 1414 1415 1416 1417 1418
  }
  return status;
}

uint64_t DBImpl::GetMaxTotalWalSize() const {
  mutex_.AssertHeld();
  return mutable_db_options_.max_total_wal_size == 0
             ? 4 * max_total_in_memory_state_
             : mutable_db_options_.max_total_wal_size;
}

// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
Status DBImpl::DelayWrite(uint64_t num_bytes,
                          const WriteOptions& write_options) {
  uint64_t time_delayed = 0;
  bool delayed = false;
  {
    StopWatch sw(env_, stats_, WRITE_STALL, &time_delayed);
    uint64_t delay = write_controller_.GetDelay(env_, num_bytes);
    if (delay > 0) {
      if (write_options.no_slowdown) {
1419
        return Status::Incomplete("Write stall");
S
Siying Dong 已提交
1420 1421 1422
      }
      TEST_SYNC_POINT("DBImpl::DelayWrite:Sleep");

1423 1424 1425 1426
      // Notify write_thread_ about the stall so it can setup a barrier and
      // fail any pending writers with no_slowdown
      write_thread_.BeginWriteStall();
      TEST_SYNC_POINT("DBImpl::DelayWrite:BeginWriteStallDone");
S
Siying Dong 已提交
1427 1428 1429 1430 1431 1432 1433 1434 1435 1436 1437 1438 1439 1440 1441 1442
      mutex_.Unlock();
      // We will delay the write until we have slept for delay ms or
      // we don't need a delay anymore
      const uint64_t kDelayInterval = 1000;
      uint64_t stall_end = sw.start_time() + delay;
      while (write_controller_.NeedsDelay()) {
        if (env_->NowMicros() >= stall_end) {
          // We already delayed this write `delay` microseconds
          break;
        }

        delayed = true;
        // Sleep for 0.001 seconds
        env_->SleepForMicroseconds(kDelayInterval);
      }
      mutex_.Lock();
1443
      write_thread_.EndWriteStall();
S
Siying Dong 已提交
1444 1445
    }

1446 1447 1448 1449 1450
    // Don't wait if there's a background error, even if its a soft error. We
    // might wait here indefinitely as the background compaction may never
    // finish successfully, resulting in the stall condition lasting
    // indefinitely
    while (error_handler_.GetBGError().ok() && write_controller_.IsStopped()) {
S
Siying Dong 已提交
1451
      if (write_options.no_slowdown) {
1452
        return Status::Incomplete("Write stall");
S
Siying Dong 已提交
1453 1454
      }
      delayed = true;
1455 1456 1457 1458

      // Notify write_thread_ about the stall so it can setup a barrier and
      // fail any pending writers with no_slowdown
      write_thread_.BeginWriteStall();
S
Siying Dong 已提交
1459 1460
      TEST_SYNC_POINT("DBImpl::DelayWrite:Wait");
      bg_cv_.Wait();
1461
      write_thread_.EndWriteStall();
S
Siying Dong 已提交
1462 1463 1464 1465
    }
  }
  assert(!delayed || !write_options.no_slowdown);
  if (delayed) {
1466 1467
    default_cf_internal_stats_->AddDBStats(
        InternalStats::kIntStatsWriteStallMicros, time_delayed);
S
Siying Dong 已提交
1468 1469 1470
    RecordTick(stats_, STALL_MICROS, time_delayed);
  }

1471 1472 1473 1474 1475 1476 1477 1478 1479 1480 1481 1482 1483
  // If DB is not in read-only mode and write_controller is not stopping
  // writes, we can ignore any background errors and allow the write to
  // proceed
  Status s;
  if (write_controller_.IsStopped()) {
    // If writes are still stopped, it means we bailed due to a background
    // error
    s = Status::Incomplete(error_handler_.GetBGError().ToString());
  }
  if (error_handler_.IsDBStopped()) {
    s = error_handler_.GetBGError();
  }
  return s;
S
Siying Dong 已提交
1484 1485
}

1486 1487 1488 1489 1490 1491 1492 1493 1494 1495 1496 1497 1498 1499
Status DBImpl::ThrottleLowPriWritesIfNeeded(const WriteOptions& write_options,
                                            WriteBatch* my_batch) {
  assert(write_options.low_pri);
  // This is called outside the DB mutex. Although it is safe to make the call,
  // the consistency condition is not guaranteed to hold. It's OK to live with
  // it in this case.
  // If we need to speed compaction, it means the compaction is left behind
  // and we start to limit low pri writes to a limit.
  if (write_controller_.NeedSpeedupCompaction()) {
    if (allow_2pc() && (my_batch->HasCommit() || my_batch->HasRollback())) {
      // For 2PC, we only rate limit prepare, not commit.
      return Status::OK();
    }
    if (write_options.no_slowdown) {
1500
      return Status::Incomplete("Low priority write stall");
1501 1502 1503 1504 1505 1506
    } else {
      assert(my_batch != nullptr);
      // Rate limit those writes. The reason that we don't completely wait
      // is that in case the write is heavy, low pri writes may never have
      // a chance to run. Now we guarantee we are still slowly making
      // progress.
1507
      PERF_TIMER_GUARD(write_delay_time);
1508 1509 1510
      write_controller_.low_pri_rate_limiter()->Request(
          my_batch->GetDataSize(), Env::IO_HIGH, nullptr /* stats */,
          RateLimiter::OpType::kWrite);
1511 1512 1513 1514 1515
    }
  }
  return Status::OK();
}

1516 1517 1518 1519 1520 1521 1522 1523 1524 1525 1526 1527 1528 1529 1530 1531 1532 1533 1534 1535 1536 1537 1538 1539 1540 1541 1542 1543 1544 1545 1546 1547 1548 1549
void DBImpl::MaybeFlushStatsCF(autovector<ColumnFamilyData*>* cfds) {
  assert(cfds != nullptr);
  if (!cfds->empty() && immutable_db_options_.persist_stats_to_disk) {
    ColumnFamilyData* cfd_stats =
        versions_->GetColumnFamilySet()->GetColumnFamily(
            kPersistentStatsColumnFamilyName);
    if (cfd_stats != nullptr && !cfd_stats->mem()->IsEmpty()) {
      for (ColumnFamilyData* cfd : *cfds) {
        if (cfd == cfd_stats) {
          // stats CF already included in cfds
          return;
        }
      }
      // force flush stats CF when its log number is less than all other CF's
      // log numbers
      bool force_flush_stats_cf = true;
      for (auto* loop_cfd : *versions_->GetColumnFamilySet()) {
        if (loop_cfd == cfd_stats) {
          continue;
        }
        if (loop_cfd->GetLogNumber() <= cfd_stats->GetLogNumber()) {
          force_flush_stats_cf = false;
        }
      }
      if (force_flush_stats_cf) {
        cfds->push_back(cfd_stats);
        ROCKS_LOG_INFO(immutable_db_options_.info_log,
                       "Force flushing stats CF with automated flush "
                       "to avoid holding old logs");
      }
    }
  }
}

1550 1551 1552 1553 1554 1555 1556 1557 1558
Status DBImpl::TrimMemtableHistory(WriteContext* context) {
  autovector<ColumnFamilyData*> cfds;
  ColumnFamilyData* tmp_cfd;
  while ((tmp_cfd = trim_history_scheduler_.TakeNextColumnFamily()) !=
         nullptr) {
    cfds.push_back(tmp_cfd);
  }
  for (auto& cfd : cfds) {
    autovector<MemTable*> to_delete;
1559 1560
    bool trimmed = cfd->imm()->TrimHistory(
        &to_delete, cfd->mem()->ApproximateMemoryUsage());
1561 1562 1563 1564
    if (!to_delete.empty()) {
      for (auto m : to_delete) {
        delete m;
      }
1565 1566
    }
    if (trimmed) {
1567 1568 1569
      context->superversion_context.NewSuperVersion();
      assert(context->superversion_context.new_superversion.get() != nullptr);
      cfd->InstallSuperVersion(&context->superversion_context, &mutex_);
1570 1571
    }

1572
    if (cfd->UnrefAndTryDelete()) {
1573 1574 1575 1576 1577 1578
      cfd = nullptr;
    }
  }
  return Status::OK();
}

S
Siying Dong 已提交
1579
Status DBImpl::ScheduleFlushes(WriteContext* context) {
Y
Yanqin Jin 已提交
1580
  autovector<ColumnFamilyData*> cfds;
1581
  if (immutable_db_options_.atomic_flush) {
Y
Yanqin Jin 已提交
1582 1583 1584 1585 1586 1587 1588 1589 1590 1591
    SelectColumnFamiliesForAtomicFlush(&cfds);
    for (auto cfd : cfds) {
      cfd->Ref();
    }
    flush_scheduler_.Clear();
  } else {
    ColumnFamilyData* tmp_cfd;
    while ((tmp_cfd = flush_scheduler_.TakeNextColumnFamily()) != nullptr) {
      cfds.push_back(tmp_cfd);
    }
1592
    MaybeFlushStatsCF(&cfds);
Y
Yanqin Jin 已提交
1593
  }
1594
  Status status;
1595 1596 1597 1598 1599
  WriteThread::Writer nonmem_w;
  if (two_write_queues_) {
    nonmem_write_thread_.EnterUnbatched(&nonmem_w, &mutex_);
  }

Y
Yanqin Jin 已提交
1600
  for (auto& cfd : cfds) {
1601 1602 1603
    if (!cfd->mem()->IsEmpty()) {
      status = SwitchMemtable(cfd, context);
    }
1604
    if (cfd->UnrefAndTryDelete()) {
Y
Yanqin Jin 已提交
1605
      cfd = nullptr;
S
Siying Dong 已提交
1606 1607
    }
    if (!status.ok()) {
1608 1609
      break;
    }
S
Siying Dong 已提交
1610
  }
1611 1612 1613 1614 1615

  if (two_write_queues_) {
    nonmem_write_thread_.ExitUnbatched(&nonmem_w);
  }

1616
  if (status.ok()) {
1617
    if (immutable_db_options_.atomic_flush) {
Y
Yanqin Jin 已提交
1618 1619 1620 1621
      AssignAtomicFlushSeq(cfds);
    }
    FlushRequest flush_req;
    GenerateFlushRequest(cfds, &flush_req);
1622 1623 1624 1625
    SchedulePendingFlush(flush_req, FlushReason::kWriteBufferFull);
    MaybeScheduleFlushOrCompaction();
  }
  return status;
S
Siying Dong 已提交
1626 1627 1628
}

#ifndef ROCKSDB_LITE
A
Andrew Kryczka 已提交
1629
void DBImpl::NotifyOnMemTableSealed(ColumnFamilyData* /*cfd*/,
S
Siying Dong 已提交
1630 1631 1632 1633 1634 1635 1636 1637 1638 1639 1640 1641 1642 1643 1644 1645
                                    const MemTableInfo& mem_table_info) {
  if (immutable_db_options_.listeners.size() == 0U) {
    return;
  }
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }

  for (auto listener : immutable_db_options_.listeners) {
    listener->OnMemTableSealed(mem_table_info);
  }
}
#endif  // ROCKSDB_LITE

// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
1646 1647
// REQUIRES: this thread is currently at the front of the 2nd writer queue if
// two_write_queues_ is true (This is to simplify the reasoning.)
1648
Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) {
S
Siying Dong 已提交
1649
  mutex_.AssertHeld();
M
Maysam Yabandeh 已提交
1650
  WriteThread::Writer nonmem_w;
1651
  std::unique_ptr<WritableFile> lfile;
S
Siying Dong 已提交
1652 1653
  log::Writer* new_log = nullptr;
  MemTable* new_mem = nullptr;
1654
  IOStatus io_s;
S
Siying Dong 已提交
1655

1656 1657 1658 1659 1660 1661 1662
  // Recoverable state is persisted in WAL. After memtable switch, WAL might
  // be deleted, so we write the state to memtable to be persisted as well.
  Status s = WriteRecoverableState();
  if (!s.ok()) {
    return s;
  }

S
Siying Dong 已提交
1663 1664 1665
  // Attempt to switch to a new memtable and trigger flush of old.
  // Do this without holding the dbmutex lock.
  assert(versions_->prev_log_number() == 0);
1666
  if (two_write_queues_) {
1667 1668
    log_write_mutex_.Lock();
  }
S
Siying Dong 已提交
1669
  bool creating_new_log = !log_empty_;
1670
  if (two_write_queues_) {
1671 1672
    log_write_mutex_.Unlock();
  }
S
Siying Dong 已提交
1673 1674
  uint64_t recycle_log_number = 0;
  if (creating_new_log && immutable_db_options_.recycle_log_file_num &&
1675 1676
      !log_recycle_files_.empty()) {
    recycle_log_number = log_recycle_files_.front();
S
Siying Dong 已提交
1677 1678 1679 1680 1681
  }
  uint64_t new_log_number =
      creating_new_log ? versions_->NewFileNumber() : logfile_number_;
  const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();

F
follitude 已提交
1682
  // Set memtable_info for memtable sealed callback
S
Siying Dong 已提交
1683 1684 1685 1686 1687 1688 1689 1690 1691 1692 1693 1694
#ifndef ROCKSDB_LITE
  MemTableInfo memtable_info;
  memtable_info.cf_name = cfd->GetName();
  memtable_info.first_seqno = cfd->mem()->GetFirstSequenceNumber();
  memtable_info.earliest_seqno = cfd->mem()->GetEarliestSequenceNumber();
  memtable_info.num_entries = cfd->mem()->num_entries();
  memtable_info.num_deletes = cfd->mem()->num_deletes();
#endif  // ROCKSDB_LITE
  // Log this later after lock release. It may be outdated, e.g., if background
  // flush happens before logging, but that should be ok.
  int num_imm_unflushed = cfd->imm()->NumNotFlushed();
  const auto preallocate_block_size =
1695
      GetWalPreallocateBlockSize(mutable_cf_options.write_buffer_size);
S
Siying Dong 已提交
1696
  mutex_.Unlock();
1697 1698 1699
  if (creating_new_log) {
    // TODO: Write buffer size passed in should be max of all CF's instead
    // of mutable_cf_options.write_buffer_size.
1700 1701 1702 1703 1704
    io_s = CreateWAL(new_log_number, recycle_log_number, preallocate_block_size,
                     &new_log);
    if (s.ok()) {
      s = io_s;
    }
1705 1706 1707 1708 1709
  }
  if (s.ok()) {
    SequenceNumber seq = versions_->LastSequence();
    new_mem = cfd->ConstructNewMemtable(mutable_cf_options, seq);
    context->superversion_context.NewSuperVersion();
S
Siying Dong 已提交
1710 1711 1712 1713 1714 1715
  }
  ROCKS_LOG_INFO(immutable_db_options_.info_log,
                 "[%s] New memtable created with log file: #%" PRIu64
                 ". Immutable memtables: %d.\n",
                 cfd->GetName().c_str(), new_log_number, num_imm_unflushed);
  mutex_.Lock();
1716 1717 1718 1719 1720 1721 1722 1723
  if (recycle_log_number != 0) {
    // Since renaming the file is done outside DB mutex, we need to ensure
    // concurrent full purges don't delete the file while we're recycling it.
    // To achieve that we hold the old log number in the recyclable list until
    // after it has been renamed.
    assert(log_recycle_files_.front() == recycle_log_number);
    log_recycle_files_.pop_front();
  }
1724
  if (s.ok() && creating_new_log) {
M
Maysam Yabandeh 已提交
1725
    log_write_mutex_.Lock();
S
Siying Dong 已提交
1726
    assert(new_log != nullptr);
1727 1728 1729
    if (!logs_.empty()) {
      // Alway flush the buffer of the last log before switching to a new one
      log::Writer* cur_log_writer = logs_.back().writer;
1730 1731 1732 1733
      io_s = cur_log_writer->WriteBuffer();
      if (s.ok()) {
        s = io_s;
      }
1734 1735 1736
      if (!s.ok()) {
        ROCKS_LOG_WARN(immutable_db_options_.info_log,
                       "[%s] Failed to switch from #%" PRIu64 " to #%" PRIu64
1737
                       "  WAL file\n",
1738 1739 1740
                       cfd->GetName().c_str(), cur_log_writer->get_log_number(),
                       new_log_number);
      }
1741
    }
1742 1743 1744 1745 1746 1747 1748
    if (s.ok()) {
      logfile_number_ = new_log_number;
      log_empty_ = true;
      log_dir_synced_ = false;
      logs_.emplace_back(logfile_number_, new_log);
      alive_log_files_.push_back(LogFileNumberSize(logfile_number_));
    }
1749
    log_write_mutex_.Unlock();
S
Siying Dong 已提交
1750
  }
1751 1752 1753 1754

  if (!s.ok()) {
    // how do we fail if we're not creating new log?
    assert(creating_new_log);
1755 1756 1757 1758 1759 1760 1761 1762 1763 1764 1765 1766 1767
    if (new_mem) {
      delete new_mem;
    }
    if (new_log) {
      delete new_log;
    }
    SuperVersion* new_superversion =
        context->superversion_context.new_superversion.release();
    if (new_superversion != nullptr) {
      delete new_superversion;
    }
    // We may have lost data from the WritableFileBuffer in-memory buffer for
    // the current log, so treat it as a fatal error and set bg_error
1768 1769 1770 1771 1772
    if (!io_s.ok()) {
      error_handler_.SetBGError(io_s, BackgroundErrorReason::kMemTable);
    } else {
      error_handler_.SetBGError(s, BackgroundErrorReason::kMemTable);
    }
1773 1774
    // Read back bg_error in order to get the right severity
    s = error_handler_.GetBGError();
1775 1776 1777
    return s;
  }

S
Siying Dong 已提交
1778 1779 1780 1781 1782 1783 1784 1785 1786 1787 1788 1789 1790 1791 1792 1793 1794 1795
  for (auto loop_cfd : *versions_->GetColumnFamilySet()) {
    // all this is just optimization to delete logs that
    // are no longer needed -- if CF is empty, that means it
    // doesn't need that particular log to stay alive, so we just
    // advance the log number. no need to persist this in the manifest
    if (loop_cfd->mem()->GetFirstSequenceNumber() == 0 &&
        loop_cfd->imm()->NumNotFlushed() == 0) {
      if (creating_new_log) {
        loop_cfd->SetLogNumber(logfile_number_);
      }
      loop_cfd->mem()->SetCreationSeq(versions_->LastSequence());
    }
  }

  cfd->mem()->SetNextLogNumber(logfile_number_);
  cfd->imm()->Add(cfd->mem(), &context->memtables_to_free_);
  new_mem->Ref();
  cfd->SetMemtable(new_mem);
1796
  InstallSuperVersionAndScheduleWork(cfd, &context->superversion_context,
1797
                                     mutable_cf_options);
1798 1799 1800 1801 1802 1803 1804
#ifndef ROCKSDB_LITE
  mutex_.Unlock();
  // Notify client that memtable is sealed, now that we have successfully
  // installed a new memtable
  NotifyOnMemTableSealed(cfd, memtable_info);
  mutex_.Lock();
#endif  // ROCKSDB_LITE
S
Siying Dong 已提交
1805 1806 1807 1808 1809
  return s;
}

size_t DBImpl::GetWalPreallocateBlockSize(uint64_t write_buffer_size) const {
  mutex_.AssertHeld();
1810 1811
  size_t bsize =
      static_cast<size_t>(write_buffer_size / 10 + write_buffer_size);
S
Siying Dong 已提交
1812 1813 1814
  // Some users might set very high write_buffer_size and rely on
  // max_total_wal_size or other parameters to control the WAL size.
  if (mutable_db_options_.max_total_wal_size > 0) {
1815 1816
    bsize = std::min<size_t>(
        bsize, static_cast<size_t>(mutable_db_options_.max_total_wal_size));
S
Siying Dong 已提交
1817 1818 1819 1820 1821 1822 1823 1824 1825 1826 1827 1828 1829 1830 1831 1832 1833
  }
  if (immutable_db_options_.db_write_buffer_size > 0) {
    bsize = std::min<size_t>(bsize, immutable_db_options_.db_write_buffer_size);
  }
  if (immutable_db_options_.write_buffer_manager &&
      immutable_db_options_.write_buffer_manager->enabled()) {
    bsize = std::min<size_t>(
        bsize, immutable_db_options_.write_buffer_manager->buffer_size());
  }

  return bsize;
}

// Default implementations of convenience methods that subclasses of DB
// can call if they wish
Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family,
               const Slice& key, const Slice& value) {
1834 1835 1836 1837 1838 1839 1840 1841 1842 1843 1844
  if (nullptr == opt.timestamp) {
    // Pre-allocate size of write batch conservatively.
    // 8 bytes are taken by header, 4 bytes for count, 1 byte for type,
    // and we allocate 11 extra bytes for key length, as well as value length.
    WriteBatch batch(key.size() + value.size() + 24);
    Status s = batch.Put(column_family, key, value);
    if (!s.ok()) {
      return s;
    }
    return Write(opt, &batch);
  }
1845 1846 1847
  const Slice* ts = opt.timestamp;
  assert(nullptr != ts);
  size_t ts_sz = ts->size();
1848 1849
  assert(column_family->GetComparator());
  assert(ts_sz == column_family->GetComparator()->timestamp_size());
1850 1851 1852
  WriteBatch batch(key.size() + ts_sz + value.size() + 24, /*max_bytes=*/0,
                   ts_sz);
  Status s = batch.Put(column_family, key, value);
1853 1854 1855
  if (!s.ok()) {
    return s;
  }
1856
  s = batch.AssignTimestamp(*ts);
1857 1858 1859
  if (!s.ok()) {
    return s;
  }
S
Siying Dong 已提交
1860 1861 1862 1863 1864
  return Write(opt, &batch);
}

Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                  const Slice& key) {
Y
Yanqin Jin 已提交
1865 1866 1867 1868 1869 1870 1871 1872 1873 1874 1875 1876 1877 1878 1879 1880 1881 1882 1883 1884 1885 1886 1887 1888
  if (nullptr == opt.timestamp) {
    WriteBatch batch;
    Status s = batch.Delete(column_family, key);
    if (!s.ok()) {
      return s;
    }
    return Write(opt, &batch);
  }
  const Slice* ts = opt.timestamp;
  assert(ts != nullptr);
  const size_t ts_sz = ts->size();
  constexpr size_t kKeyAndValueLenSize = 11;
  constexpr size_t kWriteBatchOverhead =
      WriteBatchInternal::kHeader + sizeof(ValueType) + kKeyAndValueLenSize;
  WriteBatch batch(key.size() + ts_sz + kWriteBatchOverhead, /*max_bytes=*/0,
                   ts_sz);
  Status s = batch.Delete(column_family, key);
  if (!s.ok()) {
    return s;
  }
  s = batch.AssignTimestamp(*ts);
  if (!s.ok()) {
    return s;
  }
S
Siying Dong 已提交
1889 1890 1891 1892 1893 1894 1895 1896 1897 1898 1899 1900 1901 1902 1903 1904 1905 1906 1907 1908 1909
  return Write(opt, &batch);
}

Status DB::SingleDelete(const WriteOptions& opt,
                        ColumnFamilyHandle* column_family, const Slice& key) {
  WriteBatch batch;
  batch.SingleDelete(column_family, key);
  return Write(opt, &batch);
}

Status DB::DeleteRange(const WriteOptions& opt,
                       ColumnFamilyHandle* column_family,
                       const Slice& begin_key, const Slice& end_key) {
  WriteBatch batch;
  batch.DeleteRange(column_family, begin_key, end_key);
  return Write(opt, &batch);
}

Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                 const Slice& key, const Slice& value) {
  WriteBatch batch;
1910 1911 1912 1913
  Status s = batch.Merge(column_family, key, value);
  if (!s.ok()) {
    return s;
  }
S
Siying Dong 已提交
1914 1915
  return Write(opt, &batch);
}
1916
}  // namespace ROCKSDB_NAMESPACE