backupable_db.cc 46.7 KB
Newer Older
I
Igor Canadi 已提交
1 2 3 4 5 6 7 8 9
//  Copyright (c) 2013, Facebook, Inc.  All rights reserved.
//  This source code is licensed under the BSD-style license found in the
//  LICENSE file in the root directory of this source tree. An additional grant
//  of patent rights can be found in the PATENTS file in the same directory.
//
// 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.

I
Igor Canadi 已提交
10 11
#ifndef ROCKSDB_LITE

12
#include "rocksdb/utilities/backupable_db.h"
I
Igor Canadi 已提交
13 14
#include "db/filename.h"
#include "util/coding.h"
L
Lei Jin 已提交
15
#include "util/crc32c.h"
I
Igor Canadi 已提交
16 17
#include "rocksdb/transaction_log.h"

L
liuhuahang 已提交
18
#ifndef __STDC_FORMAT_MACROS
I
Igor Canadi 已提交
19
#define __STDC_FORMAT_MACROS
L
liuhuahang 已提交
20
#endif
I
Igor Canadi 已提交
21 22

#include <inttypes.h>
23
#include <stdlib.h>
I
Igor Canadi 已提交
24 25 26 27 28
#include <algorithm>
#include <vector>
#include <map>
#include <string>
#include <limits>
I
Igor Canadi 已提交
29
#include <atomic>
30
#include <unordered_map>
I
Igor Canadi 已提交
31 32 33

namespace rocksdb {

I
Igor Canadi 已提交
34
namespace {
L
Lei Jin 已提交
35
class BackupRateLimiter {
I
Igor Canadi 已提交
36
 public:
L
Lei Jin 已提交
37 38
  BackupRateLimiter(Env* env, uint64_t max_bytes_per_second,
                   uint64_t bytes_per_check)
I
Igor Canadi 已提交
39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57
      : env_(env),
        max_bytes_per_second_(max_bytes_per_second),
        bytes_per_check_(bytes_per_check),
        micros_start_time_(env->NowMicros()),
        bytes_since_start_(0) {}

  void ReportAndWait(uint64_t bytes_since_last_call) {
    bytes_since_start_ += bytes_since_last_call;
    if (bytes_since_start_ < bytes_per_check_) {
      // not enough bytes to be rate-limited
      return;
    }

    uint64_t now = env_->NowMicros();
    uint64_t interval = now - micros_start_time_;
    uint64_t should_take_micros =
        (bytes_since_start_ * kMicrosInSecond) / max_bytes_per_second_;

    if (should_take_micros > interval) {
58 59
      env_->SleepForMicroseconds(
          static_cast<int>(should_take_micros - interval));
I
Igor Canadi 已提交
60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76
      now = env_->NowMicros();
    }
    // reset interval
    micros_start_time_ = now;
    bytes_since_start_ = 0;
  }

 private:
  Env* env_;
  uint64_t max_bytes_per_second_;
  uint64_t bytes_per_check_;
  uint64_t micros_start_time_;
  uint64_t bytes_since_start_;
  static const uint64_t kMicrosInSecond = 1000 * 1000LL;
};
}  // namespace

77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97
void BackupStatistics::IncrementNumberSuccessBackup() {
  number_success_backup++;
}
void BackupStatistics::IncrementNumberFailBackup() {
  number_fail_backup++;
}

uint32_t BackupStatistics::GetNumberSuccessBackup() const {
  return number_success_backup;
}
uint32_t BackupStatistics::GetNumberFailBackup() const {
  return number_fail_backup;
}

std::string BackupStatistics::ToString() const {
  char result[50];
  snprintf(result, sizeof(result), "# success backup: %u, # fail backup: %u",
           GetNumberSuccessBackup(), GetNumberFailBackup());
  return result;
}

I
Igor Canadi 已提交
98
void BackupableDBOptions::Dump(Logger* logger) const {
I
Igor Canadi 已提交
99 100 101
  Log(logger, "        Options.backup_dir: %s", backup_dir.c_str());
  Log(logger, "        Options.backup_env: %p", backup_env);
  Log(logger, " Options.share_table_files: %d",
I
Igor Canadi 已提交
102
      static_cast<int>(share_table_files));
I
Igor Canadi 已提交
103 104 105
  Log(logger, "          Options.info_log: %p", info_log);
  Log(logger, "              Options.sync: %d", static_cast<int>(sync));
  Log(logger, "  Options.destroy_old_data: %d",
I
Igor Canadi 已提交
106
      static_cast<int>(destroy_old_data));
I
Igor Canadi 已提交
107
  Log(logger, "  Options.backup_log_files: %d",
108
      static_cast<int>(backup_log_files));
I
Igor Canadi 已提交
109 110
  Log(logger, " Options.backup_rate_limit: %" PRIu64, backup_rate_limit);
  Log(logger, "Options.restore_rate_limit: %" PRIu64, restore_rate_limit);
I
Igor Canadi 已提交
111 112
}

I
Igor Canadi 已提交
113 114
// -------- BackupEngineImpl class ---------
class BackupEngineImpl : public BackupEngine {
I
Igor Canadi 已提交
115
 public:
I
Igor Canadi 已提交
116 117
  BackupEngineImpl(Env* db_env, const BackupableDBOptions& options,
                   bool read_only = false);
I
Igor Canadi 已提交
118
  ~BackupEngineImpl();
I
Igor Canadi 已提交
119 120 121
  Status CreateNewBackup(DB* db, bool flush_before_backup = false);
  Status PurgeOldBackups(uint32_t num_backups_to_keep);
  Status DeleteBackup(BackupID backup_id);
I
Igor Canadi 已提交
122 123 124
  void StopBackup() {
    stop_backup_.store(true, std::memory_order_release);
  }
H
Hasnain Lakhani 已提交
125
  Status GarbageCollect();
I
Igor Canadi 已提交
126 127

  void GetBackupInfo(std::vector<BackupInfo>* backup_info);
H
Hasnain Lakhani 已提交
128
  void GetCorruptedBackups(std::vector<BackupID>* corrupt_backup_ids);
129 130 131 132 133 134 135 136 137 138
  Status RestoreDBFromBackup(BackupID backup_id, const std::string& db_dir,
                             const std::string& wal_dir,
                             const RestoreOptions& restore_options =
                                 RestoreOptions());
  Status RestoreDBFromLatestBackup(const std::string& db_dir,
                                   const std::string& wal_dir,
                                   const RestoreOptions& restore_options =
                                       RestoreOptions()) {
    return RestoreDBFromBackup(latest_backup_id_, db_dir, wal_dir,
                               restore_options);
I
Igor Canadi 已提交
139 140 141
  }

 private:
142 143
  void DeleteChildren(const std::string& dir, uint32_t file_type_filter = 0);

L
Lei Jin 已提交
144 145 146 147
  struct FileInfo {
    FileInfo(const std::string& fname, uint64_t sz, uint32_t checksum)
      : refs(0), filename(fname), size(sz), checksum_value(checksum) {}

148 149 150
    FileInfo(const FileInfo&) = delete;
    FileInfo& operator=(const FileInfo&) = delete;

L
Lei Jin 已提交
151 152 153
    int refs;
    const std::string filename;
    const uint64_t size;
154
    const uint32_t checksum_value;
L
Lei Jin 已提交
155 156
  };

I
Igor Canadi 已提交
157 158 159
  class BackupMeta {
   public:
    BackupMeta(const std::string& meta_filename,
160 161
        std::unordered_map<std::string, std::shared_ptr<FileInfo>>* file_infos,
        Env* env)
I
Igor Canadi 已提交
162
      : timestamp_(0), size_(0), meta_filename_(meta_filename),
L
Lei Jin 已提交
163
        file_infos_(file_infos), env_(env) {}
I
Igor Canadi 已提交
164

165 166 167
    BackupMeta(const BackupMeta&) = delete;
    BackupMeta& operator=(const BackupMeta&) = delete;

I
Igor Canadi 已提交
168 169 170 171 172 173 174 175 176 177 178
    ~BackupMeta() {}

    void RecordTimestamp() {
      env_->GetCurrentTime(&timestamp_);
    }
    int64_t GetTimestamp() const {
      return timestamp_;
    }
    uint64_t GetSize() const {
      return size_;
    }
179
    uint32_t GetNumberFiles() { return static_cast<uint32_t>(files_.size()); }
180 181 182 183 184 185
    void SetSequenceNumber(uint64_t sequence_number) {
      sequence_number_ = sequence_number;
    }
    uint64_t GetSequenceNumber() {
      return sequence_number_;
    }
I
Igor Canadi 已提交
186

187
    Status AddFile(std::shared_ptr<FileInfo> file_info);
L
Lei Jin 已提交
188

I
Igor Canadi 已提交
189
    void Delete(bool delete_meta = true);
I
Igor Canadi 已提交
190 191 192 193 194

    bool Empty() {
      return files_.empty();
    }

195
    std::shared_ptr<FileInfo> GetFile(const std::string& filename) const {
196 197 198
      auto it = file_infos_->find(filename);
      if (it == file_infos_->end())
        return nullptr;
199
      return it->second;
200 201
    }

202
    const std::vector<std::shared_ptr<FileInfo>>& GetFiles() {
I
Igor Canadi 已提交
203 204 205 206 207 208 209 210
      return files_;
    }

    Status LoadFromFile(const std::string& backup_dir);
    Status StoreToFile(bool sync);

   private:
    int64_t timestamp_;
211 212 213
    // sequence number is only approximate, should not be used
    // by clients
    uint64_t sequence_number_;
I
Igor Canadi 已提交
214 215 216
    uint64_t size_;
    std::string const meta_filename_;
    // files with relative paths (without "/" prefix!!)
217 218
    std::vector<std::shared_ptr<FileInfo>> files_;
    std::unordered_map<std::string, std::shared_ptr<FileInfo>>* file_infos_;
I
Igor Canadi 已提交
219
    Env* env_;
220

221 222
    static const size_t max_backup_meta_file_size_ = 10 * 1024 * 1024;  // 10MB
  };  // BackupMeta
I
Igor Canadi 已提交
223 224 225 226 227 228 229 230 231

  inline std::string GetAbsolutePath(
      const std::string &relative_path = "") const {
    assert(relative_path.size() == 0 || relative_path[0] != '/');
    return options_.backup_dir + "/" + relative_path;
  }
  inline std::string GetPrivateDirRel() const {
    return "private";
  }
232 233 234
  inline std::string GetSharedChecksumDirRel() const {
    return "shared_checksum";
  }
I
Igor Canadi 已提交
235
  inline std::string GetPrivateFileRel(BackupID backup_id,
I
Igor Canadi 已提交
236 237
                                       bool tmp = false,
                                       const std::string& file = "") const {
I
Igor Canadi 已提交
238
    assert(file.size() == 0 || file[0] != '/');
I
Igor Canadi 已提交
239 240
    return GetPrivateDirRel() + "/" + std::to_string(backup_id) +
           (tmp ? ".tmp" : "") + "/" + file;
I
Igor Canadi 已提交
241
  }
I
Igor Canadi 已提交
242 243
  inline std::string GetSharedFileRel(const std::string& file = "",
                                      bool tmp = false) const {
I
Igor Canadi 已提交
244
    assert(file.size() == 0 || file[0] != '/');
I
Igor Canadi 已提交
245
    return "shared/" + file + (tmp ? ".tmp" : "");
I
Igor Canadi 已提交
246
  }
247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267
  inline std::string GetSharedFileWithChecksumRel(const std::string& file = "",
                                                  bool tmp = false) const {
    assert(file.size() == 0 || file[0] != '/');
    return GetSharedChecksumDirRel() + "/" + file + (tmp ? ".tmp" : "");
  }
  inline std::string GetSharedFileWithChecksum(const std::string& file,
                                               const uint32_t checksum_value,
                                               const uint64_t file_size) const {
    assert(file.size() == 0 || file[0] != '/');
    std::string file_copy = file;
    return file_copy.insert(file_copy.find_last_of('.'),
                            "_" + std::to_string(checksum_value)
                              + "_" + std::to_string(file_size));
  }
  inline std::string GetFileFromChecksumFile(const std::string& file) const {
    assert(file.size() == 0 || file[0] != '/');
    std::string file_copy = file;
    size_t first_underscore = file_copy.find_first_of('_');
    return file_copy.erase(first_underscore,
                           file_copy.find_last_of('.') - first_underscore);
  }
I
Igor Canadi 已提交
268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285
  inline std::string GetLatestBackupFile(bool tmp = false) const {
    return GetAbsolutePath(std::string("LATEST_BACKUP") + (tmp ? ".tmp" : ""));
  }
  inline std::string GetBackupMetaDir() const {
    return GetAbsolutePath("meta");
  }
  inline std::string GetBackupMetaFile(BackupID backup_id) const {
    return GetBackupMetaDir() + "/" + std::to_string(backup_id);
  }

  Status GetLatestBackupFileContents(uint32_t* latest_backup);
  Status PutLatestBackupFileContents(uint32_t latest_backup);
  // if size_limit == 0, there is no size limit, copy everything
  Status CopyFile(const std::string& src,
                  const std::string& dst,
                  Env* src_env,
                  Env* dst_env,
                  bool sync,
L
Lei Jin 已提交
286
                  BackupRateLimiter* rate_limiter,
I
Igor Canadi 已提交
287
                  uint64_t* size = nullptr,
L
Lei Jin 已提交
288
                  uint32_t* checksum_value = nullptr,
I
Igor Canadi 已提交
289 290 291 292 293 294
                  uint64_t size_limit = 0);
  // if size_limit == 0, there is no size limit, copy everything
  Status BackupFile(BackupID backup_id,
                    BackupMeta* backup,
                    bool shared,
                    const std::string& src_dir,
295
                    const std::string& src_fname,  // starts with "/"
L
Lei Jin 已提交
296
                    BackupRateLimiter* rate_limiter,
297 298
                    uint64_t size_limit = 0,
                    bool shared_checksum = false);
L
Lei Jin 已提交
299 300 301 302 303 304

  Status CalculateChecksum(const std::string& src,
                           Env* src_env,
                           uint64_t size_limit,
                           uint32_t* checksum_value);

I
Igor Canadi 已提交
305 306
  // backup state data
  BackupID latest_backup_id_;
307 308 309 310 311
  std::map<BackupID, unique_ptr<BackupMeta>> backups_;
  std::map<BackupID,
           std::pair<Status, unique_ptr<BackupMeta>>> corrupt_backups_;
  std::unordered_map<std::string,
                     std::shared_ptr<FileInfo>> backuped_file_infos_;
I
Igor Canadi 已提交
312
  std::atomic<bool> stop_backup_;
I
Igor Canadi 已提交
313 314 315 316 317 318

  // options data
  BackupableDBOptions options_;
  Env* db_env_;
  Env* backup_env_;

I
Igor Canadi 已提交
319 320 321 322 323 324
  // directories
  unique_ptr<Directory> backup_directory_;
  unique_ptr<Directory> shared_directory_;
  unique_ptr<Directory> meta_directory_;
  unique_ptr<Directory> private_directory_;

I
Igor Canadi 已提交
325 326
  static const size_t kDefaultCopyFileBufferSize = 5 * 1024 * 1024LL;  // 5MB
  size_t copy_file_buffer_size_;
I
Igor Canadi 已提交
327
  bool read_only_;
328
  BackupStatistics backup_statistics_;
I
Igor Canadi 已提交
329 330
};

331 332
BackupEngine* BackupEngine::NewBackupEngine(
    Env* db_env, const BackupableDBOptions& options) {
I
Igor Canadi 已提交
333 334 335
  return new BackupEngineImpl(db_env, options);
}

H
Hasnain Lakhani 已提交
336 337 338 339 340 341 342
Status BackupEngine::Open(Env* env,
                          const BackupableDBOptions& options,
                          BackupEngine** backup_engine_ptr) {
  *backup_engine_ptr = new BackupEngineImpl(env, options);
  return Status::OK();
}

I
Igor Canadi 已提交
343
BackupEngineImpl::BackupEngineImpl(Env* db_env,
I
Igor Canadi 已提交
344 345
                                   const BackupableDBOptions& options,
                                   bool read_only)
I
Igor Canadi 已提交
346 347 348
    : stop_backup_(false),
      options_(options),
      db_env_(db_env),
I
Igor Canadi 已提交
349
      backup_env_(options.backup_env != nullptr ? options.backup_env : db_env_),
I
Igor Canadi 已提交
350 351 352 353 354
      copy_file_buffer_size_(kDefaultCopyFileBufferSize),
      read_only_(read_only) {
  if (read_only_) {
    Log(options_.info_log, "Starting read_only backup engine");
  }
I
Igor Canadi 已提交
355 356
  options_.Dump(options_.info_log);

I
Igor Canadi 已提交
357 358 359 360 361
  if (!read_only_) {
    // create all the dirs we need
    backup_env_->CreateDirIfMissing(GetAbsolutePath());
    backup_env_->NewDirectory(GetAbsolutePath(), &backup_directory_);
    if (options_.share_table_files) {
362 363 364 365 366 367 368 369 370 371
      if (options_.share_files_with_checksum) {
        backup_env_->CreateDirIfMissing(GetAbsolutePath(
            GetSharedFileWithChecksumRel()));
        backup_env_->NewDirectory(GetAbsolutePath(
            GetSharedFileWithChecksumRel()), &shared_directory_);
      } else {
        backup_env_->CreateDirIfMissing(GetAbsolutePath(GetSharedFileRel()));
        backup_env_->NewDirectory(GetAbsolutePath(GetSharedFileRel()),
                                  &shared_directory_);
      }
I
Igor Canadi 已提交
372 373 374 375 376 377
    }
    backup_env_->CreateDirIfMissing(GetAbsolutePath(GetPrivateDirRel()));
    backup_env_->NewDirectory(GetAbsolutePath(GetPrivateDirRel()),
                              &private_directory_);
    backup_env_->CreateDirIfMissing(GetBackupMetaDir());
    backup_env_->NewDirectory(GetBackupMetaDir(), &meta_directory_);
I
Igor Canadi 已提交
378
  }
I
Igor Canadi 已提交
379 380 381 382 383 384 385 386

  std::vector<std::string> backup_meta_files;
  backup_env_->GetChildren(GetBackupMetaDir(), &backup_meta_files);
  // create backups_ structure
  for (auto& file : backup_meta_files) {
    BackupID backup_id = 0;
    sscanf(file.c_str(), "%u", &backup_id);
    if (backup_id == 0 || file != std::to_string(backup_id)) {
I
Igor Canadi 已提交
387 388 389 390
      if (!read_only_) {
        // invalid file name, delete that
        backup_env_->DeleteFile(GetBackupMetaDir() + "/" + file);
      }
I
Igor Canadi 已提交
391 392 393
      continue;
    }
    assert(backups_.find(backup_id) == backups_.end());
394 395 396 397
    backups_.insert(std::move(
        std::make_pair(backup_id, unique_ptr<BackupMeta>(new BackupMeta(
                                      GetBackupMetaFile(backup_id),
                                      &backuped_file_infos_, backup_env_)))));
I
Igor Canadi 已提交
398 399
  }

400
  if (options_.destroy_old_data) {  // Destory old data
I
Igor Canadi 已提交
401
    assert(!read_only_);
H
Hasnain Lakhani 已提交
402 403
    PurgeOldBackups(0);
    (void) GarbageCollect();
I
Igor Canadi 已提交
404 405
    // start from beginning
    latest_backup_id_ = 0;
406
  } else {  // Load data from storage
I
Igor Canadi 已提交
407 408
    // load the backups if any
    for (auto& backup : backups_) {
409
      Status s = backup.second->LoadFromFile(options_.backup_dir);
I
Igor Canadi 已提交
410
      if (!s.ok()) {
I
Igor Canadi 已提交
411 412
        Log(options_.info_log, "Backup %u corrupted -- %s", backup.first,
            s.ToString().c_str());
H
Hasnain Lakhani 已提交
413
        corrupt_backups_.insert(std::make_pair(
414
              backup.first, std::make_pair(s, std::move(backup.second))));
I
Igor Canadi 已提交
415 416
      }
    }
H
Hasnain Lakhani 已提交
417

418
    for (const auto& corrupt : corrupt_backups_) {
H
Hasnain Lakhani 已提交
419
      backups_.erase(backups_.find(corrupt.first));
I
Igor Canadi 已提交
420 421 422
    }

    Status s = GetLatestBackupFileContents(&latest_backup_id_);
I
Igor Canadi 已提交
423

I
Igor Canadi 已提交
424 425 426 427 428 429 430 431 432 433 434
    // If latest backup file is corrupted or non-existent
    // set latest backup as the biggest backup we have
    // or 0 if we have no backups
    if (!s.ok() ||
        backups_.find(latest_backup_id_) == backups_.end()) {
      auto itr = backups_.end();
      latest_backup_id_ = (itr == backups_.begin()) ? 0 : (--itr)->first;
    }
  }

  // delete any backups that claim to be later than latest
H
Hasnain Lakhani 已提交
435 436 437 438 439 440 441
  std::vector<BackupID> later_ids;
  for (auto itr = backups_.lower_bound(latest_backup_id_ + 1);
       itr != backups_.end(); itr++) {
    later_ids.push_back(itr->first);
  }
  for (auto id : later_ids) {
    DeleteBackup(id);
I
Igor Canadi 已提交
442 443
  }

I
Igor Canadi 已提交
444 445 446 447
  if (!read_only_) {
    PutLatestBackupFileContents(latest_backup_id_);  // Ignore errors
  }
  Log(options_.info_log, "Initialized BackupEngine, the latest backup is %u.",
I
Igor Canadi 已提交
448 449 450
      latest_backup_id_);
}

I
Igor Canadi 已提交
451
BackupEngineImpl::~BackupEngineImpl() { LogFlush(options_.info_log); }
I
Igor Canadi 已提交
452

I
Igor Canadi 已提交
453
Status BackupEngineImpl::CreateNewBackup(DB* db, bool flush_before_backup) {
I
Igor Canadi 已提交
454
  assert(!read_only_);
I
Igor Canadi 已提交
455 456 457 458
  Status s;
  std::vector<std::string> live_files;
  VectorLogPtr live_wal_files;
  uint64_t manifest_file_size = 0;
459
  uint64_t sequence_number = db->GetLatestSequenceNumber();
I
Igor Canadi 已提交
460 461 462 463 464 465 466

  s = db->DisableFileDeletions();
  if (s.ok()) {
    // this will return live_files prefixed with "/"
    s = db->GetLiveFiles(live_files, &manifest_file_size, flush_before_backup);
  }
  // if we didn't flush before backup, we need to also get WAL files
467
  if (s.ok() && !flush_before_backup && options_.backup_log_files) {
I
Igor Canadi 已提交
468 469 470 471
    // returns file names prefixed with "/"
    s = db->GetSortedWalFiles(live_wal_files);
  }
  if (!s.ok()) {
472
    db->EnableFileDeletions(false);
I
Igor Canadi 已提交
473 474 475 476 477
    return s;
  }

  BackupID new_backup_id = latest_backup_id_ + 1;
  assert(backups_.find(new_backup_id) == backups_.end());
478 479 480 481
  auto ret = backups_.insert(std::move(
      std::make_pair(new_backup_id, unique_ptr<BackupMeta>(new BackupMeta(
                                        GetBackupMetaFile(new_backup_id),
                                        &backuped_file_infos_, backup_env_)))));
I
Igor Canadi 已提交
482 483
  assert(ret.second == true);
  auto& new_backup = ret.first->second;
484 485
  new_backup->RecordTimestamp();
  new_backup->SetSequenceNumber(sequence_number);
I
Igor Canadi 已提交
486

487 488
  auto start_backup = backup_env_-> NowMicros();

I
Igor Canadi 已提交
489 490 491
  Log(options_.info_log, "Started the backup process -- creating backup %u",
      new_backup_id);

I
Igor Canadi 已提交
492 493 494
  // create temporary private dir
  s = backup_env_->CreateDir(
      GetAbsolutePath(GetPrivateFileRel(new_backup_id, true)));
I
Igor Canadi 已提交
495

L
Lei Jin 已提交
496
  unique_ptr<BackupRateLimiter> rate_limiter;
I
Igor Canadi 已提交
497 498
  if (options_.backup_rate_limit > 0) {
    copy_file_buffer_size_ = options_.backup_rate_limit / 10;
L
Lei Jin 已提交
499 500
    rate_limiter.reset(new BackupRateLimiter(db_env_,
          options_.backup_rate_limit, copy_file_buffer_size_));
I
Igor Canadi 已提交
501 502
  }

I
Igor Canadi 已提交
503 504 505 506 507
  // copy live_files
  for (size_t i = 0; s.ok() && i < live_files.size(); ++i) {
    uint64_t number;
    FileType type;
    bool ok = ParseFileName(live_files[i], &number, &type);
I
Igor Canadi 已提交
508 509 510 511
    if (!ok) {
      assert(false);
      return Status::Corruption("Can't parse file name. This is very bad");
    }
I
Igor Canadi 已提交
512
    // we should only get sst, manifest and current files here
I
Igor Canadi 已提交
513 514
    assert(type == kTableFile || type == kDescriptorFile ||
           type == kCurrentFile);
I
Igor Canadi 已提交
515 516

    // rules:
517
    // * if it's kTableFile, then it's shared
I
Igor Canadi 已提交
518 519
    // * if it's kDescriptorFile, limit the size to manifest_file_size
    s = BackupFile(new_backup_id,
520
                   new_backup.get(),
I
Igor Canadi 已提交
521
                   options_.share_table_files && type == kTableFile,
I
Igor Canadi 已提交
522 523
                   db->GetName(),            /* src_dir */
                   live_files[i],            /* src_fname */
I
Igor Canadi 已提交
524
                   rate_limiter.get(),
525 526
                   (type == kDescriptorFile) ? manifest_file_size : 0,
                   options_.share_files_with_checksum && type == kTableFile);
I
Igor Canadi 已提交
527 528 529 530 531 532 533 534
  }

  // copy WAL files
  for (size_t i = 0; s.ok() && i < live_wal_files.size(); ++i) {
    if (live_wal_files[i]->Type() == kAliveLogFile) {
      // we only care about live log files
      // copy the file into backup_dir/files/<new backup>/
      s = BackupFile(new_backup_id,
535
                     new_backup.get(),
I
Igor Canadi 已提交
536 537
                     false, /* not shared */
                     db->GetOptions().wal_dir,
I
Igor Canadi 已提交
538 539
                     live_wal_files[i]->PathName(),
                     rate_limiter.get());
I
Igor Canadi 已提交
540 541 542 543
    }
  }

  // we copied all the files, enable file deletions
544
  db->EnableFileDeletions(false);
I
Igor Canadi 已提交
545

I
Igor Canadi 已提交
546 547 548
  if (s.ok()) {
    // move tmp private backup to real backup folder
    s = backup_env_->RenameFile(
549
        GetAbsolutePath(GetPrivateFileRel(new_backup_id, true)),  // tmp
I
Igor Canadi 已提交
550 551 552
        GetAbsolutePath(GetPrivateFileRel(new_backup_id, false)));
  }

553 554
  auto backup_time = backup_env_->NowMicros() - start_backup;

I
Igor Canadi 已提交
555 556
  if (s.ok()) {
    // persist the backup metadata on the disk
557
    s = new_backup->StoreToFile(options_.sync);
I
Igor Canadi 已提交
558 559 560 561 562
  }
  if (s.ok()) {
    // install the newly created backup meta! (atomic)
    s = PutLatestBackupFileContents(new_backup_id);
  }
I
Igor Canadi 已提交
563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584
  if (s.ok() && options_.sync) {
    unique_ptr<Directory> backup_private_directory;
    backup_env_->NewDirectory(
        GetAbsolutePath(GetPrivateFileRel(new_backup_id, false)),
        &backup_private_directory);
    if (backup_private_directory != nullptr) {
      backup_private_directory->Fsync();
    }
    if (private_directory_ != nullptr) {
      private_directory_->Fsync();
    }
    if (meta_directory_ != nullptr) {
      meta_directory_->Fsync();
    }
    if (shared_directory_ != nullptr) {
      shared_directory_->Fsync();
    }
    if (backup_directory_ != nullptr) {
      backup_directory_->Fsync();
    }
  }

585 586 587
  if (s.ok()) {
    backup_statistics_.IncrementNumberSuccessBackup();
  }
I
Igor Canadi 已提交
588
  if (!s.ok()) {
589
    backup_statistics_.IncrementNumberFailBackup();
I
Igor Canadi 已提交
590 591
    // clean all the files we might have created
    Log(options_.info_log, "Backup failed -- %s", s.ToString().c_str());
592 593
    Log(options_.info_log, "Backup Statistics %s\n",
        backup_statistics_.ToString().c_str());
I
Igor Canadi 已提交
594
    backups_.erase(new_backup_id);
H
Hasnain Lakhani 已提交
595
    (void) GarbageCollect();
I
Igor Canadi 已提交
596 597 598 599 600 601 602
    return s;
  }

  // here we know that we succeeded and installed the new backup
  // in the LATEST_BACKUP file
  latest_backup_id_ = new_backup_id;
  Log(options_.info_log, "Backup DONE. All is good");
603 604

  // backup_speed is in byte/second
605
  double backup_speed = new_backup->GetSize() / (1.048576 * backup_time);
606
  Log(options_.info_log, "Backup number of files: %u",
607
      new_backup->GetNumberFiles());
I
Igor Canadi 已提交
608
  Log(options_.info_log, "Backup size: %" PRIu64 " bytes",
609
      new_backup->GetSize());
I
Igor Canadi 已提交
610
  Log(options_.info_log, "Backup time: %" PRIu64 " microseconds", backup_time);
611 612 613
  Log(options_.info_log, "Backup speed: %.3f MB/s", backup_speed);
  Log(options_.info_log, "Backup Statistics %s",
      backup_statistics_.ToString().c_str());
I
Igor Canadi 已提交
614 615 616
  return s;
}

I
Igor Canadi 已提交
617
Status BackupEngineImpl::PurgeOldBackups(uint32_t num_backups_to_keep) {
I
Igor Canadi 已提交
618
  assert(!read_only_);
I
Igor Canadi 已提交
619 620
  Log(options_.info_log, "Purging old backups, keeping %u",
      num_backups_to_keep);
H
Hasnain Lakhani 已提交
621 622 623 624 625 626 627 628
  std::vector<BackupID> to_delete;
  auto itr = backups_.begin();
  while ((backups_.size() - to_delete.size()) > num_backups_to_keep) {
    to_delete.push_back(itr->first);
    itr++;
  }
  for (auto backup_id : to_delete) {
    DeleteBackup(backup_id);
I
Igor Canadi 已提交
629 630 631 632
  }
  return Status::OK();
}

I
Igor Canadi 已提交
633
Status BackupEngineImpl::DeleteBackup(BackupID backup_id) {
I
Igor Canadi 已提交
634
  assert(!read_only_);
I
Igor Canadi 已提交
635 636
  Log(options_.info_log, "Deleting backup %u", backup_id);
  auto backup = backups_.find(backup_id);
H
Hasnain Lakhani 已提交
637
  if (backup != backups_.end()) {
638
    backup->second->Delete();
H
Hasnain Lakhani 已提交
639 640 641 642 643 644
    backups_.erase(backup);
  } else {
    auto corrupt = corrupt_backups_.find(backup_id);
    if (corrupt == corrupt_backups_.end()) {
      return Status::NotFound("Backup not found");
    }
645
    corrupt->second.second->Delete();
H
Hasnain Lakhani 已提交
646 647 648 649 650
    corrupt_backups_.erase(corrupt);
  }

  std::vector<std::string> to_delete;
  for (auto& itr : backuped_file_infos_) {
651
    if (itr.second->refs == 0) {
H
Hasnain Lakhani 已提交
652 653 654 655 656 657 658 659
      Status s = backup_env_->DeleteFile(GetAbsolutePath(itr.first));
      Log(options_.info_log, "Deleting %s -- %s", itr.first.c_str(),
          s.ToString().c_str());
      to_delete.push_back(itr.first);
    }
  }
  for (auto& td : to_delete) {
    backuped_file_infos_.erase(td);
I
Igor Canadi 已提交
660
  }
H
Hasnain Lakhani 已提交
661 662 663 664 665 666 667

  // take care of private dirs -- GarbageCollect() will take care of them
  // if they are not empty
  std::string private_dir = GetPrivateFileRel(backup_id);
  Status s = backup_env_->DeleteDir(GetAbsolutePath(private_dir));
  Log(options_.info_log, "Deleting private dir %s -- %s",
      private_dir.c_str(), s.ToString().c_str());
I
Igor Canadi 已提交
668 669 670
  return Status::OK();
}

I
Igor Canadi 已提交
671
void BackupEngineImpl::GetBackupInfo(std::vector<BackupInfo>* backup_info) {
I
Igor Canadi 已提交
672 673
  backup_info->reserve(backups_.size());
  for (auto& backup : backups_) {
674
    if (!backup.second->Empty()) {
675
        backup_info->push_back(BackupInfo(
676 677 678
            backup.first, backup.second->GetTimestamp(),
            backup.second->GetSize(),
            backup.second->GetNumberFiles()));
I
Igor Canadi 已提交
679 680 681 682
    }
  }
}

H
Hasnain Lakhani 已提交
683 684 685 686 687 688 689 690 691
void
BackupEngineImpl::GetCorruptedBackups(
    std::vector<BackupID>* corrupt_backup_ids) {
  corrupt_backup_ids->reserve(corrupt_backups_.size());
  for (auto& backup : corrupt_backups_) {
    corrupt_backup_ids->push_back(backup.first);
  }
}

692 693 694
Status BackupEngineImpl::RestoreDBFromBackup(
    BackupID backup_id, const std::string& db_dir, const std::string& wal_dir,
    const RestoreOptions& restore_options) {
H
Hasnain Lakhani 已提交
695 696 697 698
  auto corrupt_itr = corrupt_backups_.find(backup_id);
  if (corrupt_itr != corrupt_backups_.end()) {
    return corrupt_itr->second.first;
  }
I
Igor Canadi 已提交
699 700 701 702 703
  auto backup_itr = backups_.find(backup_id);
  if (backup_itr == backups_.end()) {
    return Status::NotFound("Backup not found");
  }
  auto& backup = backup_itr->second;
704
  if (backup->Empty()) {
I
Igor Canadi 已提交
705 706 707 708
    return Status::NotFound("Backup not found");
  }

  Log(options_.info_log, "Restoring backup id %u\n", backup_id);
709 710
  Log(options_.info_log, "keep_log_files: %d\n",
      static_cast<int>(restore_options.keep_log_files));
I
Igor Canadi 已提交
711 712 713 714 715

  // just in case. Ignore errors
  db_env_->CreateDirIfMissing(db_dir);
  db_env_->CreateDirIfMissing(wal_dir);

716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739 740 741 742
  if (restore_options.keep_log_files) {
    // delete files in db_dir, but keep all the log files
    DeleteChildren(db_dir, 1 << kLogFile);
    // move all the files from archive dir to wal_dir
    std::string archive_dir = ArchivalDirectory(wal_dir);
    std::vector<std::string> archive_files;
    db_env_->GetChildren(archive_dir, &archive_files);  // ignore errors
    for (const auto& f : archive_files) {
      uint64_t number;
      FileType type;
      bool ok = ParseFileName(f, &number, &type);
      if (ok && type == kLogFile) {
        Log(options_.info_log, "Moving log file from archive/ to wal_dir: %s",
            f.c_str());
        Status s =
            db_env_->RenameFile(archive_dir + "/" + f, wal_dir + "/" + f);
        if (!s.ok()) {
          // if we can't move log file from archive_dir to wal_dir,
          // we should fail, since it might mean data loss
          return s;
        }
      }
    }
  } else {
    DeleteChildren(wal_dir);
    DeleteChildren(ArchivalDirectory(wal_dir));
    DeleteChildren(db_dir);
743
  }
I
Igor Canadi 已提交
744

L
Lei Jin 已提交
745
  unique_ptr<BackupRateLimiter> rate_limiter;
I
Igor Canadi 已提交
746 747
  if (options_.restore_rate_limit > 0) {
    copy_file_buffer_size_ = options_.restore_rate_limit / 10;
L
Lei Jin 已提交
748 749
    rate_limiter.reset(new BackupRateLimiter(db_env_,
          options_.restore_rate_limit, copy_file_buffer_size_));
I
Igor Canadi 已提交
750
  }
I
Igor Canadi 已提交
751
  Status s;
752 753
  for (const auto& file_info : backup->GetFiles()) {
    const std::string &file = file_info->filename;
I
Igor Canadi 已提交
754 755 756
    std::string dst;
    // 1. extract the filename
    size_t slash = file.find_last_of('/');
757 758
    // file will either be shared/<file>, shared_checksum/<file_crc32_size>
    // or private/<number>/<file>
I
Igor Canadi 已提交
759 760 761
    assert(slash != std::string::npos);
    dst = file.substr(slash + 1);

762 763 764 765 766 767
    // if the file was in shared_checksum, extract the real file name
    // in this case the file is <number>_<checksum>_<size>.<type>
    if (file.substr(0, slash) == GetSharedChecksumDirRel()) {
      dst = GetFileFromChecksumFile(dst);
    }

I
Igor Canadi 已提交
768 769 770 771 772 773 774 775 776 777 778 779 780
    // 2. find the filetype
    uint64_t number;
    FileType type;
    bool ok = ParseFileName(dst, &number, &type);
    if (!ok) {
      return Status::Corruption("Backup corrupted");
    }
    // 3. Construct the final path
    // kLogFile lives in wal_dir and all the rest live in db_dir
    dst = ((type == kLogFile) ? wal_dir : db_dir) +
      "/" + dst;

    Log(options_.info_log, "Restoring %s to %s\n", file.c_str(), dst.c_str());
L
Lei Jin 已提交
781 782
    uint32_t checksum_value;
    s = CopyFile(GetAbsolutePath(file), dst, backup_env_, db_env_, false,
I
Igor Canadi 已提交
783
                 rate_limiter.get(), nullptr /* size */, &checksum_value);
I
Igor Canadi 已提交
784 785 786
    if (!s.ok()) {
      break;
    }
L
Lei Jin 已提交
787

788
    if (file_info->checksum_value != checksum_value) {
L
Lei Jin 已提交
789 790 791
      s = Status::Corruption("Checksum check failed");
      break;
    }
I
Igor Canadi 已提交
792 793 794 795 796 797 798
  }

  Log(options_.info_log, "Restoring done -- %s\n", s.ToString().c_str());
  return s;
}

// latest backup id is an ASCII representation of latest backup id
I
Igor Canadi 已提交
799
Status BackupEngineImpl::GetLatestBackupFileContents(uint32_t* latest_backup) {
I
Igor Canadi 已提交
800 801 802 803 804 805 806 807 808
  Status s;
  unique_ptr<SequentialFile> file;
  s = backup_env_->NewSequentialFile(GetLatestBackupFile(),
                                     &file,
                                     EnvOptions());
  if (!s.ok()) {
    return s;
  }

809 810
  char buf[11];
  Slice data;
I
Igor Canadi 已提交
811 812 813 814
  s = file->Read(10, &data, buf);
  if (!s.ok() || data.size() == 0) {
    return s.ok() ? Status::Corruption("Latest backup file corrupted") : s;
  }
815
  buf[data.size()] = 0;
I
Igor Canadi 已提交
816

817
  *latest_backup = 0;
I
Igor Canadi 已提交
818 819 820 821 822 823 824 825 826 827 828
  sscanf(data.data(), "%u", latest_backup);
  if (backup_env_->FileExists(GetBackupMetaFile(*latest_backup)) == false) {
    s = Status::Corruption("Latest backup file corrupted");
  }
  return Status::OK();
}

// this operation HAS to be atomic
// writing 4 bytes to the file is atomic alright, but we should *never*
// do something like 1. delete file, 2. write new file
// We write to a tmp file and then atomically rename
I
Igor Canadi 已提交
829
Status BackupEngineImpl::PutLatestBackupFileContents(uint32_t latest_backup) {
I
Igor Canadi 已提交
830
  assert(!read_only_);
I
Igor Canadi 已提交
831 832 833 834 835 836 837 838 839 840 841 842
  Status s;
  unique_ptr<WritableFile> file;
  EnvOptions env_options;
  env_options.use_mmap_writes = false;
  s = backup_env_->NewWritableFile(GetLatestBackupFile(true),
                                   &file,
                                   env_options);
  if (!s.ok()) {
    backup_env_->DeleteFile(GetLatestBackupFile(true));
    return s;
  }

843
  char file_contents[10];
I
Igor Canadi 已提交
844 845 846 847 848 849 850 851 852 853 854 855 856 857 858 859
  int len = sprintf(file_contents, "%u\n", latest_backup);
  s = file->Append(Slice(file_contents, len));
  if (s.ok() && options_.sync) {
    file->Sync();
  }
  if (s.ok()) {
    s = file->Close();
  }
  if (s.ok()) {
    // atomically replace real file with new tmp
    s = backup_env_->RenameFile(GetLatestBackupFile(true),
                                GetLatestBackupFile(false));
  }
  return s;
}

L
Lei Jin 已提交
860 861 862 863 864 865 866
Status BackupEngineImpl::CopyFile(
    const std::string& src,
    const std::string& dst, Env* src_env,
    Env* dst_env, bool sync,
    BackupRateLimiter* rate_limiter, uint64_t* size,
    uint32_t* checksum_value,
    uint64_t size_limit) {
I
Igor Canadi 已提交
867 868 869 870 871
  Status s;
  unique_ptr<WritableFile> dst_file;
  unique_ptr<SequentialFile> src_file;
  EnvOptions env_options;
  env_options.use_mmap_writes = false;
I
Igor Canadi 已提交
872
  env_options.use_os_buffer = false;
I
Igor Canadi 已提交
873 874 875
  if (size != nullptr) {
    *size = 0;
  }
L
Lei Jin 已提交
876 877 878
  if (checksum_value != nullptr) {
    *checksum_value = 0;
  }
I
Igor Canadi 已提交
879 880 881 882 883 884 885 886 887 888 889 890 891 892

  // Check if size limit is set. if not, set it to very big number
  if (size_limit == 0) {
    size_limit = std::numeric_limits<uint64_t>::max();
  }

  s = src_env->NewSequentialFile(src, &src_file, env_options);
  if (s.ok()) {
    s = dst_env->NewWritableFile(dst, &dst_file, env_options);
  }
  if (!s.ok()) {
    return s;
  }

893 894
  unique_ptr<char[]> buf(new char[copy_file_buffer_size_]);
  Slice data;
I
Igor Canadi 已提交
895 896

  do {
I
Igor Canadi 已提交
897 898 899
    if (stop_backup_.load(std::memory_order_acquire)) {
      return Status::Incomplete("Backup stopped");
    }
I
Igor Canadi 已提交
900 901
    size_t buffer_to_read = (copy_file_buffer_size_ < size_limit) ?
      copy_file_buffer_size_ : size_limit;
902
    s = src_file->Read(buffer_to_read, &data, buf.get());
I
Igor Canadi 已提交
903
    size_limit -= data.size();
L
Lei Jin 已提交
904 905 906 907 908

    if (!s.ok()) {
      return s;
    }

I
Igor Canadi 已提交
909 910 911
    if (size != nullptr) {
      *size += data.size();
    }
L
Lei Jin 已提交
912 913 914
    if (checksum_value != nullptr) {
      *checksum_value = crc32c::Extend(*checksum_value, data.data(),
                                       data.size());
I
Igor Canadi 已提交
915
    }
L
Lei Jin 已提交
916
    s = dst_file->Append(data);
I
Igor Canadi 已提交
917 918 919
    if (rate_limiter != nullptr) {
      rate_limiter->ReportAndWait(data.size());
    }
I
Igor Canadi 已提交
920 921 922 923 924 925 926 927 928 929
  } while (s.ok() && data.size() > 0 && size_limit > 0);

  if (s.ok() && sync) {
    s = dst_file->Sync();
  }

  return s;
}

// src_fname will always start with "/"
I
Igor Canadi 已提交
930 931 932
Status BackupEngineImpl::BackupFile(BackupID backup_id, BackupMeta* backup,
                                    bool shared, const std::string& src_dir,
                                    const std::string& src_fname,
L
Lei Jin 已提交
933
                                    BackupRateLimiter* rate_limiter,
934 935
                                    uint64_t size_limit,
                                    bool shared_checksum) {
I
Igor Canadi 已提交
936 937 938

  assert(src_fname.size() > 0 && src_fname[0] == '/');
  std::string dst_relative = src_fname.substr(1);
I
Igor Canadi 已提交
939
  std::string dst_relative_tmp;
940 941 942 943 944 945 946 947 948 949 950 951 952 953 954 955 956 957 958 959 960
  Status s;
  uint64_t size;
  uint32_t checksum_value = 0;

  if (shared && shared_checksum) {
    // add checksum and file length to the file name
    s = CalculateChecksum(src_dir + src_fname,
                          db_env_,
                          size_limit,
                          &checksum_value);
    if (s.ok()) {
        s = db_env_->GetFileSize(src_dir + src_fname, &size);
    }
    if (!s.ok()) {
         return s;
    }
    dst_relative = GetSharedFileWithChecksum(dst_relative, checksum_value,
                                             size);
    dst_relative_tmp = GetSharedFileWithChecksumRel(dst_relative, true);
    dst_relative = GetSharedFileWithChecksumRel(dst_relative, false);
  } else if (shared) {
I
Igor Canadi 已提交
961 962
    dst_relative_tmp = GetSharedFileRel(dst_relative, true);
    dst_relative = GetSharedFileRel(dst_relative, false);
I
Igor Canadi 已提交
963
  } else {
I
Igor Canadi 已提交
964 965
    dst_relative_tmp = GetPrivateFileRel(backup_id, true, dst_relative);
    dst_relative = GetPrivateFileRel(backup_id, false, dst_relative);
I
Igor Canadi 已提交
966 967
  }
  std::string dst_path = GetAbsolutePath(dst_relative);
I
Igor Canadi 已提交
968
  std::string dst_path_tmp = GetAbsolutePath(dst_relative_tmp);
I
Igor Canadi 已提交
969 970 971 972

  // if it's shared, we also need to check if it exists -- if it does,
  // no need to copy it again
  if (shared && backup_env_->FileExists(dst_path)) {
973 974 975 976 977 978 979 980 981 982 983 984 985
    if (shared_checksum) {
      Log(options_.info_log,
          "%s already present, with checksum %u and size %" PRIu64,
          src_fname.c_str(), checksum_value, size);
    } else {
      backup_env_->GetFileSize(dst_path, &size);  // Ignore error
      Log(options_.info_log, "%s already present, calculate checksum",
          src_fname.c_str());
      s = CalculateChecksum(src_dir + src_fname,
                            db_env_,
                            size_limit,
                            &checksum_value);
    }
I
Igor Canadi 已提交
986 987 988
  } else {
    Log(options_.info_log, "Copying %s", src_fname.c_str());
    s = CopyFile(src_dir + src_fname,
I
Igor Canadi 已提交
989
                 dst_path_tmp,
I
Igor Canadi 已提交
990 991 992
                 db_env_,
                 backup_env_,
                 options_.sync,
I
Igor Canadi 已提交
993
                 rate_limiter,
I
Igor Canadi 已提交
994
                 &size,
L
Lei Jin 已提交
995
                 &checksum_value,
I
Igor Canadi 已提交
996
                 size_limit);
I
Igor Canadi 已提交
997 998 999
    if (s.ok() && shared) {
      s = backup_env_->RenameFile(dst_path_tmp, dst_path);
    }
I
Igor Canadi 已提交
1000 1001
  }
  if (s.ok()) {
1002 1003
    s = backup->AddFile(std::make_shared<FileInfo>(
          dst_relative, size, checksum_value));
L
Lei Jin 已提交
1004 1005 1006 1007
  }
  return s;
}

I
Igor Canadi 已提交
1008 1009 1010
Status BackupEngineImpl::CalculateChecksum(const std::string& src, Env* src_env,
                                           uint64_t size_limit,
                                           uint32_t* checksum_value) {
L
Lei Jin 已提交
1011 1012 1013 1014 1015 1016 1017
  *checksum_value = 0;
  if (size_limit == 0) {
    size_limit = std::numeric_limits<uint64_t>::max();
  }

  EnvOptions env_options;
  env_options.use_mmap_writes = false;
I
Igor Canadi 已提交
1018
  env_options.use_os_buffer = false;
L
Lei Jin 已提交
1019 1020 1021 1022 1023

  std::unique_ptr<SequentialFile> src_file;
  Status s = src_env->NewSequentialFile(src, &src_file, env_options);
  if (!s.ok()) {
    return s;
I
Igor Canadi 已提交
1024
  }
L
Lei Jin 已提交
1025 1026 1027 1028 1029 1030 1031 1032 1033 1034 1035 1036 1037 1038 1039 1040 1041 1042 1043 1044

  std::unique_ptr<char[]> buf(new char[copy_file_buffer_size_]);
  Slice data;

  do {
    if (stop_backup_.load(std::memory_order_acquire)) {
      return Status::Incomplete("Backup stopped");
    }
    size_t buffer_to_read = (copy_file_buffer_size_ < size_limit) ?
      copy_file_buffer_size_ : size_limit;
    s = src_file->Read(buffer_to_read, &data, buf.get());

    if (!s.ok()) {
      return s;
    }

    size_limit -= data.size();
    *checksum_value = crc32c::Extend(*checksum_value, data.data(), data.size());
  } while (data.size() > 0 && size_limit > 0);

I
Igor Canadi 已提交
1045 1046 1047
  return s;
}

1048 1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059 1060 1061 1062 1063 1064
void BackupEngineImpl::DeleteChildren(const std::string& dir,
                                      uint32_t file_type_filter) {
  std::vector<std::string> children;
  db_env_->GetChildren(dir, &children);  // ignore errors

  for (const auto& f : children) {
    uint64_t number;
    FileType type;
    bool ok = ParseFileName(f, &number, &type);
    if (ok && (file_type_filter & (1 << type))) {
      // don't delete this file
      continue;
    }
    db_env_->DeleteFile(dir + "/" + f);  // ignore errors
  }
}

H
Hasnain Lakhani 已提交
1065
Status BackupEngineImpl::GarbageCollect() {
I
Igor Canadi 已提交
1066
  assert(!read_only_);
I
Igor Canadi 已提交
1067
  Log(options_.info_log, "Starting garbage collection");
H
Hasnain Lakhani 已提交
1068 1069 1070 1071 1072 1073 1074 1075 1076 1077 1078 1079 1080 1081

  // delete obsolete shared files
  std::vector<std::string> shared_children;
  backup_env_->GetChildren(GetAbsolutePath(GetSharedFileRel()),
                           &shared_children);
  for (auto& child : shared_children) {
    std::string rel_fname = GetSharedFileRel(child);
    // if it's not refcounted, delete it
    if (backuped_file_infos_.find(rel_fname) == backuped_file_infos_.end()) {
      // this might be a directory, but DeleteFile will just fail in that
      // case, so we're good
      Status s = backup_env_->DeleteFile(GetAbsolutePath(rel_fname));
      if (s.ok()) {
        Log(options_.info_log, "Deleted %s", rel_fname.c_str());
I
Igor Canadi 已提交
1082 1083
      }
    }
H
Hasnain Lakhani 已提交
1084
  }
I
Igor Canadi 已提交
1085

H
Hasnain Lakhani 已提交
1086 1087 1088 1089 1090 1091 1092 1093 1094 1095 1096 1097 1098 1099 1100 1101 1102 1103 1104 1105 1106 1107 1108
  // delete obsolete private files
  std::vector<std::string> private_children;
  backup_env_->GetChildren(GetAbsolutePath(GetPrivateDirRel()),
                           &private_children);
  for (auto& child : private_children) {
    BackupID backup_id = 0;
    bool tmp_dir = child.find(".tmp") != std::string::npos;
    sscanf(child.c_str(), "%u", &backup_id);
    if (!tmp_dir &&  // if it's tmp_dir, delete it
        (backup_id == 0 || backups_.find(backup_id) != backups_.end())) {
      // it's either not a number or it's still alive. continue
      continue;
    }
    // here we have to delete the dir and all its children
    std::string full_private_path =
        GetAbsolutePath(GetPrivateFileRel(backup_id, tmp_dir));
    std::vector<std::string> subchildren;
    backup_env_->GetChildren(full_private_path, &subchildren);
    for (auto& subchild : subchildren) {
      Status s = backup_env_->DeleteFile(full_private_path + subchild);
      if (s.ok()) {
        Log(options_.info_log, "Deleted %s",
            (full_private_path + subchild).c_str());
I
Igor Canadi 已提交
1109 1110
      }
    }
H
Hasnain Lakhani 已提交
1111 1112 1113 1114
    // finally delete the private dir
    Status s = backup_env_->DeleteDir(full_private_path);
    Log(options_.info_log, "Deleted dir %s -- %s", full_private_path.c_str(),
        s.ToString().c_str());
I
Igor Canadi 已提交
1115
  }
H
Hasnain Lakhani 已提交
1116 1117

  return Status::OK();
I
Igor Canadi 已提交
1118 1119 1120 1121
}

// ------- BackupMeta class --------

1122 1123 1124
Status BackupEngineImpl::BackupMeta::AddFile(
    std::shared_ptr<FileInfo> file_info) {
  auto itr = file_infos_->find(file_info->filename);
L
Lei Jin 已提交
1125
  if (itr == file_infos_->end()) {
1126
    auto ret = file_infos_->insert({file_info->filename, file_info});
L
Lei Jin 已提交
1127
    if (ret.second) {
1128 1129
      itr = ret.first;
      itr->second->refs = 1;
L
Lei Jin 已提交
1130 1131 1132 1133
    } else {
      // if this happens, something is seriously wrong
      return Status::Corruption("In memory metadata insertion error");
    }
I
Igor Canadi 已提交
1134
  } else {
1135
    if (itr->second->checksum_value != file_info->checksum_value) {
L
Lei Jin 已提交
1136 1137
      return Status::Corruption("Checksum mismatch for existing backup file");
    }
1138
    ++itr->second->refs;  // increase refcount if already present
I
Igor Canadi 已提交
1139
  }
L
Lei Jin 已提交
1140

1141 1142 1143
  size_ += file_info->size;
  files_.push_back(itr->second);

L
Lei Jin 已提交
1144
  return Status::OK();
I
Igor Canadi 已提交
1145 1146
}

I
Igor Canadi 已提交
1147
void BackupEngineImpl::BackupMeta::Delete(bool delete_meta) {
L
Lei Jin 已提交
1148
  for (const auto& file : files_) {
1149
    --file->refs;  // decrease refcount
I
Igor Canadi 已提交
1150 1151 1152
  }
  files_.clear();
  // delete meta file
I
Igor Canadi 已提交
1153 1154 1155
  if (delete_meta) {
    env_->DeleteFile(meta_filename_);
  }
I
Igor Canadi 已提交
1156 1157 1158 1159 1160
  timestamp_ = 0;
}

// each backup meta file is of the format:
// <timestamp>
1161
// <seq number>
I
Igor Canadi 已提交
1162
// <number of files>
L
Lei Jin 已提交
1163 1164
// <file1> <crc32(literal string)> <crc32_value>
// <file2> <crc32(literal string)> <crc32_value>
I
Igor Canadi 已提交
1165
// ...
I
Igor Canadi 已提交
1166 1167
Status BackupEngineImpl::BackupMeta::LoadFromFile(
    const std::string& backup_dir) {
I
Igor Canadi 已提交
1168 1169 1170 1171 1172 1173 1174 1175
  assert(Empty());
  Status s;
  unique_ptr<SequentialFile> backup_meta_file;
  s = env_->NewSequentialFile(meta_filename_, &backup_meta_file, EnvOptions());
  if (!s.ok()) {
    return s;
  }

1176 1177 1178
  unique_ptr<char[]> buf(new char[max_backup_meta_file_size_ + 1]);
  Slice data;
  s = backup_meta_file->Read(max_backup_meta_file_size_, &data, buf.get());
I
Igor Canadi 已提交
1179 1180

  if (!s.ok() || data.size() == max_backup_meta_file_size_) {
L
Lei Jin 已提交
1181
    return s.ok() ? Status::Corruption("File size too big") : s;
I
Igor Canadi 已提交
1182 1183 1184 1185
  }
  buf[data.size()] = 0;

  uint32_t num_files = 0;
1186 1187 1188 1189 1190
  char *next;
  timestamp_ = strtoull(data.data(), &next, 10);
  data.remove_prefix(next - data.data() + 1); // +1 for '\n'
  sequence_number_ = strtoull(data.data(), &next, 10);
  data.remove_prefix(next - data.data() + 1); // +1 for '\n'
1191
  num_files = static_cast<uint32_t>(strtoul(data.data(), &next, 10));
1192
  data.remove_prefix(next - data.data() + 1); // +1 for '\n'
I
Igor Canadi 已提交
1193

1194
  std::vector<std::shared_ptr<FileInfo>> files;
1195

1196 1197
  Slice checksum_prefix("crc32 ");

I
Igor Canadi 已提交
1198
  for (uint32_t i = 0; s.ok() && i < num_files; ++i) {
L
Lei Jin 已提交
1199 1200 1201
    auto line = GetSliceUntil(&data, '\n');
    std::string filename = GetSliceUntil(&line, ' ').ToString();

I
Igor Canadi 已提交
1202
    uint64_t size;
1203 1204
    const std::shared_ptr<FileInfo> file_info = GetFile(filename);
    if (file_info) {
1205 1206 1207 1208 1209 1210
      size = file_info->size;
    } else {
      s = env_->GetFileSize(backup_dir + "/" + filename, &size);
      if (!s.ok()) {
        return s;
      }
I
Igor Canadi 已提交
1211
    }
L
Lei Jin 已提交
1212 1213 1214 1215 1216 1217

    if (line.empty()) {
      return Status::Corruption("File checksum is missing");
    }

    uint32_t checksum_value = 0;
1218 1219
    if (line.starts_with(checksum_prefix)) {
      line.remove_prefix(checksum_prefix.size());
1220 1221
      checksum_value = static_cast<uint32_t>(
          strtoul(line.data(), nullptr, 10));
1222
      if (line != std::to_string(checksum_value)) {
L
Lei Jin 已提交
1223 1224 1225 1226 1227 1228
        return Status::Corruption("Invalid checksum value");
      }
    } else {
      return Status::Corruption("Unknown checksum type");
    }

1229
    files.emplace_back(new FileInfo(filename, size, checksum_value));
1230 1231
  }

I
Igor Canadi 已提交
1232 1233 1234 1235 1236
  if (s.ok() && data.size() > 0) {
    // file has to be read completely. if not, we count it as corruption
    s = Status::Corruption("Tailing data in backup meta file");
  }

1237
  if (s.ok()) {
1238
    files_.reserve(files.size());
L
Lei Jin 已提交
1239 1240 1241 1242 1243
    for (const auto& file_info : files) {
      s = AddFile(file_info);
      if (!s.ok()) {
        break;
      }
1244
    }
I
Igor Canadi 已提交
1245 1246 1247 1248 1249
  }

  return s;
}

I
Igor Canadi 已提交
1250
Status BackupEngineImpl::BackupMeta::StoreToFile(bool sync) {
I
Igor Canadi 已提交
1251 1252 1253 1254 1255 1256 1257 1258 1259 1260
  Status s;
  unique_ptr<WritableFile> backup_meta_file;
  EnvOptions env_options;
  env_options.use_mmap_writes = false;
  s = env_->NewWritableFile(meta_filename_ + ".tmp", &backup_meta_file,
                            env_options);
  if (!s.ok()) {
    return s;
  }

1261
  unique_ptr<char[]> buf(new char[max_backup_meta_file_size_]);
I
Igor Canadi 已提交
1262
  int len = 0, buf_size = max_backup_meta_file_size_;
1263
  len += snprintf(buf.get(), buf_size, "%" PRId64 "\n", timestamp_);
1264 1265
  len += snprintf(buf.get() + len, buf_size - len, "%" PRIu64 "\n",
                  sequence_number_);
1266
  len += snprintf(buf.get() + len, buf_size - len, "%zu\n", files_.size());
L
Lei Jin 已提交
1267 1268 1269
  for (const auto& file : files_) {
    // use crc32 for now, switch to something else if needed
    len += snprintf(buf.get() + len, buf_size - len, "%s crc32 %u\n",
1270
                    file->filename.c_str(), file->checksum_value);
I
Igor Canadi 已提交
1271 1272
  }

1273
  s = backup_meta_file->Append(Slice(buf.get(), (size_t)len));
I
Igor Canadi 已提交
1274 1275 1276 1277 1278 1279 1280 1281 1282 1283 1284 1285
  if (s.ok() && sync) {
    s = backup_meta_file->Sync();
  }
  if (s.ok()) {
    s = backup_meta_file->Close();
  }
  if (s.ok()) {
    s = env_->RenameFile(meta_filename_ + ".tmp", meta_filename_);
  }
  return s;
}

I
Igor Canadi 已提交
1286 1287 1288
// -------- BackupEngineReadOnlyImpl ---------
class BackupEngineReadOnlyImpl : public BackupEngineReadOnly {
 public:
I
Igor Canadi 已提交
1289 1290 1291
  BackupEngineReadOnlyImpl(Env* db_env, const BackupableDBOptions& options)
      : backup_engine_(new BackupEngineImpl(db_env, options, true)) {}

I
Igor Canadi 已提交
1292 1293 1294 1295 1296 1297
  virtual ~BackupEngineReadOnlyImpl() {}

  virtual void GetBackupInfo(std::vector<BackupInfo>* backup_info) {
    backup_engine_->GetBackupInfo(backup_info);
  }

H
Hasnain Lakhani 已提交
1298 1299 1300 1301
  virtual void GetCorruptedBackups(std::vector<BackupID>* corrupt_backup_ids) {
    backup_engine_->GetCorruptedBackups(corrupt_backup_ids);
  }

I
Igor Canadi 已提交
1302 1303 1304 1305 1306 1307 1308 1309 1310 1311 1312 1313 1314 1315 1316
  virtual Status RestoreDBFromBackup(
      BackupID backup_id, const std::string& db_dir, const std::string& wal_dir,
      const RestoreOptions& restore_options = RestoreOptions()) {
    return backup_engine_->RestoreDBFromBackup(backup_id, db_dir, wal_dir,
                                               restore_options);
  }

  virtual Status RestoreDBFromLatestBackup(
      const std::string& db_dir, const std::string& wal_dir,
      const RestoreOptions& restore_options = RestoreOptions()) {
    return backup_engine_->RestoreDBFromLatestBackup(db_dir, wal_dir,
                                                     restore_options);
  }

 private:
I
Igor Canadi 已提交
1317
  std::unique_ptr<BackupEngineImpl> backup_engine_;
I
Igor Canadi 已提交
1318 1319 1320 1321 1322 1323 1324 1325 1326 1327 1328
};

BackupEngineReadOnly* BackupEngineReadOnly::NewReadOnlyBackupEngine(
    Env* db_env, const BackupableDBOptions& options) {
  if (options.destroy_old_data) {
    assert(false);
    return nullptr;
  }
  return new BackupEngineReadOnlyImpl(db_env, options);
}

I
Igor Canadi 已提交
1329 1330
// --- BackupableDB methods --------

1331
BackupableDB::BackupableDB(DB* db, const BackupableDBOptions& options)
I
Igor Canadi 已提交
1332
    : StackableDB(db),
I
Igor Canadi 已提交
1333
      backup_engine_(new BackupEngineImpl(db->GetEnv(), options)) {}
I
Igor Canadi 已提交
1334 1335 1336 1337 1338 1339 1340 1341 1342 1343 1344 1345 1346

BackupableDB::~BackupableDB() {
  delete backup_engine_;
}

Status BackupableDB::CreateNewBackup(bool flush_before_backup) {
  return backup_engine_->CreateNewBackup(this, flush_before_backup);
}

void BackupableDB::GetBackupInfo(std::vector<BackupInfo>* backup_info) {
  backup_engine_->GetBackupInfo(backup_info);
}

H
Hasnain Lakhani 已提交
1347 1348 1349 1350 1351
void
BackupableDB::GetCorruptedBackups(std::vector<BackupID>* corrupt_backup_ids) {
  backup_engine_->GetCorruptedBackups(corrupt_backup_ids);
}

I
Igor Canadi 已提交
1352 1353 1354 1355 1356 1357 1358 1359
Status BackupableDB::PurgeOldBackups(uint32_t num_backups_to_keep) {
  return backup_engine_->PurgeOldBackups(num_backups_to_keep);
}

Status BackupableDB::DeleteBackup(BackupID backup_id) {
  return backup_engine_->DeleteBackup(backup_id);
}

I
Igor Canadi 已提交
1360 1361 1362 1363
void BackupableDB::StopBackup() {
  backup_engine_->StopBackup();
}

H
Hasnain Lakhani 已提交
1364 1365 1366 1367
Status BackupableDB::GarbageCollect() {
  return backup_engine_->GarbageCollect();
}

I
Igor Canadi 已提交
1368 1369 1370 1371
// --- RestoreBackupableDB methods ------

RestoreBackupableDB::RestoreBackupableDB(Env* db_env,
                                         const BackupableDBOptions& options)
I
Igor Canadi 已提交
1372
    : backup_engine_(new BackupEngineImpl(db_env, options)) {}
I
Igor Canadi 已提交
1373 1374 1375 1376 1377 1378 1379 1380 1381 1382

RestoreBackupableDB::~RestoreBackupableDB() {
  delete backup_engine_;
}

void
RestoreBackupableDB::GetBackupInfo(std::vector<BackupInfo>* backup_info) {
  backup_engine_->GetBackupInfo(backup_info);
}

H
Hasnain Lakhani 已提交
1383 1384 1385 1386 1387
void RestoreBackupableDB::GetCorruptedBackups(
    std::vector<BackupID>* corrupt_backup_ids) {
  backup_engine_->GetCorruptedBackups(corrupt_backup_ids);
}

1388 1389 1390 1391 1392
Status RestoreBackupableDB::RestoreDBFromBackup(
    BackupID backup_id, const std::string& db_dir, const std::string& wal_dir,
    const RestoreOptions& restore_options) {
  return backup_engine_->RestoreDBFromBackup(backup_id, db_dir, wal_dir,
                                             restore_options);
I
Igor Canadi 已提交
1393 1394
}

1395 1396 1397 1398 1399
Status RestoreBackupableDB::RestoreDBFromLatestBackup(
    const std::string& db_dir, const std::string& wal_dir,
    const RestoreOptions& restore_options) {
  return backup_engine_->RestoreDBFromLatestBackup(db_dir, wal_dir,
                                                   restore_options);
I
Igor Canadi 已提交
1400 1401 1402 1403 1404 1405 1406 1407 1408 1409
}

Status RestoreBackupableDB::PurgeOldBackups(uint32_t num_backups_to_keep) {
  return backup_engine_->PurgeOldBackups(num_backups_to_keep);
}

Status RestoreBackupableDB::DeleteBackup(BackupID backup_id) {
  return backup_engine_->DeleteBackup(backup_id);
}

H
Hasnain Lakhani 已提交
1410 1411 1412 1413
Status RestoreBackupableDB::GarbageCollect() {
  return backup_engine_->GarbageCollect();
}

I
Igor Canadi 已提交
1414
}  // namespace rocksdb
I
Igor Canadi 已提交
1415 1416

#endif  // ROCKSDB_LITE