db_iter.cc 53.5 KB
Newer Older
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).
5
//
J
jorlow@chromium.org 已提交
6 7 8 9 10
// 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.

#include "db/db_iter.h"
11

12
#include <iostream>
S
Stanislau Hlebik 已提交
13
#include <limits>
14
#include <string>
J
jorlow@chromium.org 已提交
15 16

#include "db/dbformat.h"
17
#include "db/merge_context.h"
18
#include "db/merge_helper.h"
19
#include "db/pinned_iterators_manager.h"
20
#include "file/filename.h"
21
#include "logging/logging.h"
22
#include "memory/arena.h"
23
#include "monitoring/perf_context_imp.h"
24 25 26
#include "rocksdb/env.h"
#include "rocksdb/iterator.h"
#include "rocksdb/merge_operator.h"
27
#include "rocksdb/options.h"
28
#include "rocksdb/system_clock.h"
S
sdong 已提交
29
#include "table/internal_iterator.h"
30
#include "table/iterator_wrapper.h"
31
#include "trace_replay/trace_replay.h"
J
jorlow@chromium.org 已提交
32
#include "util/mutexlock.h"
33
#include "util/string_util.h"
34
#include "util/user_comparator_wrapper.h"
J
jorlow@chromium.org 已提交
35

36
namespace ROCKSDB_NAMESPACE {
J
jorlow@chromium.org 已提交
37

38
DBIter::DBIter(Env* _env, const ReadOptions& read_options,
39
               const ImmutableOptions& cf_options,
40
               const MutableCFOptions& mutable_cf_options,
L
Levi Tamasi 已提交
41 42 43
               const Comparator* cmp, InternalIterator* iter,
               const Version* version, SequenceNumber s, bool arena_mode,
               uint64_t max_sequential_skip_in_iterations,
44
               ReadCallback* read_callback, DBImpl* db_impl,
L
Levi Tamasi 已提交
45
               ColumnFamilyData* cfd, bool expose_blob_index)
46 47
    : prefix_extractor_(mutable_cf_options.prefix_extractor.get()),
      env_(_env),
48
      clock_(cf_options.clock),
49
      logger_(cf_options.logger),
50
      user_comparator_(cmp),
51
      merge_operator_(cf_options.merge_operator.get()),
52
      iter_(iter),
L
Levi Tamasi 已提交
53
      version_(version),
54 55
      read_callback_(read_callback),
      sequence_(s),
56
      statistics_(cf_options.stats),
Y
Yanqin Jin 已提交
57 58
      max_skip_(max_sequential_skip_in_iterations),
      max_skippable_internal_keys_(read_options.max_skippable_internal_keys),
59 60 61 62 63 64 65
      num_internal_keys_skipped_(0),
      iterate_lower_bound_(read_options.iterate_lower_bound),
      iterate_upper_bound_(read_options.iterate_upper_bound),
      direction_(kForward),
      valid_(false),
      current_entry_is_merged_(false),
      is_key_seqnum_zero_(false),
66 67 68
      prefix_same_as_start_(mutable_cf_options.prefix_extractor
                                ? read_options.prefix_same_as_start
                                : false),
69
      pin_thru_lifetime_(read_options.pin_data),
S
sdong 已提交
70 71 72
      expect_total_order_inner_iter_(prefix_extractor_ == nullptr ||
                                     read_options.total_order_seek ||
                                     read_options.auto_prefix_mode),
L
Levi Tamasi 已提交
73 74 75
      read_tier_(read_options.read_tier),
      verify_checksums_(read_options.verify_checksums),
      expose_blob_index_(expose_blob_index),
76 77 78 79 80
      is_blob_(false),
      arena_mode_(arena_mode),
      range_del_agg_(&cf_options.internal_comparator, s),
      db_impl_(db_impl),
      cfd_(cfd),
Y
Yanqin Jin 已提交
81 82
      start_seqnum_(read_options.iter_start_seqnum),
      timestamp_ub_(read_options.timestamp),
83
      timestamp_lb_(read_options.iter_start_ts),
Y
Yanqin Jin 已提交
84
      timestamp_size_(timestamp_ub_ ? timestamp_ub_->size() : 0) {
85 86 87 88 89
  RecordTick(statistics_, NO_ITERATOR_CREATED);
  if (pin_thru_lifetime_) {
    pinned_iters_mgr_.StartPinning();
  }
  if (iter_.iter()) {
90
    iter_.iter()->SetPinnedItersMgr(&pinned_iters_mgr_);
J
jorlow@chromium.org 已提交
91
  }
Y
Yanqin Jin 已提交
92
  assert(timestamp_size_ == user_comparator_.timestamp_size());
93
}
94

95 96 97
Status DBIter::GetProperty(std::string prop_name, std::string* prop) {
  if (prop == nullptr) {
    return Status::InvalidArgument("prop is nullptr");
98
  }
99 100 101 102
  if (prop_name == "rocksdb.iterator.super-version-number") {
    // First try to pass the value returned from inner iterator.
    return iter_.iter()->GetProperty(prop_name, prop);
  } else if (prop_name == "rocksdb.iterator.is-key-pinned") {
103
    if (valid_) {
104 105 106
      *prop = (pin_thru_lifetime_ && saved_key_.IsKeyPinned()) ? "1" : "0";
    } else {
      *prop = "Iterator is not valid.";
107
    }
108 109 110 111
    return Status::OK();
  } else if (prop_name == "rocksdb.iterator.internal-key") {
    *prop = saved_key_.GetUserKey().ToString();
    return Status::OK();
112
  }
113 114
  return Status::InvalidArgument("Unidentified property.");
}
115

116
bool DBIter::ParseKey(ParsedInternalKey* ikey) {
117
  Status s = ParseInternalKey(iter_.key(), ikey, false /* log_err_key */);
118 119
  if (!s.ok()) {
    status_ = Status::Corruption("In DBIter: ", s.getState());
120
    valid_ = false;
121
    ROCKS_LOG_ERROR(logger_, "In DBIter: %s", status_.getState());
J
jorlow@chromium.org 已提交
122 123 124 125 126 127
    return false;
  } else {
    return true;
  }
}

J
jorlow@chromium.org 已提交
128 129
void DBIter::Next() {
  assert(valid_);
130
  assert(status_.ok());
J
jorlow@chromium.org 已提交
131

132
  PERF_CPU_TIMER_GUARD(iter_next_cpu_nanos, clock_);
133 134
  // Release temporarily pinned blocks from last operation
  ReleaseTempPinnedData();
135 136 137
  local_stats_.skip_count_ += num_internal_keys_skipped_;
  local_stats_.skip_count_--;
  num_internal_keys_skipped_ = 0;
138
  bool ok = true;
S
Stanislau Hlebik 已提交
139
  if (direction_ == kReverse) {
140
    is_key_seqnum_zero_ = false;
141 142 143
    if (!ReverseToForward()) {
      ok = false;
    }
144
  } else if (!current_entry_is_merged_) {
145 146 147 148 149
    // If the current value is not a merge, the iter position is the
    // current key, which is already returned. We can safely issue a
    // Next() without checking the current key.
    // If the current key is a merge, very likely iter already points
    // to the next internal position.
150 151
    assert(iter_.Valid());
    iter_.Next();
152
    PERF_COUNTER_ADD(internal_key_skipped_count, 1);
J
jorlow@chromium.org 已提交
153
  }
J
jorlow@chromium.org 已提交
154

155
  local_stats_.next_count_++;
156
  if (ok && iter_.Valid()) {
157 158
    if (prefix_same_as_start_) {
      assert(prefix_extractor_ != nullptr);
159 160 161 162
      const Slice prefix = prefix_.GetUserKey();
      FindNextUserEntry(true /* skipping the current user key */, &prefix);
    } else {
      FindNextUserEntry(true /* skipping the current user key */, nullptr);
163
    }
164
  } else {
165
    is_key_seqnum_zero_ = false;
166 167
    valid_ = false;
  }
168 169 170 171
  if (statistics_ != nullptr && valid_) {
    local_stats_.next_found_count_++;
    local_stats_.bytes_read_ += (key().size() + value().size());
  }
J
jorlow@chromium.org 已提交
172 173
}

L
Levi Tamasi 已提交
174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194
bool DBIter::SetBlobValueIfNeeded(const Slice& user_key,
                                  const Slice& blob_index) {
  assert(!is_blob_);

  if (expose_blob_index_) {  // Stacked BlobDB implementation
    is_blob_ = true;
    return true;
  }

  if (!version_) {
    status_ = Status::Corruption("Encountered unexpected blob index.");
    valid_ = false;
    return false;
  }

  // TODO: consider moving ReadOptions from ArenaWrappedDBIter to DBIter to
  // avoid having to copy options back and forth.
  ReadOptions read_options;
  read_options.read_tier = read_tier_;
  read_options.verify_checksums = verify_checksums_;

195 196 197 198
  constexpr uint64_t* bytes_read = nullptr;

  const Status s = version_->GetBlob(read_options, user_key, blob_index,
                                     &blob_value_, bytes_read);
L
Levi Tamasi 已提交
199 200 201 202 203 204 205 206 207 208 209

  if (!s.ok()) {
    status_ = s;
    valid_ = false;
    return false;
  }

  is_blob_ = true;
  return true;
}

210
// PRE: saved_key_ has the current user key if skipping_saved_key
211 212 213 214 215 216
// POST: saved_key_ should have the next user key if valid_,
//       if the current entry is a result of merge
//           current_entry_is_merged_ => true
//           saved_value_             => the merged value
//
// NOTE: In between, saved_key_ can point to a user key that has
217
//       a delete marker or a sequence number higher than sequence_
218
//       saved_key_ MUST have a proper user_key before calling this function
219
//
Y
Yanqin Jin 已提交
220
// The prefix parameter, if not null, indicates that we need to iterate
221 222 223
// within the prefix, and the iterator needs to be made invalid, if no
// more entry for the prefix can be found.
bool DBIter::FindNextUserEntry(bool skipping_saved_key, const Slice* prefix) {
224
  PERF_TIMER_GUARD(find_next_user_entry_time);
225
  return FindNextUserEntryInternal(skipping_saved_key, prefix);
226 227 228
}

// Actual implementation of DBIter::FindNextUserEntry()
229 230
bool DBIter::FindNextUserEntryInternal(bool skipping_saved_key,
                                       const Slice* prefix) {
J
jorlow@chromium.org 已提交
231
  // Loop until we hit an acceptable entry to yield
232
  assert(iter_.Valid());
233
  assert(status_.ok());
J
jorlow@chromium.org 已提交
234
  assert(direction_ == kForward);
235
  current_entry_is_merged_ = false;
236 237 238

  // How many times in a row we have skipped an entry with user key less than
  // or equal to saved_key_. We could skip these entries either because
239
  // sequence numbers were too high or because skipping_saved_key = true.
240
  // What saved_key_ contains throughout this method:
Y
Yanqin Jin 已提交
241 242 243 244 245 246 247 248
  //  - if skipping_saved_key : saved_key_ contains the key that we need
  //                            to skip, and we haven't seen any keys greater
  //                            than that,
  //  - if num_skipped > 0    : saved_key_ contains the key that we have skipped
  //                            num_skipped times, and we haven't seen any keys
  //                            greater than that,
  //  - none of the above     : saved_key_ can contain anything, it doesn't
  //                            matter.
249
  uint64_t num_skipped = 0;
250 251 252 253 254
  // For write unprepared, the target sequence number in reseek could be larger
  // than the snapshot, and thus needs to be skipped again. This could result in
  // an infinite loop of reseeks. To avoid that, we limit the number of reseeks
  // to one.
  bool reseek_done = false;
255

Y
Yi Wu 已提交
256 257
  is_blob_ = false;

J
jorlow@chromium.org 已提交
258
  do {
259 260 261
    // Will update is_key_seqnum_zero_ as soon as we parsed the current key
    // but we need to save the previous value to be used in the loop.
    bool is_prev_key_seqnum_zero = is_key_seqnum_zero_;
262
    if (!ParseKey(&ikey_)) {
263
      is_key_seqnum_zero_ = false;
264
      return false;
265
    }
266 267
    Slice user_key_without_ts =
        StripTimestampFromUserKey(ikey_.user_key, timestamp_size_);
268

269 270
    is_key_seqnum_zero_ = (ikey_.sequence == 0);

271 272
    assert(iterate_upper_bound_ == nullptr ||
           iter_.UpperBoundCheckResult() != IterBoundCheck::kInbound ||
Y
Yanqin Jin 已提交
273
           user_comparator_.CompareWithoutTimestamp(
274
               user_key_without_ts, /*a_has_ts=*/false, *iterate_upper_bound_,
Y
Yanqin Jin 已提交
275
               /*b_has_ts=*/false) < 0);
276 277
    if (iterate_upper_bound_ != nullptr &&
        iter_.UpperBoundCheckResult() != IterBoundCheck::kInbound &&
Y
Yanqin Jin 已提交
278
        user_comparator_.CompareWithoutTimestamp(
279
            user_key_without_ts, /*a_has_ts=*/false, *iterate_upper_bound_,
Y
Yanqin Jin 已提交
280
            /*b_has_ts=*/false) >= 0) {
281 282
      break;
    }
283

284 285
    assert(prefix == nullptr || prefix_extractor_ != nullptr);
    if (prefix != nullptr &&
286 287
        prefix_extractor_->Transform(user_key_without_ts).compare(*prefix) !=
            0) {
288
      assert(prefix_same_as_start_);
289 290 291
      break;
    }

292
    if (TooManyInternalKeysSkipped()) {
293
      return false;
294 295
    }

Y
Yanqin Jin 已提交
296
    assert(ikey_.user_key.size() >= timestamp_size_);
297 298 299
    Slice ts = timestamp_size_ > 0 ? ExtractTimestampFromUserKey(
                                         ikey_.user_key, timestamp_size_)
                                   : Slice();
300 301
    bool more_recent = false;
    if (IsVisible(ikey_.sequence, ts, &more_recent)) {
302 303 304 305
      // If the previous entry is of seqnum 0, the current entry will not
      // possibly be skipped. This condition can potentially be relaxed to
      // prev_key.seq <= ikey_.sequence. We are cautious because it will be more
      // prone to bugs causing the same user key with the same sequence number.
306 307
      // Note that with current timestamp implementation, the same user key can
      // have different timestamps and zero sequence number on the bottommost
Y
Yanqin Jin 已提交
308
      // level. This may change in the future.
309 310
      if ((!is_prev_key_seqnum_zero || timestamp_size_ > 0) &&
          skipping_saved_key &&
311
          CompareKeyForSkip(ikey_.user_key, saved_key_.GetUserKey()) <= 0) {
312 313 314
        num_skipped++;  // skip this entry
        PERF_COUNTER_ADD(internal_key_skipped_count, 1);
      } else {
315
        assert(!skipping_saved_key ||
316
               CompareKeyForSkip(ikey_.user_key, saved_key_.GetUserKey()) > 0);
317 318 319 320 321
        if (!iter_.PrepareValue()) {
          assert(!iter_.status().ok());
          valid_ = false;
          return false;
        }
322
        num_skipped = 0;
323
        reseek_done = false;
324
        switch (ikey_.type) {
325
          case kTypeDeletion:
Y
Yanqin Jin 已提交
326
          case kTypeDeletionWithTimestamp:
327 328 329
          case kTypeSingleDeletion:
            // Arrange to skip all upcoming entries for this key since
            // they are hidden by this deletion.
330 331 332
            // if iterartor specified start_seqnum we
            // 1) return internal key, including the type
            // 2) return ikey only if ikey.seqnum >= start_seqnum_
333
            // note that if deletion seqnum is < start_seqnum_ we
334
            // just skip it like in normal iterator.
335 336 337 338 339 340 341 342 343 344 345 346 347 348
            if (start_seqnum_ > 0) {
              if (ikey_.sequence >= start_seqnum_) {
                saved_key_.SetInternalKey(ikey_);
                valid_ = true;
                return true;
              } else {
                saved_key_.SetUserKey(
                    ikey_.user_key,
                    !pin_thru_lifetime_ ||
                        !iter_.iter()->IsKeyPinned() /* copy */);
                skipping_saved_key = true;
                PERF_COUNTER_ADD(internal_delete_skipped_count, 1);
              }
            } else if (timestamp_lb_) {
349
              saved_key_.SetInternalKey(ikey_);
350 351
              valid_ = true;
              return true;
352 353
            } else {
              saved_key_.SetUserKey(
354 355
                  ikey_.user_key, !pin_thru_lifetime_ ||
                                      !iter_.iter()->IsKeyPinned() /* copy */);
356
              skipping_saved_key = true;
357
              PERF_COUNTER_ADD(internal_delete_skipped_count, 1);
358
            }
359 360
            break;
          case kTypeValue:
Y
Yi Wu 已提交
361
          case kTypeBlobIndex:
362 363 364
            if (start_seqnum_ > 0) {
              if (ikey_.sequence >= start_seqnum_) {
                saved_key_.SetInternalKey(ikey_);
L
Levi Tamasi 已提交
365 366 367 368 369 370 371

                if (ikey_.type == kTypeBlobIndex) {
                  if (!SetBlobValueIfNeeded(ikey_.user_key, iter_.value())) {
                    return false;
                  }
                }

Y
Yi Wu 已提交
372
                valid_ = true;
373
                return true;
374 375
              } else {
                // this key and all previous versions shouldn't be included,
376
                // skipping_saved_key
377 378 379 380
                saved_key_.SetUserKey(
                    ikey_.user_key,
                    !pin_thru_lifetime_ ||
                        !iter_.iter()->IsKeyPinned() /* copy */);
381
                skipping_saved_key = true;
Y
Yi Wu 已提交
382
              }
383 384
            } else if (timestamp_lb_) {
              saved_key_.SetInternalKey(ikey_);
L
Levi Tamasi 已提交
385 386 387 388 389 390 391

              if (ikey_.type == kTypeBlobIndex) {
                if (!SetBlobValueIfNeeded(ikey_.user_key, iter_.value())) {
                  return false;
                }
              }

392 393
              valid_ = true;
              return true;
394
            } else {
395
              saved_key_.SetUserKey(
396 397
                  ikey_.user_key, !pin_thru_lifetime_ ||
                                      !iter_.iter()->IsKeyPinned() /* copy */);
398
              if (range_del_agg_.ShouldDelete(
399
                      ikey_, RangeDelPositioningMode::kForwardTraversal)) {
400 401
                // Arrange to skip all upcoming entries for this key since
                // they are hidden by this deletion.
402
                skipping_saved_key = true;
403
                num_skipped = 0;
404
                reseek_done = false;
405
                PERF_COUNTER_ADD(internal_delete_skipped_count, 1);
L
Levi Tamasi 已提交
406 407 408 409 410
              } else {
                if (ikey_.type == kTypeBlobIndex) {
                  if (!SetBlobValueIfNeeded(ikey_.user_key, iter_.value())) {
                    return false;
                  }
411
                }
412

413
                valid_ = true;
414
                return true;
415
              }
416 417 418
            }
            break;
          case kTypeMerge:
419
            saved_key_.SetUserKey(
420
                ikey_.user_key,
421
                !pin_thru_lifetime_ || !iter_.iter()->IsKeyPinned() /* copy */);
422
            if (range_del_agg_.ShouldDelete(
423
                    ikey_, RangeDelPositioningMode::kForwardTraversal)) {
424 425
              // Arrange to skip all upcoming entries for this key since
              // they are hidden by this deletion.
426
              skipping_saved_key = true;
427
              num_skipped = 0;
428
              reseek_done = false;
429 430 431 432 433 434
              PERF_COUNTER_ADD(internal_delete_skipped_count, 1);
            } else {
              // By now, we are sure the current ikey is going to yield a
              // value
              current_entry_is_merged_ = true;
              valid_ = true;
435
              return MergeValuesNewToOld();  // Go to a different state machine
436 437 438
            }
            break;
          default:
439 440 441 442 443
            valid_ = false;
            status_ = Status::Corruption(
                "Unknown value type: " +
                std::to_string(static_cast<unsigned int>(ikey_.type)));
            return false;
444
        }
J
jorlow@chromium.org 已提交
445
      }
446
    } else {
447 448 449
      if (more_recent) {
        PERF_COUNTER_ADD(internal_recent_skipped_count, 1);
      }
450

451 452 453
      // This key was inserted after our snapshot was taken or skipped by
      // timestamp range. If this happens too many times in a row for the same
      // user key, we want to seek to the target sequence number.
Y
Yanqin Jin 已提交
454 455
      int cmp = user_comparator_.CompareWithoutTimestamp(
          ikey_.user_key, saved_key_.GetUserKey());
Y
Yanqin Jin 已提交
456
      if (cmp == 0 || (skipping_saved_key && cmp < 0)) {
457 458
        num_skipped++;
      } else {
459
        saved_key_.SetUserKey(
460
            ikey_.user_key,
461
            !iter_.iter()->IsKeyPinned() || !pin_thru_lifetime_ /* copy */);
462
        skipping_saved_key = false;
463
        num_skipped = 0;
464
        reseek_done = false;
465
      }
J
jorlow@chromium.org 已提交
466
    }
467 468 469

    // If we have sequentially iterated via numerous equal keys, then it's
    // better to seek so that we can avoid too many key comparisons.
470 471 472 473 474
    //
    // To avoid infinite loops, do not reseek if we have already attempted to
    // reseek previously.
    //
    // TODO(lth): If we reseek to sequence number greater than ikey_.sequence,
Y
Yanqin Jin 已提交
475
    // then it does not make sense to reseek as we would actually land further
476 477
    // away from the desired key. There is opportunity for optimization here.
    if (num_skipped > max_skip_ && !reseek_done) {
478
      is_key_seqnum_zero_ = false;
479
      num_skipped = 0;
480
      reseek_done = true;
481
      std::string last_key;
482
      if (skipping_saved_key) {
483 484 485
        // We're looking for the next user-key but all we see are the same
        // user-key with decreasing sequence numbers. Fast forward to
        // sequence number 0 and type deletion (the smallest type).
Y
Yanqin Jin 已提交
486 487 488 489 490
        if (timestamp_size_ == 0) {
          AppendInternalKey(
              &last_key,
              ParsedInternalKey(saved_key_.GetUserKey(), 0, kTypeDeletion));
        } else {
491
          const std::string kTsMin(timestamp_size_, '\0');
Y
Yanqin Jin 已提交
492 493 494
          AppendInternalKeyWithDifferentTimestamp(
              &last_key,
              ParsedInternalKey(saved_key_.GetUserKey(), 0, kTypeDeletion),
495
              kTsMin);
Y
Yanqin Jin 已提交
496
        }
497 498
        // Don't set skipping_saved_key = false because we may still see more
        // user-keys equal to saved_key_.
499 500 501 502 503 504
      } else {
        // We saw multiple entries with this user key and sequence numbers
        // higher than sequence_. Fast forward to sequence_.
        // Note that this only covers a case when a higher key was overwritten
        // many times since our snapshot was taken, not the case when a lot of
        // different keys were inserted after our snapshot was taken.
Y
Yanqin Jin 已提交
505 506 507 508 509 510 511 512 513 514 515
        if (timestamp_size_ == 0) {
          AppendInternalKey(
              &last_key, ParsedInternalKey(saved_key_.GetUserKey(), sequence_,
                                           kValueTypeForSeek));
        } else {
          AppendInternalKeyWithDifferentTimestamp(
              &last_key,
              ParsedInternalKey(saved_key_.GetUserKey(), sequence_,
                                kValueTypeForSeek),
              *timestamp_ub_);
        }
516
      }
517
      iter_.Seek(last_key);
518 519
      RecordTick(statistics_, NUMBER_OF_RESEEKS_IN_ITERATION);
    } else {
520
      iter_.Next();
521
    }
522
  } while (iter_.Valid());
523

J
jorlow@chromium.org 已提交
524
  valid_ = false;
525
  return iter_.status().ok();
J
jorlow@chromium.org 已提交
526 527
}

528 529
// Merge values of the same user key starting from the current iter_ position
// Scan from the newer entries to older entries.
530
// PRE: iter_.key() points to the first merge type entry
531
//      saved_key_ stores the user key
532
//      iter_.PrepareValue() has been called
533 534
// POST: saved_value_ has the merged value for the user key
//       iter_ points to the next entry (or invalid)
535
bool DBIter::MergeValuesNewToOld() {
536
  if (!merge_operator_) {
537
    ROCKS_LOG_ERROR(logger_, "Options::merge_operator is null.");
538
    status_ = Status::InvalidArgument("merge_operator_ must be set.");
539
    valid_ = false;
540
    return false;
D
Deon Nicholas 已提交
541
  }
542

543 544
  // Temporarily pin the blocks that hold merge operands
  TempPinData();
545
  merge_context_.Clear();
546
  // Start the merge process by pushing the first operand
547 548
  merge_context_.PushOperand(
      iter_.value(), iter_.iter()->IsValuePinned() /* operand_pinned */);
549
  TEST_SYNC_POINT("DBIter::MergeValuesNewToOld:PushedFirstOperand");
550 551

  ParsedInternalKey ikey;
552
  for (iter_.Next(); iter_.Valid(); iter_.Next()) {
553
    TEST_SYNC_POINT("DBIter::MergeValuesNewToOld:SteppedToNextOperand");
554
    if (!ParseKey(&ikey)) {
555
      return false;
556 557
    }

558
    if (!user_comparator_.Equal(ikey.user_key, saved_key_.GetUserKey())) {
559 560
      // hit the next user key, stop right here
      break;
561 562
    }
    if (kTypeDeletion == ikey.type || kTypeSingleDeletion == ikey.type ||
563
               range_del_agg_.ShouldDelete(
564
                   ikey, RangeDelPositioningMode::kForwardTraversal)) {
565 566
      // hit a delete with the same user key, stop right here
      // iter_ is positioned after delete
567
      iter_.Next();
568
      break;
569 570 571 572 573 574 575
    }
    if (!iter_.PrepareValue()) {
      valid_ = false;
      return false;
    }

    if (kTypeValue == ikey.type) {
576 577
      // hit a put, merge the put value with operands and store the
      // final result in saved_value_. We are done!
578
      const Slice val = iter_.value();
579
      Status s = Merge(&val, ikey.user_key);
580
      if (!s.ok()) {
581
        return false;
582
      }
583
      // iter_ is positioned after put
584 585
      iter_.Next();
      if (!iter_.status().ok()) {
586 587 588 589
        valid_ = false;
        return false;
      }
      return true;
A
Andres Noetzli 已提交
590
    } else if (kTypeMerge == ikey.type) {
591 592
      // hit a merge, add the value as an operand and run associative merge.
      // when complete, add result to operands and continue.
593 594
      merge_context_.PushOperand(
          iter_.value(), iter_.iter()->IsValuePinned() /* operand_pinned */);
595
      PERF_COUNTER_ADD(internal_merge_count, 1);
Y
Yi Wu 已提交
596
    } else if (kTypeBlobIndex == ikey.type) {
597 598 599 600 601 602 603 604 605 606 607 608 609 610 611 612 613 614 615 616 617 618 619 620 621
      if (expose_blob_index_) {
        status_ =
            Status::NotSupported("BlobDB does not support merge operator.");
        valid_ = false;
        return false;
      }
      // hit a put, merge the put value with operands and store the
      // final result in saved_value_. We are done!
      if (!SetBlobValueIfNeeded(ikey.user_key, iter_.value())) {
        return false;
      }
      valid_ = true;
      const Slice blob_value = value();
      Status s = Merge(&blob_value, ikey.user_key);
      if (!s.ok()) {
        return false;
      }
      is_blob_ = false;
      // iter_ is positioned after put
      iter_.Next();
      if (!iter_.status().ok()) {
        valid_ = false;
        return false;
      }
      return true;
A
Andres Noetzli 已提交
622
    } else {
623 624 625 626 627
      valid_ = false;
      status_ = Status::Corruption(
          "Unrecognized value type: " +
          std::to_string(static_cast<unsigned int>(ikey.type)));
      return false;
628 629 630
    }
  }

631
  if (!iter_.status().ok()) {
632 633 634 635
    valid_ = false;
    return false;
  }

636 637 638 639
  // we either exhausted all internal keys under this user key, or hit
  // a deletion marker.
  // feed null as the existing value to the merge operator, such that
  // client can differentiate this scenario and do things accordingly.
640
  Status s = Merge(nullptr, saved_key_.GetUserKey());
641
  if (!s.ok()) {
642
    return false;
643
  }
644 645
  assert(status_.ok());
  return true;
646 647
}

J
jorlow@chromium.org 已提交
648 649
void DBIter::Prev() {
  assert(valid_);
650
  assert(status_.ok());
651

652
  PERF_CPU_TIMER_GUARD(iter_prev_cpu_nanos, clock_);
653
  ReleaseTempPinnedData();
654
  ResetInternalKeysSkippedCounter();
655
  bool ok = true;
S
Stanislau Hlebik 已提交
656
  if (direction_ == kForward) {
657 658 659 660 661
    if (!ReverseToBackward()) {
      ok = false;
    }
  }
  if (ok) {
662 663 664 665 666 667
    Slice prefix;
    if (prefix_same_as_start_) {
      assert(prefix_extractor_ != nullptr);
      prefix = prefix_.GetUserKey();
    }
    PrevInternal(prefix_same_as_start_ ? &prefix : nullptr);
S
Stanislau Hlebik 已提交
668
  }
669

M
Manuel Ung 已提交
670
  if (statistics_ != nullptr) {
671
    local_stats_.prev_count_++;
M
Manuel Ung 已提交
672
    if (valid_) {
673 674
      local_stats_.prev_found_count_++;
      local_stats_.bytes_read_ += (key().size() + value().size());
M
Manuel Ung 已提交
675 676
    }
  }
S
Stanislau Hlebik 已提交
677
}
J
jorlow@chromium.org 已提交
678

679
bool DBIter::ReverseToForward() {
680
  assert(iter_.status().ok());
681 682 683 684

  // When moving backwards, iter_ is positioned on _previous_ key, which may
  // not exist or may have different prefix than the current key().
  // If that's the case, seek iter_ to current key.
S
sdong 已提交
685
  if (!expect_total_order_inner_iter() || !iter_.Valid()) {
686
    IterKey last_key;
687 688 689 690 691 692 693 694
    ParsedInternalKey pikey(saved_key_.GetUserKey(), kMaxSequenceNumber,
                            kValueTypeForSeek);
    if (timestamp_size_ > 0) {
      // TODO: pre-create kTsMax.
      const std::string kTsMax(timestamp_size_, '\xff');
      pikey.SetTimestamp(kTsMax);
    }
    last_key.SetInternalKey(pikey);
695
    iter_.Seek(last_key.GetInternalKey());
696
    RecordTick(statistics_, NUMBER_OF_RESEEKS_IN_ITERATION);
697
  }
698

699
  direction_ = kForward;
700
  // Skip keys less than the current key() (a.k.a. saved_key_).
701
  while (iter_.Valid()) {
702 703 704 705
    ParsedInternalKey ikey;
    if (!ParseKey(&ikey)) {
      return false;
    }
706
    if (user_comparator_.Compare(ikey.user_key, saved_key_.GetUserKey()) >= 0) {
707 708
      return true;
    }
709
    iter_.Next();
710 711
  }

712
  if (!iter_.status().ok()) {
713 714
    valid_ = false;
    return false;
715
  }
716 717

  return true;
718 719
}

720 721
// Move iter_ to the key before saved_key_.
bool DBIter::ReverseToBackward() {
722
  assert(iter_.status().ok());
723 724 725 726 727

  // When current_entry_is_merged_ is true, iter_ may be positioned on the next
  // key, which may not exist or may have prefix different from current.
  // If that's the case, seek to saved_key_.
  if (current_entry_is_merged_ &&
S
sdong 已提交
728
      (!expect_total_order_inner_iter() || !iter_.Valid())) {
729
    IterKey last_key;
730 731 732 733 734
    // Using kMaxSequenceNumber and kValueTypeForSeek
    // (not kValueTypeForSeekForPrev) to seek to a key strictly smaller
    // than saved_key_.
    last_key.SetInternalKey(ParsedInternalKey(
        saved_key_.GetUserKey(), kMaxSequenceNumber, kValueTypeForSeek));
S
sdong 已提交
735
    if (!expect_total_order_inner_iter()) {
736
      iter_.SeekForPrev(last_key.GetInternalKey());
737 738 739 740 741
    } else {
      // Some iterators may not support SeekForPrev(), so we avoid using it
      // when prefix seek mode is disabled. This is somewhat expensive
      // (an extra Prev(), as well as an extra change of direction of iter_),
      // so we may need to reconsider it later.
742 743 744
      iter_.Seek(last_key.GetInternalKey());
      if (!iter_.Valid() && iter_.status().ok()) {
        iter_.SeekToLast();
745
      }
746
    }
747
    RecordTick(statistics_, NUMBER_OF_RESEEKS_IN_ITERATION);
748 749 750
  }

  direction_ = kReverse;
751
  return FindUserKeyBeforeSavedKey();
752 753
}

754
void DBIter::PrevInternal(const Slice* prefix) {
755
  while (iter_.Valid()) {
756
    saved_key_.SetUserKey(
757 758
        ExtractUserKey(iter_.key()),
        !iter_.iter()->IsKeyPinned() || !pin_thru_lifetime_ /* copy */);
759

760 761
    assert(prefix == nullptr || prefix_extractor_ != nullptr);
    if (prefix != nullptr &&
762 763 764
        prefix_extractor_
                ->Transform(StripTimestampFromUserKey(saved_key_.GetUserKey(),
                                                      timestamp_size_))
765 766
                .compare(*prefix) != 0) {
      assert(prefix_same_as_start_);
767 768 769 770 771
      // Current key does not have the same prefix as start
      valid_ = false;
      return;
    }

772
    assert(iterate_lower_bound_ == nullptr || iter_.MayBeOutOfLowerBound() ||
773 774 775
           user_comparator_.CompareWithoutTimestamp(
               saved_key_.GetUserKey(), /*a_has_ts=*/true,
               *iterate_lower_bound_, /*b_has_ts=*/false) >= 0);
776
    if (iterate_lower_bound_ != nullptr && iter_.MayBeOutOfLowerBound() &&
777 778 779
        user_comparator_.CompareWithoutTimestamp(
            saved_key_.GetUserKey(), /*a_has_ts=*/true, *iterate_lower_bound_,
            /*b_has_ts=*/false) < 0) {
780 781 782 783 784
      // We've iterated earlier than the user-specified lower bound.
      valid_ = false;
      return;
    }

785
    if (!FindValueForCurrentKey()) {  // assigns valid_
S
Stanislau Hlebik 已提交
786
      return;
J
jorlow@chromium.org 已提交
787
    }
788

789 790 791
    // Whether or not we found a value for current key, we need iter_ to end up
    // on a smaller key.
    if (!FindUserKeyBeforeSavedKey()) {
792 793 794
      return;
    }

795 796 797
    if (valid_) {
      // Found the value.
      return;
S
Stanislau Hlebik 已提交
798
    }
799 800 801

    if (TooManyInternalKeysSkipped(false)) {
      return;
S
Stanislau Hlebik 已提交
802 803
    }
  }
804

S
Stanislau Hlebik 已提交
805 806
  // We haven't found any key - iterator is not valid
  valid_ = false;
J
jorlow@chromium.org 已提交
807 808
}

809 810 811 812 813 814 815 816 817 818 819
// Used for backwards iteration.
// Looks at the entries with user key saved_key_ and finds the most up-to-date
// value for it, or executes a merge, or determines that the value was deleted.
// Sets valid_ to true if the value is found and is ready to be presented to
// the user through value().
// Sets valid_ to false if the value was deleted, and we should try another key.
// Returns false if an error occurred, and !status().ok() and !valid_.
//
// PRE: iter_ is positioned on the last entry with user key equal to saved_key_.
// POST: iter_ is positioned on one of the entries equal to saved_key_, or on
//       the entry just before them, or on the entry just after them.
S
Stanislau Hlebik 已提交
820
bool DBIter::FindValueForCurrentKey() {
821
  assert(iter_.Valid());
822
  merge_context_.Clear();
823
  current_entry_is_merged_ = false;
824 825
  // last entry before merge (could be kTypeDeletion,
  // kTypeDeletionWithTimestamp, kTypeSingleDeletion or kTypeValue)
S
Stanislau Hlebik 已提交
826 827
  ValueType last_not_merge_type = kTypeDeletion;
  ValueType last_key_entry_type = kTypeDeletion;
J
jorlow@chromium.org 已提交
828

829 830 831
  // Temporarily pin blocks that hold (merge operands / the value)
  ReleaseTempPinnedData();
  TempPinData();
S
Stanislau Hlebik 已提交
832
  size_t num_skipped = 0;
833
  while (iter_.Valid()) {
834 835 836 837 838
    ParsedInternalKey ikey;
    if (!ParseKey(&ikey)) {
      return false;
    }

Y
Yanqin Jin 已提交
839 840 841 842 843 844 845
    assert(ikey.user_key.size() >= timestamp_size_);
    Slice ts;
    if (timestamp_size_ > 0) {
      ts = Slice(ikey.user_key.data() + ikey.user_key.size() - timestamp_size_,
                 timestamp_size_);
    }
    if (!IsVisible(ikey.sequence, ts) ||
846 847
        !user_comparator_.EqualWithoutTimestamp(ikey.user_key,
                                                saved_key_.GetUserKey())) {
848 849
      break;
    }
850 851 852
    if (!ts.empty()) {
      saved_timestamp_.assign(ts.data(), ts.size());
    }
853 854 855 856
    if (TooManyInternalKeysSkipped()) {
      return false;
    }

857 858 859
    // This user key has lots of entries.
    // We're going from old to new, and it's taking too long. Let's do a Seek()
    // and go from new to old. This helps when a key was overwritten many times.
860
    if (num_skipped >= max_skip_) {
S
Stanislau Hlebik 已提交
861 862 863
      return FindValueForCurrentKeyUsingSeek();
    }

864 865 866 867 868
    if (!iter_.PrepareValue()) {
      valid_ = false;
      return false;
    }

S
Stanislau Hlebik 已提交
869 870 871
    last_key_entry_type = ikey.type;
    switch (last_key_entry_type) {
      case kTypeValue:
Y
Yi Wu 已提交
872
      case kTypeBlobIndex:
873
        if (range_del_agg_.ShouldDelete(
874
                ikey, RangeDelPositioningMode::kBackwardTraversal)) {
A
Andrew Kryczka 已提交
875 876
          last_key_entry_type = kTypeRangeDeletion;
          PERF_COUNTER_ADD(internal_delete_skipped_count, 1);
877
        } else if (iter_.iter()->IsValuePinned()) {
878
          pinned_value_ = iter_.value();
879 880 881 882 883
        } else {
          valid_ = false;
          status_ = Status::NotSupported(
              "Backward iteration not supported if underlying iterator's value "
              "cannot be pinned.");
A
Andrew Kryczka 已提交
884
        }
885
        merge_context_.Clear();
A
Andrew Kryczka 已提交
886
        last_not_merge_type = last_key_entry_type;
887 888 889
        if (!status_.ok()) {
          return false;
        }
S
Stanislau Hlebik 已提交
890 891
        break;
      case kTypeDeletion:
892
      case kTypeDeletionWithTimestamp:
A
Andres Noetzli 已提交
893
      case kTypeSingleDeletion:
894
        merge_context_.Clear();
A
Andres Noetzli 已提交
895
        last_not_merge_type = last_key_entry_type;
896
        PERF_COUNTER_ADD(internal_delete_skipped_count, 1);
S
Stanislau Hlebik 已提交
897 898
        break;
      case kTypeMerge:
899
        if (range_del_agg_.ShouldDelete(
900
                ikey, RangeDelPositioningMode::kBackwardTraversal)) {
A
Andrew Kryczka 已提交
901 902 903 904 905 906 907
          merge_context_.Clear();
          last_key_entry_type = kTypeRangeDeletion;
          last_not_merge_type = last_key_entry_type;
          PERF_COUNTER_ADD(internal_delete_skipped_count, 1);
        } else {
          assert(merge_operator_ != nullptr);
          merge_context_.PushOperandBack(
908 909
              iter_.value(),
              iter_.iter()->IsValuePinned() /* operand_pinned */);
910
          PERF_COUNTER_ADD(internal_merge_count, 1);
A
Andrew Kryczka 已提交
911
        }
S
Stanislau Hlebik 已提交
912 913
        break;
      default:
914 915 916 917 918
        valid_ = false;
        status_ = Status::Corruption(
            "Unknown value type: " +
            std::to_string(static_cast<unsigned int>(last_key_entry_type)));
        return false;
S
Stanislau Hlebik 已提交
919 920
    }

921
    PERF_COUNTER_ADD(internal_key_skipped_count, 1);
922
    iter_.Prev();
S
Stanislau Hlebik 已提交
923
    ++num_skipped;
924 925
  }

926
  if (!iter_.status().ok()) {
927 928
    valid_ = false;
    return false;
S
Stanislau Hlebik 已提交
929 930
  }

931
  Status s;
S
sdong 已提交
932
  s.PermitUncheckedError();
Y
Yi Wu 已提交
933
  is_blob_ = false;
S
Stanislau Hlebik 已提交
934 935
  switch (last_key_entry_type) {
    case kTypeDeletion:
936
    case kTypeDeletionWithTimestamp:
A
Andres Noetzli 已提交
937
    case kTypeSingleDeletion:
A
Andrew Kryczka 已提交
938
    case kTypeRangeDeletion:
S
Stanislau Hlebik 已提交
939
      valid_ = false;
940
      return true;
S
Stanislau Hlebik 已提交
941
    case kTypeMerge:
942
      current_entry_is_merged_ = true;
A
Aaron Gao 已提交
943
      if (last_not_merge_type == kTypeDeletion ||
A
Andrew Kryczka 已提交
944 945
          last_not_merge_type == kTypeSingleDeletion ||
          last_not_merge_type == kTypeRangeDeletion) {
946 947 948 949 950
        s = Merge(nullptr, saved_key_.GetUserKey());
        if (!s.ok()) {
          return false;
        }
        return true;
Y
Yi Wu 已提交
951
      } else if (last_not_merge_type == kTypeBlobIndex) {
952 953 954 955 956 957 958 959 960 961 962 963 964 965 966 967 968
        if (expose_blob_index_) {
          status_ =
              Status::NotSupported("BlobDB does not support merge operator.");
          valid_ = false;
          return false;
        }
        if (!SetBlobValueIfNeeded(saved_key_.GetUserKey(), pinned_value_)) {
          return false;
        }
        valid_ = true;
        const Slice blob_value = value();
        s = Merge(&blob_value, saved_key_.GetUserKey());
        if (!s.ok()) {
          return false;
        }
        is_blob_ = false;
        return true;
969
      } else {
S
Stanislau Hlebik 已提交
970
        assert(last_not_merge_type == kTypeValue);
971 972 973 974 975
        s = Merge(&pinned_value_, saved_key_.GetUserKey());
        if (!s.ok()) {
          return false;
        }
        return true;
976
      }
S
Stanislau Hlebik 已提交
977 978
      break;
    case kTypeValue:
979
      // do nothing - we've already has value in pinned_value_
S
Stanislau Hlebik 已提交
980
      break;
Y
Yi Wu 已提交
981
    case kTypeBlobIndex:
L
Levi Tamasi 已提交
982
      if (!SetBlobValueIfNeeded(saved_key_.GetUserKey(), pinned_value_)) {
983
        return false;
Y
Yi Wu 已提交
984 985
      }
      break;
S
Stanislau Hlebik 已提交
986
    default:
987 988 989 990 991
      valid_ = false;
      status_ = Status::Corruption(
          "Unknown value type: " +
          std::to_string(static_cast<unsigned int>(last_key_entry_type)));
      return false;
J
jorlow@chromium.org 已提交
992
  }
993
  if (!s.ok()) {
Y
Yi Wu 已提交
994
    valid_ = false;
995
    status_ = s;
996
    return false;
997
  }
998
  valid_ = true;
S
Stanislau Hlebik 已提交
999 1000
  return true;
}
J
jorlow@chromium.org 已提交
1001

S
Stanislau Hlebik 已提交
1002 1003
// This function is used in FindValueForCurrentKey.
// We use Seek() function instead of Prev() to find necessary value
1004 1005
// TODO: This is very similar to FindNextUserEntry() and MergeValuesNewToOld().
//       Would be nice to reuse some code.
S
Stanislau Hlebik 已提交
1006
bool DBIter::FindValueForCurrentKeyUsingSeek() {
1007 1008 1009
  // FindValueForCurrentKey will enable pinning before calling
  // FindValueForCurrentKeyUsingSeek()
  assert(pinned_iters_mgr_.PinningEnabled());
S
Stanislau Hlebik 已提交
1010
  std::string last_key;
1011 1012 1013 1014 1015 1016 1017 1018 1019 1020 1021
  if (0 == timestamp_size_) {
    AppendInternalKey(&last_key,
                      ParsedInternalKey(saved_key_.GetUserKey(), sequence_,
                                        kValueTypeForSeek));
  } else {
    AppendInternalKeyWithDifferentTimestamp(
        &last_key,
        ParsedInternalKey(saved_key_.GetUserKey(), sequence_,
                          kValueTypeForSeek),
        *timestamp_ub_);
  }
1022
  iter_.Seek(last_key);
S
Stanislau Hlebik 已提交
1023 1024
  RecordTick(statistics_, NUMBER_OF_RESEEKS_IN_ITERATION);

1025 1026
  // In case read_callback presents, the value we seek to may not be visible.
  // Find the next value that's visible.
S
Stanislau Hlebik 已提交
1027
  ParsedInternalKey ikey;
1028
  is_blob_ = false;
1029
  while (true) {
1030
    if (!iter_.Valid()) {
1031
      valid_ = false;
1032
      return iter_.status().ok();
1033 1034 1035 1036 1037
    }

    if (!ParseKey(&ikey)) {
      return false;
    }
Y
Yanqin Jin 已提交
1038 1039 1040 1041 1042 1043 1044
    assert(ikey.user_key.size() >= timestamp_size_);
    Slice ts;
    if (timestamp_size_ > 0) {
      ts = Slice(ikey.user_key.data() + ikey.user_key.size() - timestamp_size_,
                 timestamp_size_);
    }

1045 1046
    if (!user_comparator_.EqualWithoutTimestamp(ikey.user_key,
                                                saved_key_.GetUserKey())) {
1047 1048 1049 1050 1051 1052 1053
      // No visible values for this key, even though FindValueForCurrentKey()
      // has seen some. This is possible if we're using a tailing iterator, and
      // the entries were discarded in a compaction.
      valid_ = false;
      return true;
    }

Y
Yanqin Jin 已提交
1054
    if (IsVisible(ikey.sequence, ts)) {
1055 1056
      break;
    }
1057

1058
    iter_.Next();
1059
  }
S
Stanislau Hlebik 已提交
1060

A
Andrew Kryczka 已提交
1061
  if (ikey.type == kTypeDeletion || ikey.type == kTypeSingleDeletion ||
1062
      range_del_agg_.ShouldDelete(
1063 1064
          ikey, RangeDelPositioningMode::kBackwardTraversal) ||
      kTypeDeletionWithTimestamp == ikey.type) {
J
jorlow@chromium.org 已提交
1065
    valid_ = false;
1066
    return true;
S
Stanislau Hlebik 已提交
1067
  }
1068 1069 1070 1071
  if (!iter_.PrepareValue()) {
    valid_ = false;
    return false;
  }
1072 1073 1074 1075
  if (timestamp_size_ > 0) {
    Slice ts = ExtractTimestampFromUserKey(ikey.user_key, timestamp_size_);
    saved_timestamp_.assign(ts.data(), ts.size());
  }
Y
Yi Wu 已提交
1076
  if (ikey.type == kTypeValue || ikey.type == kTypeBlobIndex) {
1077 1078
    assert(iter_.iter()->IsValuePinned());
    pinned_value_ = iter_.value();
L
Levi Tamasi 已提交
1079 1080 1081 1082 1083 1084
    if (ikey.type == kTypeBlobIndex) {
      if (!SetBlobValueIfNeeded(ikey.user_key, pinned_value_)) {
        return false;
      }
    }

A
Andrew Kryczka 已提交
1085 1086 1087
    valid_ = true;
    return true;
  }
S
Stanislau Hlebik 已提交
1088 1089 1090

  // kTypeMerge. We need to collect all kTypeMerge values and save them
  // in operands
1091
  assert(ikey.type == kTypeMerge);
1092
  current_entry_is_merged_ = true;
1093
  merge_context_.Clear();
1094 1095
  merge_context_.PushOperand(
      iter_.value(), iter_.iter()->IsValuePinned() /* operand_pinned */);
1096
  while (true) {
1097
    iter_.Next();
S
Stanislau Hlebik 已提交
1098

1099 1100
    if (!iter_.Valid()) {
      if (!iter_.status().ok()) {
1101 1102 1103 1104
        valid_ = false;
        return false;
      }
      break;
S
Stanislau Hlebik 已提交
1105
    }
1106 1107 1108
    if (!ParseKey(&ikey)) {
      return false;
    }
1109
    if (!user_comparator_.Equal(ikey.user_key, saved_key_.GetUserKey())) {
1110 1111 1112 1113
      break;
    }
    if (ikey.type == kTypeDeletion || ikey.type == kTypeSingleDeletion ||
        range_del_agg_.ShouldDelete(
1114
            ikey, RangeDelPositioningMode::kForwardTraversal)) {
1115
      break;
1116 1117 1118 1119 1120 1121 1122
    }
    if (!iter_.PrepareValue()) {
      valid_ = false;
      return false;
    }

    if (ikey.type == kTypeValue) {
1123
      const Slice val = iter_.value();
1124
      Status s = Merge(&val, saved_key_.GetUserKey());
1125 1126 1127 1128 1129
      if (!s.ok()) {
        return false;
      }
      return true;
    } else if (ikey.type == kTypeMerge) {
1130 1131
      merge_context_.PushOperand(
          iter_.value(), iter_.iter()->IsValuePinned() /* operand_pinned */);
1132 1133
      PERF_COUNTER_ADD(internal_merge_count, 1);
    } else if (ikey.type == kTypeBlobIndex) {
1134 1135 1136 1137 1138 1139 1140 1141 1142 1143 1144 1145 1146 1147 1148 1149 1150
      if (expose_blob_index_) {
        status_ =
            Status::NotSupported("BlobDB does not support merge operator.");
        valid_ = false;
        return false;
      }
      if (!SetBlobValueIfNeeded(ikey.user_key, iter_.value())) {
        return false;
      }
      valid_ = true;
      const Slice blob_value = value();
      Status s = Merge(&blob_value, saved_key_.GetUserKey());
      if (!s.ok()) {
        return false;
      }
      is_blob_ = false;
      return true;
1151
    } else {
1152 1153 1154 1155 1156
      valid_ = false;
      status_ = Status::Corruption(
          "Unknown value type: " +
          std::to_string(static_cast<unsigned int>(ikey.type)));
      return false;
1157
    }
S
Stanislau Hlebik 已提交
1158 1159
  }

1160
  Status s = Merge(nullptr, saved_key_.GetUserKey());
1161 1162
  if (!s.ok()) {
    return false;
1163
  }
S
Stanislau Hlebik 已提交
1164

1165 1166 1167
  // Make sure we leave iter_ in a good state. If it's valid and we don't care
  // about prefixes, that's already good enough. Otherwise it needs to be
  // seeked to the current key.
S
sdong 已提交
1168 1169
  if (!expect_total_order_inner_iter() || !iter_.Valid()) {
    if (!expect_total_order_inner_iter()) {
1170
      iter_.SeekForPrev(last_key);
1171
    } else {
1172 1173 1174
      iter_.Seek(last_key);
      if (!iter_.Valid() && iter_.status().ok()) {
        iter_.SeekToLast();
1175 1176 1177
      }
    }
    RecordTick(statistics_, NUMBER_OF_RESEEKS_IN_ITERATION);
S
Stanislau Hlebik 已提交
1178
  }
1179 1180 1181

  valid_ = true;
  return true;
S
Stanislau Hlebik 已提交
1182 1183
}

1184 1185 1186 1187 1188 1189 1190 1191 1192 1193 1194 1195 1196
Status DBIter::Merge(const Slice* val, const Slice& user_key) {
  Status s = MergeHelper::TimedFullMerge(
      merge_operator_, user_key, val, merge_context_.GetOperands(),
      &saved_value_, logger_, statistics_, clock_, &pinned_value_, true);
  if (!s.ok()) {
    valid_ = false;
    status_ = s;
    return s;
  }
  valid_ = true;
  return s;
}

1197 1198 1199 1200
// Move backwards until the key smaller than saved_key_.
// Changes valid_ only if return value is false.
bool DBIter::FindUserKeyBeforeSavedKey() {
  assert(status_.ok());
S
Stanislau Hlebik 已提交
1201
  size_t num_skipped = 0;
1202
  while (iter_.Valid()) {
1203 1204 1205
    ParsedInternalKey ikey;
    if (!ParseKey(&ikey)) {
      return false;
1206 1207
    }

1208 1209
    if (user_comparator_.CompareWithoutTimestamp(ikey.user_key,
                                                 saved_key_.GetUserKey()) < 0) {
1210 1211 1212 1213 1214
      return true;
    }

    if (TooManyInternalKeysSkipped()) {
      return false;
S
Stanislau Hlebik 已提交
1215
    }
1216

S
Siying Dong 已提交
1217
    assert(ikey.sequence != kMaxSequenceNumber);
Y
Yanqin Jin 已提交
1218 1219 1220 1221 1222 1223 1224
    assert(ikey.user_key.size() >= timestamp_size_);
    Slice ts;
    if (timestamp_size_ > 0) {
      ts = Slice(ikey.user_key.data() + ikey.user_key.size() - timestamp_size_,
                 timestamp_size_);
    }
    if (!IsVisible(ikey.sequence, ts)) {
1225 1226 1227 1228
      PERF_COUNTER_ADD(internal_recent_skipped_count, 1);
    } else {
      PERF_COUNTER_ADD(internal_key_skipped_count, 1);
    }
1229

1230
    if (num_skipped >= max_skip_) {
1231 1232
      num_skipped = 0;
      IterKey last_key;
1233 1234 1235 1236 1237 1238 1239 1240
      ParsedInternalKey pikey(saved_key_.GetUserKey(), kMaxSequenceNumber,
                              kValueTypeForSeek);
      if (timestamp_size_ > 0) {
        // TODO: pre-create kTsMax.
        const std::string kTsMax(timestamp_size_, '\xff');
        pikey.SetTimestamp(kTsMax);
      }
      last_key.SetInternalKey(pikey);
1241 1242
      // It would be more efficient to use SeekForPrev() here, but some
      // iterators may not support it.
1243
      iter_.Seek(last_key.GetInternalKey());
1244
      RecordTick(statistics_, NUMBER_OF_RESEEKS_IN_ITERATION);
1245
      if (!iter_.Valid()) {
1246 1247 1248 1249 1250 1251
        break;
      }
    } else {
      ++num_skipped;
    }

1252
    iter_.Prev();
S
Stanislau Hlebik 已提交
1253
  }
1254

1255
  if (!iter_.status().ok()) {
1256 1257 1258 1259 1260
    valid_ = false;
    return false;
  }

  return true;
S
Stanislau Hlebik 已提交
1261 1262
}

1263 1264 1265 1266 1267 1268 1269 1270 1271 1272 1273 1274
bool DBIter::TooManyInternalKeysSkipped(bool increment) {
  if ((max_skippable_internal_keys_ > 0) &&
      (num_internal_keys_skipped_ > max_skippable_internal_keys_)) {
    valid_ = false;
    status_ = Status::Incomplete("Too many internal keys skipped.");
    return true;
  } else if (increment) {
    num_internal_keys_skipped_++;
  }
  return false;
}

1275 1276
bool DBIter::IsVisible(SequenceNumber sequence, const Slice& ts,
                       bool* more_recent) {
Y
Yanqin Jin 已提交
1277
  // Remember that comparator orders preceding timestamp as larger.
1278 1279 1280 1281 1282 1283 1284 1285 1286 1287 1288 1289 1290
  // TODO(yanqin): support timestamp in read_callback_.
  bool visible_by_seq = (read_callback_ == nullptr)
                            ? sequence <= sequence_
                            : read_callback_->IsVisible(sequence);

  bool visible_by_ts =
      (timestamp_ub_ == nullptr ||
       user_comparator_.CompareTimestamp(ts, *timestamp_ub_) <= 0) &&
      (timestamp_lb_ == nullptr ||
       user_comparator_.CompareTimestamp(ts, *timestamp_lb_) >= 0);

  if (more_recent) {
    *more_recent = !visible_by_seq;
1291
  }
1292
  return visible_by_seq && visible_by_ts;
1293
}
1294

1295
void DBIter::SetSavedKeyToSeekTarget(const Slice& target) {
1296
  is_key_seqnum_zero_ = false;
1297
  SequenceNumber seq = sequence_;
1298
  saved_key_.Clear();
Y
Yanqin Jin 已提交
1299
  saved_key_.SetInternalKey(target, seq, kValueTypeForSeek, timestamp_ub_);
1300

Z
zhangjinpeng1987 已提交
1301
  if (iterate_lower_bound_ != nullptr &&
Y
Yanqin Jin 已提交
1302 1303 1304
      user_comparator_.CompareWithoutTimestamp(
          saved_key_.GetUserKey(), /*a_has_ts=*/true, *iterate_lower_bound_,
          /*b_has_ts=*/false) < 0) {
1305
    // Seek key is smaller than the lower bound.
Z
zhangjinpeng1987 已提交
1306
    saved_key_.Clear();
Y
Yanqin Jin 已提交
1307 1308
    saved_key_.SetInternalKey(*iterate_lower_bound_, seq, kValueTypeForSeek,
                              timestamp_ub_);
Z
zhangjinpeng1987 已提交
1309
  }
1310 1311 1312 1313 1314 1315 1316
}

void DBIter::SetSavedKeyToSeekForPrevTarget(const Slice& target) {
  is_key_seqnum_zero_ = false;
  saved_key_.Clear();
  // now saved_key is used to store internal key.
  saved_key_.SetInternalKey(target, 0 /* sequence_number */,
1317 1318 1319 1320 1321 1322 1323
                            kValueTypeForSeekForPrev, timestamp_ub_);

  if (timestamp_size_ > 0) {
    const std::string kTsMin(timestamp_size_, '\0');
    Slice ts = kTsMin;
    saved_key_.UpdateInternalKey(/*seq=*/0, kValueTypeForSeekForPrev, &ts);
  }
1324 1325

  if (iterate_upper_bound_ != nullptr &&
1326 1327 1328
      user_comparator_.CompareWithoutTimestamp(
          saved_key_.GetUserKey(), /*a_has_ts=*/true, *iterate_upper_bound_,
          /*b_has_ts=*/false) >= 0) {
1329
    saved_key_.Clear();
1330 1331 1332 1333 1334 1335 1336 1337
    saved_key_.SetInternalKey(*iterate_upper_bound_, kMaxSequenceNumber,
                              kValueTypeForSeekForPrev, timestamp_ub_);
    if (timestamp_size_ > 0) {
      const std::string kTsMax(timestamp_size_, '\xff');
      Slice ts = kTsMax;
      saved_key_.UpdateInternalKey(kMaxSequenceNumber, kValueTypeForSeekForPrev,
                                   &ts);
    }
1338 1339 1340 1341
  }
}

void DBIter::Seek(const Slice& target) {
1342 1343
  PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, clock_);
  StopWatch sw(clock_, statistics_, DB_SEEK);
1344 1345 1346

#ifndef ROCKSDB_LITE
  if (db_impl_ != nullptr && cfd_ != nullptr) {
1347
    // TODO: What do we do if this returns an error?
1348 1349 1350 1351 1352 1353 1354 1355 1356 1357 1358 1359 1360
    Slice lower_bound, upper_bound;
    if (iterate_lower_bound_ != nullptr) {
      lower_bound = *iterate_lower_bound_;
    } else {
      lower_bound = Slice("");
    }
    if (iterate_upper_bound_ != nullptr) {
      upper_bound = *iterate_upper_bound_;
    } else {
      upper_bound = Slice("");
    }
    db_impl_->TraceIteratorSeek(cfd_->GetID(), target, lower_bound, upper_bound)
        .PermitUncheckedError();
1361 1362 1363 1364 1365 1366
  }
#endif  // ROCKSDB_LITE

  status_ = Status::OK();
  ReleaseTempPinnedData();
  ResetInternalKeysSkippedCounter();
Z
zhangjinpeng1987 已提交
1367

1368
  // Seek the inner iterator based on the target key.
1369 1370
  {
    PERF_TIMER_GUARD(seek_internal_seek_time);
1371 1372

    SetSavedKeyToSeekTarget(target);
1373
    iter_.Seek(saved_key_.GetInternalKey());
1374

1375
    range_del_agg_.InvalidateRangeDelMapPositions();
1376
    RecordTick(statistics_, NUMBER_DB_SEEK);
1377
  }
1378 1379 1380 1381 1382 1383 1384 1385 1386 1387
  if (!iter_.Valid()) {
    valid_ = false;
    return;
  }
  direction_ = kForward;

  // Now the inner iterator is placed to the target position. From there,
  // we need to find out the next key that is visible to the user.
  ClearSavedValue();
  if (prefix_same_as_start_) {
1388
    // The case where the iterator needs to be invalidated if it has exhausted
1389 1390
    // keys within the same prefix of the seek key.
    assert(prefix_extractor_ != nullptr);
Y
Yanqin Jin 已提交
1391
    Slice target_prefix = prefix_extractor_->Transform(target);
1392 1393 1394
    FindNextUserEntry(false /* not skipping saved_key */,
                      &target_prefix /* prefix */);
    if (valid_) {
Y
Yanqin Jin 已提交
1395
      // Remember the prefix of the seek key for the future Next() call to
1396 1397
      // check.
      prefix_.SetUserKey(target_prefix);
M
Manuel Ung 已提交
1398
    }
J
jorlow@chromium.org 已提交
1399
  } else {
1400 1401 1402 1403
    FindNextUserEntry(false /* not skipping saved_key */, nullptr);
  }
  if (!valid_) {
    return;
J
jorlow@chromium.org 已提交
1404
  }
1405

1406 1407 1408 1409 1410
  // Updating stats and perf context counters.
  if (statistics_ != nullptr) {
    // Decrement since we don't want to count this key as skipped
    RecordTick(statistics_, NUMBER_DB_SEEK_FOUND);
    RecordTick(statistics_, ITER_BYTES_READ, key().size() + value().size());
1411
  }
1412
  PERF_COUNTER_ADD(iter_read_bytes, key().size() + value().size());
J
jorlow@chromium.org 已提交
1413
}
J
jorlow@chromium.org 已提交
1414

A
Aaron Gao 已提交
1415
void DBIter::SeekForPrev(const Slice& target) {
1416 1417
  PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, clock_);
  StopWatch sw(clock_, statistics_, DB_SEEK);
1418 1419 1420

#ifndef ROCKSDB_LITE
  if (db_impl_ != nullptr && cfd_ != nullptr) {
1421
    // TODO: What do we do if this returns an error?
1422 1423 1424 1425 1426 1427 1428 1429 1430 1431 1432 1433 1434 1435
    Slice lower_bound, upper_bound;
    if (iterate_lower_bound_ != nullptr) {
      lower_bound = *iterate_lower_bound_;
    } else {
      lower_bound = Slice("");
    }
    if (iterate_upper_bound_ != nullptr) {
      upper_bound = *iterate_upper_bound_;
    } else {
      upper_bound = Slice("");
    }
    db_impl_
        ->TraceIteratorSeekForPrev(cfd_->GetID(), target, lower_bound,
                                   upper_bound)
1436
        .PermitUncheckedError();
1437 1438 1439
  }
#endif  // ROCKSDB_LITE

1440
  status_ = Status::OK();
A
Aaron Gao 已提交
1441
  ReleaseTempPinnedData();
1442
  ResetInternalKeysSkippedCounter();
Z
zhangjinpeng1987 已提交
1443

1444
  // Seek the inner iterator based on the target key.
A
Aaron Gao 已提交
1445 1446
  {
    PERF_TIMER_GUARD(seek_internal_seek_time);
1447
    SetSavedKeyToSeekForPrevTarget(target);
1448
    iter_.SeekForPrev(saved_key_.GetInternalKey());
1449
    range_del_agg_.InvalidateRangeDelMapPositions();
1450
    RecordTick(statistics_, NUMBER_DB_SEEK);
A
Aaron Gao 已提交
1451
  }
1452 1453 1454
  if (!iter_.Valid()) {
    valid_ = false;
    return;
1455
  }
1456
  direction_ = kReverse;
1457

1458 1459 1460 1461 1462
  // Now the inner iterator is placed to the target position. From there,
  // we need to find out the first key that is visible to the user in the
  // backward direction.
  ClearSavedValue();
  if (prefix_same_as_start_) {
1463
    // The case where the iterator needs to be invalidated if it has exhausted
1464 1465
    // keys within the same prefix of the seek key.
    assert(prefix_extractor_ != nullptr);
Y
Yanqin Jin 已提交
1466
    Slice target_prefix = prefix_extractor_->Transform(target);
1467 1468 1469 1470 1471
    PrevInternal(&target_prefix);
    if (valid_) {
      // Remember the prefix of the seek key for the future Prev() call to
      // check.
      prefix_.SetUserKey(target_prefix);
A
Aaron Gao 已提交
1472 1473
    }
  } else {
1474
    PrevInternal(nullptr);
A
Aaron Gao 已提交
1475
  }
1476 1477 1478 1479 1480 1481

  // Report stats and perf context.
  if (statistics_ != nullptr && valid_) {
    RecordTick(statistics_, NUMBER_DB_SEEK_FOUND);
    RecordTick(statistics_, ITER_BYTES_READ, key().size() + value().size());
    PERF_COUNTER_ADD(iter_read_bytes, key().size() + value().size());
A
Aaron Gao 已提交
1482 1483 1484
  }
}

J
jorlow@chromium.org 已提交
1485
void DBIter::SeekToFirst() {
1486 1487 1488 1489
  if (iterate_lower_bound_ != nullptr) {
    Seek(*iterate_lower_bound_);
    return;
  }
1490
  PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, clock_);
1491 1492
  // Don't use iter_::Seek() if we set a prefix extractor
  // because prefix seek will be used.
S
sdong 已提交
1493
  if (!expect_total_order_inner_iter()) {
1494 1495 1496
    max_skip_ = std::numeric_limits<uint64_t>::max();
  }
  status_ = Status::OK();
J
jorlow@chromium.org 已提交
1497
  direction_ = kForward;
1498
  ReleaseTempPinnedData();
1499
  ResetInternalKeysSkippedCounter();
J
jorlow@chromium.org 已提交
1500
  ClearSavedValue();
1501
  is_key_seqnum_zero_ = false;
1502 1503 1504

  {
    PERF_TIMER_GUARD(seek_internal_seek_time);
1505
    iter_.SeekToFirst();
1506
    range_del_agg_.InvalidateRangeDelMapPositions();
1507 1508
  }

M
Manuel Ung 已提交
1509
  RecordTick(statistics_, NUMBER_DB_SEEK);
1510
  if (iter_.Valid()) {
1511
    saved_key_.SetUserKey(
1512 1513
        ExtractUserKey(iter_.key()),
        !iter_.iter()->IsKeyPinned() || !pin_thru_lifetime_ /* copy */);
1514 1515
    FindNextUserEntry(false /* not skipping saved_key */,
                      nullptr /* no prefix check */);
M
Manuel Ung 已提交
1516 1517 1518 1519
    if (statistics_ != nullptr) {
      if (valid_) {
        RecordTick(statistics_, NUMBER_DB_SEEK_FOUND);
        RecordTick(statistics_, ITER_BYTES_READ, key().size() + value().size());
1520
        PERF_COUNTER_ADD(iter_read_bytes, key().size() + value().size());
M
Manuel Ung 已提交
1521 1522
      }
    }
J
jorlow@chromium.org 已提交
1523 1524
  } else {
    valid_ = false;
J
jorlow@chromium.org 已提交
1525
  }
1526 1527
  if (valid_ && prefix_same_as_start_) {
    assert(prefix_extractor_ != nullptr);
1528 1529
    prefix_.SetUserKey(prefix_extractor_->Transform(
        StripTimestampFromUserKey(saved_key_.GetUserKey(), timestamp_size_)));
1530
  }
J
jorlow@chromium.org 已提交
1531 1532
}

J
jorlow@chromium.org 已提交
1533
void DBIter::SeekToLast() {
1534 1535 1536
  if (iterate_upper_bound_ != nullptr) {
    // Seek to last key strictly less than ReadOptions.iterate_upper_bound.
    SeekForPrev(*iterate_upper_bound_);
1537 1538 1539
    if (Valid() && 0 == user_comparator_.CompareWithoutTimestamp(
                            *iterate_upper_bound_, /*a_has_ts=*/false, key(),
                            /*b_has_ts=*/false)) {
1540
      ReleaseTempPinnedData();
1541
      PrevInternal(nullptr);
1542 1543 1544 1545
    }
    return;
  }

1546
  PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, clock_);
S
Stanislau Hlebik 已提交
1547
  // Don't use iter_::Seek() if we set a prefix extractor
1548
  // because prefix seek will be used.
S
sdong 已提交
1549
  if (!expect_total_order_inner_iter()) {
S
Stanislau Hlebik 已提交
1550 1551
    max_skip_ = std::numeric_limits<uint64_t>::max();
  }
1552
  status_ = Status::OK();
J
jorlow@chromium.org 已提交
1553
  direction_ = kReverse;
1554
  ReleaseTempPinnedData();
1555
  ResetInternalKeysSkippedCounter();
J
jorlow@chromium.org 已提交
1556
  ClearSavedValue();
1557
  is_key_seqnum_zero_ = false;
1558 1559 1560

  {
    PERF_TIMER_GUARD(seek_internal_seek_time);
1561
    iter_.SeekToLast();
1562
    range_del_agg_.InvalidateRangeDelMapPositions();
1563
  }
1564
  PrevInternal(nullptr);
M
Manuel Ung 已提交
1565 1566 1567 1568 1569
  if (statistics_ != nullptr) {
    RecordTick(statistics_, NUMBER_DB_SEEK);
    if (valid_) {
      RecordTick(statistics_, NUMBER_DB_SEEK_FOUND);
      RecordTick(statistics_, ITER_BYTES_READ, key().size() + value().size());
1570
      PERF_COUNTER_ADD(iter_read_bytes, key().size() + value().size());
M
Manuel Ung 已提交
1571 1572
    }
  }
1573 1574
  if (valid_ && prefix_same_as_start_) {
    assert(prefix_extractor_ != nullptr);
1575 1576
    prefix_.SetUserKey(prefix_extractor_->Transform(
        StripTimestampFromUserKey(saved_key_.GetUserKey(), timestamp_size_)));
1577
  }
J
jorlow@chromium.org 已提交
1578 1579
}

1580
Iterator* NewDBIterator(Env* env, const ReadOptions& read_options,
1581
                        const ImmutableOptions& cf_options,
1582
                        const MutableCFOptions& mutable_cf_options,
1583
                        const Comparator* user_key_comparator,
L
Levi Tamasi 已提交
1584
                        InternalIterator* internal_iter, const Version* version,
1585
                        const SequenceNumber& sequence,
Y
Yi Wu 已提交
1586
                        uint64_t max_sequential_skip_in_iterations,
1587
                        ReadCallback* read_callback, DBImpl* db_impl,
L
Levi Tamasi 已提交
1588 1589 1590 1591 1592 1593
                        ColumnFamilyData* cfd, bool expose_blob_index) {
  DBIter* db_iter =
      new DBIter(env, read_options, cf_options, mutable_cf_options,
                 user_key_comparator, internal_iter, version, sequence, false,
                 max_sequential_skip_in_iterations, read_callback, db_impl, cfd,
                 expose_blob_index);
1594
  return db_iter;
1595 1596
}

1597
}  // namespace ROCKSDB_NAMESPACE