repair.cc 25 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691
  1. // Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
  2. // This source code is licensed under both the GPLv2 (found in the
  3. // COPYING file in the root directory) and Apache 2.0 License
  4. // (found in the LICENSE.Apache file in the root directory).
  5. //
  6. // Copyright (c) 2011 The LevelDB Authors. All rights reserved.
  7. // Use of this source code is governed by a BSD-style license that can be
  8. // found in the LICENSE file. See the AUTHORS file for names of contributors.
  9. //
  10. // Repairer does best effort recovery to recover as much data as possible after
  11. // a disaster without compromising consistency. It does not guarantee bringing
  12. // the database to a time consistent state.
  13. //
  14. // Repair process is broken into 4 phases:
  15. // (a) Find files
  16. // (b) Convert logs to tables
  17. // (c) Extract metadata
  18. // (d) Write Descriptor
  19. //
  20. // (a) Find files
  21. //
  22. // The repairer goes through all the files in the directory, and classifies them
  23. // based on their file name. Any file that cannot be identified by name will be
  24. // ignored.
  25. //
  26. // (b) Convert logs to table
  27. //
  28. // Every log file that is active is replayed. All sections of the file where the
  29. // checksum does not match is skipped over. We intentionally give preference to
  30. // data consistency.
  31. //
  32. // (c) Extract metadata
  33. //
  34. // We scan every table to compute
  35. // (1) smallest/largest for the table
  36. // (2) largest sequence number in the table
  37. // (3) oldest blob file referred to by the table (if applicable)
  38. //
  39. // If we are unable to scan the file, then we ignore the table.
  40. //
  41. // (d) Write Descriptor
  42. //
  43. // We generate descriptor contents:
  44. // - log number is set to zero
  45. // - next-file-number is set to 1 + largest file number we found
  46. // - last-sequence-number is set to largest sequence# found across
  47. // all tables (see 2c)
  48. // - compaction pointers are cleared
  49. // - every table file is added at level 0
  50. //
  51. // Possible optimization 1:
  52. // (a) Compute total size and use to pick appropriate max-level M
  53. // (b) Sort tables by largest sequence# in the table
  54. // (c) For each table: if it overlaps earlier table, place in level-0,
  55. // else place in level-M.
  56. // (d) We can provide options for time consistent recovery and unsafe recovery
  57. // (ignore checksum failure when applicable)
  58. // Possible optimization 2:
  59. // Store per-table metadata (smallest, largest, largest-seq#, ...)
  60. // in the table's meta section to speed up ScanTable.
  61. #ifndef ROCKSDB_LITE
  62. #include <cinttypes>
  63. #include "db/builder.h"
  64. #include "db/db_impl/db_impl.h"
  65. #include "db/dbformat.h"
  66. #include "db/log_reader.h"
  67. #include "db/log_writer.h"
  68. #include "db/memtable.h"
  69. #include "db/table_cache.h"
  70. #include "db/version_edit.h"
  71. #include "db/write_batch_internal.h"
  72. #include "env/composite_env_wrapper.h"
  73. #include "file/filename.h"
  74. #include "file/writable_file_writer.h"
  75. #include "options/cf_options.h"
  76. #include "rocksdb/comparator.h"
  77. #include "rocksdb/db.h"
  78. #include "rocksdb/env.h"
  79. #include "rocksdb/options.h"
  80. #include "rocksdb/write_buffer_manager.h"
  81. #include "table/scoped_arena_iterator.h"
  82. #include "util/string_util.h"
  83. namespace ROCKSDB_NAMESPACE {
  84. namespace {
  85. class Repairer {
  86. public:
  87. Repairer(const std::string& dbname, const DBOptions& db_options,
  88. const std::vector<ColumnFamilyDescriptor>& column_families,
  89. const ColumnFamilyOptions& default_cf_opts,
  90. const ColumnFamilyOptions& unknown_cf_opts, bool create_unknown_cfs)
  91. : dbname_(dbname),
  92. env_(db_options.env),
  93. env_options_(),
  94. db_options_(SanitizeOptions(dbname_, db_options)),
  95. immutable_db_options_(ImmutableDBOptions(db_options_)),
  96. icmp_(default_cf_opts.comparator),
  97. default_cf_opts_(
  98. SanitizeOptions(immutable_db_options_, default_cf_opts)),
  99. default_cf_iopts_(
  100. ImmutableCFOptions(immutable_db_options_, default_cf_opts_)),
  101. unknown_cf_opts_(
  102. SanitizeOptions(immutable_db_options_, unknown_cf_opts)),
  103. create_unknown_cfs_(create_unknown_cfs),
  104. raw_table_cache_(
  105. // TableCache can be small since we expect each table to be opened
  106. // once.
  107. NewLRUCache(10, db_options_.table_cache_numshardbits)),
  108. table_cache_(new TableCache(default_cf_iopts_, env_options_,
  109. raw_table_cache_.get(),
  110. /*block_cache_tracer=*/nullptr)),
  111. wb_(db_options_.db_write_buffer_size),
  112. wc_(db_options_.delayed_write_rate),
  113. vset_(dbname_, &immutable_db_options_, env_options_,
  114. raw_table_cache_.get(), &wb_, &wc_,
  115. /*block_cache_tracer=*/nullptr),
  116. next_file_number_(1),
  117. db_lock_(nullptr) {
  118. for (const auto& cfd : column_families) {
  119. cf_name_to_opts_[cfd.name] = cfd.options;
  120. }
  121. }
  122. const ColumnFamilyOptions* GetColumnFamilyOptions(
  123. const std::string& cf_name) {
  124. if (cf_name_to_opts_.find(cf_name) == cf_name_to_opts_.end()) {
  125. if (create_unknown_cfs_) {
  126. return &unknown_cf_opts_;
  127. }
  128. return nullptr;
  129. }
  130. return &cf_name_to_opts_[cf_name];
  131. }
  132. // Adds a column family to the VersionSet with cf_options_ and updates
  133. // manifest.
  134. Status AddColumnFamily(const std::string& cf_name, uint32_t cf_id) {
  135. const auto* cf_opts = GetColumnFamilyOptions(cf_name);
  136. if (cf_opts == nullptr) {
  137. return Status::Corruption("Encountered unknown column family with name=" +
  138. cf_name + ", id=" + ToString(cf_id));
  139. }
  140. Options opts(db_options_, *cf_opts);
  141. MutableCFOptions mut_cf_opts(opts);
  142. VersionEdit edit;
  143. edit.SetComparatorName(opts.comparator->Name());
  144. edit.SetLogNumber(0);
  145. edit.SetColumnFamily(cf_id);
  146. ColumnFamilyData* cfd;
  147. cfd = nullptr;
  148. edit.AddColumnFamily(cf_name);
  149. mutex_.Lock();
  150. Status status = vset_.LogAndApply(cfd, mut_cf_opts, &edit, &mutex_,
  151. nullptr /* db_directory */,
  152. false /* new_descriptor_log */, cf_opts);
  153. mutex_.Unlock();
  154. return status;
  155. }
  156. ~Repairer() {
  157. if (db_lock_ != nullptr) {
  158. env_->UnlockFile(db_lock_);
  159. }
  160. delete table_cache_;
  161. }
  162. Status Run() {
  163. Status status = env_->LockFile(LockFileName(dbname_), &db_lock_);
  164. if (!status.ok()) {
  165. return status;
  166. }
  167. status = FindFiles();
  168. if (status.ok()) {
  169. // Discard older manifests and start a fresh one
  170. for (size_t i = 0; i < manifests_.size(); i++) {
  171. ArchiveFile(dbname_ + "/" + manifests_[i]);
  172. }
  173. // Just create a DBImpl temporarily so we can reuse NewDB()
  174. DBImpl* db_impl = new DBImpl(db_options_, dbname_);
  175. status = db_impl->NewDB();
  176. delete db_impl;
  177. }
  178. if (status.ok()) {
  179. // Recover using the fresh manifest created by NewDB()
  180. status =
  181. vset_.Recover({{kDefaultColumnFamilyName, default_cf_opts_}}, false);
  182. }
  183. if (status.ok()) {
  184. // Need to scan existing SST files first so the column families are
  185. // created before we process WAL files
  186. ExtractMetaData();
  187. // ExtractMetaData() uses table_fds_ to know which SST files' metadata to
  188. // extract -- we need to clear it here since metadata for existing SST
  189. // files has been extracted already
  190. table_fds_.clear();
  191. ConvertLogFilesToTables();
  192. ExtractMetaData();
  193. status = AddTables();
  194. }
  195. if (status.ok()) {
  196. uint64_t bytes = 0;
  197. for (size_t i = 0; i < tables_.size(); i++) {
  198. bytes += tables_[i].meta.fd.GetFileSize();
  199. }
  200. ROCKS_LOG_WARN(db_options_.info_log,
  201. "**** Repaired rocksdb %s; "
  202. "recovered %" ROCKSDB_PRIszt " files; %" PRIu64
  203. " bytes. "
  204. "Some data may have been lost. "
  205. "****",
  206. dbname_.c_str(), tables_.size(), bytes);
  207. }
  208. return status;
  209. }
  210. private:
  211. struct TableInfo {
  212. FileMetaData meta;
  213. uint32_t column_family_id;
  214. std::string column_family_name;
  215. };
  216. std::string const dbname_;
  217. Env* const env_;
  218. const EnvOptions env_options_;
  219. const DBOptions db_options_;
  220. const ImmutableDBOptions immutable_db_options_;
  221. const InternalKeyComparator icmp_;
  222. const ColumnFamilyOptions default_cf_opts_;
  223. const ImmutableCFOptions default_cf_iopts_; // table_cache_ holds reference
  224. const ColumnFamilyOptions unknown_cf_opts_;
  225. const bool create_unknown_cfs_;
  226. std::shared_ptr<Cache> raw_table_cache_;
  227. TableCache* table_cache_;
  228. WriteBufferManager wb_;
  229. WriteController wc_;
  230. VersionSet vset_;
  231. std::unordered_map<std::string, ColumnFamilyOptions> cf_name_to_opts_;
  232. InstrumentedMutex mutex_;
  233. std::vector<std::string> manifests_;
  234. std::vector<FileDescriptor> table_fds_;
  235. std::vector<uint64_t> logs_;
  236. std::vector<TableInfo> tables_;
  237. uint64_t next_file_number_;
  238. // Lock over the persistent DB state. Non-nullptr iff successfully
  239. // acquired.
  240. FileLock* db_lock_;
  241. Status FindFiles() {
  242. std::vector<std::string> filenames;
  243. bool found_file = false;
  244. std::vector<std::string> to_search_paths;
  245. for (size_t path_id = 0; path_id < db_options_.db_paths.size(); path_id++) {
  246. to_search_paths.push_back(db_options_.db_paths[path_id].path);
  247. }
  248. // search wal_dir if user uses a customize wal_dir
  249. bool same = false;
  250. Status status = env_->AreFilesSame(db_options_.wal_dir, dbname_, &same);
  251. if (status.IsNotSupported()) {
  252. same = db_options_.wal_dir == dbname_;
  253. status = Status::OK();
  254. } else if (!status.ok()) {
  255. return status;
  256. }
  257. if (!same) {
  258. to_search_paths.push_back(db_options_.wal_dir);
  259. }
  260. for (size_t path_id = 0; path_id < to_search_paths.size(); path_id++) {
  261. status = env_->GetChildren(to_search_paths[path_id], &filenames);
  262. if (!status.ok()) {
  263. return status;
  264. }
  265. if (!filenames.empty()) {
  266. found_file = true;
  267. }
  268. uint64_t number;
  269. FileType type;
  270. for (size_t i = 0; i < filenames.size(); i++) {
  271. if (ParseFileName(filenames[i], &number, &type)) {
  272. if (type == kDescriptorFile) {
  273. manifests_.push_back(filenames[i]);
  274. } else {
  275. if (number + 1 > next_file_number_) {
  276. next_file_number_ = number + 1;
  277. }
  278. if (type == kLogFile) {
  279. logs_.push_back(number);
  280. } else if (type == kTableFile) {
  281. table_fds_.emplace_back(number, static_cast<uint32_t>(path_id),
  282. 0);
  283. } else {
  284. // Ignore other files
  285. }
  286. }
  287. }
  288. }
  289. }
  290. if (!found_file) {
  291. return Status::Corruption(dbname_, "repair found no files");
  292. }
  293. return Status::OK();
  294. }
  295. void ConvertLogFilesToTables() {
  296. for (size_t i = 0; i < logs_.size(); i++) {
  297. // we should use LogFileName(wal_dir, logs_[i]) here. user might uses wal_dir option.
  298. std::string logname = LogFileName(db_options_.wal_dir, logs_[i]);
  299. Status status = ConvertLogToTable(logs_[i]);
  300. if (!status.ok()) {
  301. ROCKS_LOG_WARN(db_options_.info_log,
  302. "Log #%" PRIu64 ": ignoring conversion error: %s",
  303. logs_[i], status.ToString().c_str());
  304. }
  305. ArchiveFile(logname);
  306. }
  307. }
  308. Status ConvertLogToTable(uint64_t log) {
  309. struct LogReporter : public log::Reader::Reporter {
  310. Env* env;
  311. std::shared_ptr<Logger> info_log;
  312. uint64_t lognum;
  313. void Corruption(size_t bytes, const Status& s) override {
  314. // We print error messages for corruption, but continue repairing.
  315. ROCKS_LOG_ERROR(info_log, "Log #%" PRIu64 ": dropping %d bytes; %s",
  316. lognum, static_cast<int>(bytes), s.ToString().c_str());
  317. }
  318. };
  319. // Open the log file
  320. std::string logname = LogFileName(db_options_.wal_dir, log);
  321. std::unique_ptr<SequentialFile> lfile;
  322. Status status = env_->NewSequentialFile(
  323. logname, &lfile, env_->OptimizeForLogRead(env_options_));
  324. if (!status.ok()) {
  325. return status;
  326. }
  327. std::unique_ptr<SequentialFileReader> lfile_reader(new SequentialFileReader(
  328. NewLegacySequentialFileWrapper(lfile), logname));
  329. // Create the log reader.
  330. LogReporter reporter;
  331. reporter.env = env_;
  332. reporter.info_log = db_options_.info_log;
  333. reporter.lognum = log;
  334. // We intentionally make log::Reader do checksumming so that
  335. // corruptions cause entire commits to be skipped instead of
  336. // propagating bad information (like overly large sequence
  337. // numbers).
  338. log::Reader reader(db_options_.info_log, std::move(lfile_reader), &reporter,
  339. true /*enable checksum*/, log);
  340. // Initialize per-column family memtables
  341. for (auto* cfd : *vset_.GetColumnFamilySet()) {
  342. cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
  343. kMaxSequenceNumber);
  344. }
  345. auto cf_mems = new ColumnFamilyMemTablesImpl(vset_.GetColumnFamilySet());
  346. // Read all the records and add to a memtable
  347. std::string scratch;
  348. Slice record;
  349. WriteBatch batch;
  350. int counter = 0;
  351. while (reader.ReadRecord(&record, &scratch)) {
  352. if (record.size() < WriteBatchInternal::kHeader) {
  353. reporter.Corruption(
  354. record.size(), Status::Corruption("log record too small"));
  355. continue;
  356. }
  357. WriteBatchInternal::SetContents(&batch, record);
  358. status =
  359. WriteBatchInternal::InsertInto(&batch, cf_mems, nullptr, nullptr);
  360. if (status.ok()) {
  361. counter += WriteBatchInternal::Count(&batch);
  362. } else {
  363. ROCKS_LOG_WARN(db_options_.info_log, "Log #%" PRIu64 ": ignoring %s",
  364. log, status.ToString().c_str());
  365. status = Status::OK(); // Keep going with rest of file
  366. }
  367. }
  368. // Dump a table for each column family with entries in this log file.
  369. for (auto* cfd : *vset_.GetColumnFamilySet()) {
  370. // Do not record a version edit for this conversion to a Table
  371. // since ExtractMetaData() will also generate edits.
  372. MemTable* mem = cfd->mem();
  373. if (mem->IsEmpty()) {
  374. continue;
  375. }
  376. FileMetaData meta;
  377. meta.fd = FileDescriptor(next_file_number_++, 0, 0);
  378. ReadOptions ro;
  379. ro.total_order_seek = true;
  380. Arena arena;
  381. ScopedArenaIterator iter(mem->NewIterator(ro, &arena));
  382. int64_t _current_time = 0;
  383. status = env_->GetCurrentTime(&_current_time); // ignore error
  384. const uint64_t current_time = static_cast<uint64_t>(_current_time);
  385. SnapshotChecker* snapshot_checker = DisableGCSnapshotChecker::Instance();
  386. auto write_hint = cfd->CalculateSSTWriteHint(0);
  387. std::vector<std::unique_ptr<FragmentedRangeTombstoneIterator>>
  388. range_del_iters;
  389. auto range_del_iter =
  390. mem->NewRangeTombstoneIterator(ro, kMaxSequenceNumber);
  391. if (range_del_iter != nullptr) {
  392. range_del_iters.emplace_back(range_del_iter);
  393. }
  394. LegacyFileSystemWrapper fs(env_);
  395. status = BuildTable(
  396. dbname_, env_, &fs, *cfd->ioptions(),
  397. *cfd->GetLatestMutableCFOptions(), env_options_, table_cache_,
  398. iter.get(), std::move(range_del_iters), &meta,
  399. cfd->internal_comparator(), cfd->int_tbl_prop_collector_factories(),
  400. cfd->GetID(), cfd->GetName(), {}, kMaxSequenceNumber,
  401. snapshot_checker, kNoCompression, 0 /* sample_for_compression */,
  402. CompressionOptions(), false, nullptr /* internal_stats */,
  403. TableFileCreationReason::kRecovery, nullptr /* event_logger */,
  404. 0 /* job_id */, Env::IO_HIGH, nullptr /* table_properties */,
  405. -1 /* level */, current_time, write_hint);
  406. ROCKS_LOG_INFO(db_options_.info_log,
  407. "Log #%" PRIu64 ": %d ops saved to Table #%" PRIu64 " %s",
  408. log, counter, meta.fd.GetNumber(),
  409. status.ToString().c_str());
  410. if (status.ok()) {
  411. if (meta.fd.GetFileSize() > 0) {
  412. table_fds_.push_back(meta.fd);
  413. }
  414. } else {
  415. break;
  416. }
  417. }
  418. delete cf_mems;
  419. return status;
  420. }
  421. void ExtractMetaData() {
  422. for (size_t i = 0; i < table_fds_.size(); i++) {
  423. TableInfo t;
  424. t.meta.fd = table_fds_[i];
  425. Status status = ScanTable(&t);
  426. if (!status.ok()) {
  427. std::string fname = TableFileName(
  428. db_options_.db_paths, t.meta.fd.GetNumber(), t.meta.fd.GetPathId());
  429. char file_num_buf[kFormatFileNumberBufSize];
  430. FormatFileNumber(t.meta.fd.GetNumber(), t.meta.fd.GetPathId(),
  431. file_num_buf, sizeof(file_num_buf));
  432. ROCKS_LOG_WARN(db_options_.info_log, "Table #%s: ignoring %s",
  433. file_num_buf, status.ToString().c_str());
  434. ArchiveFile(fname);
  435. } else {
  436. tables_.push_back(t);
  437. }
  438. }
  439. }
  440. Status ScanTable(TableInfo* t) {
  441. std::string fname = TableFileName(
  442. db_options_.db_paths, t->meta.fd.GetNumber(), t->meta.fd.GetPathId());
  443. int counter = 0;
  444. uint64_t file_size;
  445. Status status = env_->GetFileSize(fname, &file_size);
  446. t->meta.fd = FileDescriptor(t->meta.fd.GetNumber(), t->meta.fd.GetPathId(),
  447. file_size);
  448. std::shared_ptr<const TableProperties> props;
  449. if (status.ok()) {
  450. status = table_cache_->GetTableProperties(env_options_, icmp_, t->meta.fd,
  451. &props);
  452. }
  453. if (status.ok()) {
  454. t->column_family_id = static_cast<uint32_t>(props->column_family_id);
  455. if (t->column_family_id ==
  456. TablePropertiesCollectorFactory::Context::kUnknownColumnFamily) {
  457. ROCKS_LOG_WARN(
  458. db_options_.info_log,
  459. "Table #%" PRIu64
  460. ": column family unknown (probably due to legacy format); "
  461. "adding to default column family id 0.",
  462. t->meta.fd.GetNumber());
  463. t->column_family_id = 0;
  464. }
  465. if (vset_.GetColumnFamilySet()->GetColumnFamily(t->column_family_id) ==
  466. nullptr) {
  467. status =
  468. AddColumnFamily(props->column_family_name, t->column_family_id);
  469. }
  470. t->meta.oldest_ancester_time = props->creation_time;
  471. }
  472. ColumnFamilyData* cfd = nullptr;
  473. if (status.ok()) {
  474. cfd = vset_.GetColumnFamilySet()->GetColumnFamily(t->column_family_id);
  475. if (cfd->GetName() != props->column_family_name) {
  476. ROCKS_LOG_ERROR(
  477. db_options_.info_log,
  478. "Table #%" PRIu64
  479. ": inconsistent column family name '%s'; expected '%s' for column "
  480. "family id %" PRIu32 ".",
  481. t->meta.fd.GetNumber(), props->column_family_name.c_str(),
  482. cfd->GetName().c_str(), t->column_family_id);
  483. status = Status::Corruption(dbname_, "inconsistent column family name");
  484. }
  485. }
  486. if (status.ok()) {
  487. ReadOptions ropts;
  488. ropts.total_order_seek = true;
  489. InternalIterator* iter = table_cache_->NewIterator(
  490. ropts, env_options_, cfd->internal_comparator(), t->meta,
  491. nullptr /* range_del_agg */,
  492. cfd->GetLatestMutableCFOptions()->prefix_extractor.get(),
  493. /*table_reader_ptr=*/nullptr, /*file_read_hist=*/nullptr,
  494. TableReaderCaller::kRepair, /*arena=*/nullptr, /*skip_filters=*/false,
  495. /*level=*/-1, /*smallest_compaction_key=*/nullptr,
  496. /*largest_compaction_key=*/nullptr);
  497. ParsedInternalKey parsed;
  498. for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
  499. Slice key = iter->key();
  500. if (!ParseInternalKey(key, &parsed)) {
  501. ROCKS_LOG_ERROR(db_options_.info_log,
  502. "Table #%" PRIu64 ": unparsable key %s",
  503. t->meta.fd.GetNumber(), EscapeString(key).c_str());
  504. continue;
  505. }
  506. counter++;
  507. t->meta.UpdateBoundaries(key, iter->value(), parsed.sequence,
  508. parsed.type);
  509. }
  510. if (!iter->status().ok()) {
  511. status = iter->status();
  512. }
  513. delete iter;
  514. ROCKS_LOG_INFO(db_options_.info_log, "Table #%" PRIu64 ": %d entries %s",
  515. t->meta.fd.GetNumber(), counter,
  516. status.ToString().c_str());
  517. }
  518. return status;
  519. }
  520. Status AddTables() {
  521. std::unordered_map<uint32_t, std::vector<const TableInfo*>> cf_id_to_tables;
  522. SequenceNumber max_sequence = 0;
  523. for (size_t i = 0; i < tables_.size(); i++) {
  524. cf_id_to_tables[tables_[i].column_family_id].push_back(&tables_[i]);
  525. if (max_sequence < tables_[i].meta.fd.largest_seqno) {
  526. max_sequence = tables_[i].meta.fd.largest_seqno;
  527. }
  528. }
  529. vset_.SetLastAllocatedSequence(max_sequence);
  530. vset_.SetLastPublishedSequence(max_sequence);
  531. vset_.SetLastSequence(max_sequence);
  532. for (const auto& cf_id_and_tables : cf_id_to_tables) {
  533. auto* cfd =
  534. vset_.GetColumnFamilySet()->GetColumnFamily(cf_id_and_tables.first);
  535. VersionEdit edit;
  536. edit.SetComparatorName(cfd->user_comparator()->Name());
  537. edit.SetLogNumber(0);
  538. edit.SetNextFile(next_file_number_);
  539. edit.SetColumnFamily(cfd->GetID());
  540. // TODO(opt): separate out into multiple levels
  541. for (const auto* table : cf_id_and_tables.second) {
  542. edit.AddFile(
  543. 0, table->meta.fd.GetNumber(), table->meta.fd.GetPathId(),
  544. table->meta.fd.GetFileSize(), table->meta.smallest,
  545. table->meta.largest, table->meta.fd.smallest_seqno,
  546. table->meta.fd.largest_seqno, table->meta.marked_for_compaction,
  547. table->meta.oldest_blob_file_number,
  548. table->meta.oldest_ancester_time, table->meta.file_creation_time,
  549. table->meta.file_checksum, table->meta.file_checksum_func_name);
  550. }
  551. assert(next_file_number_ > 0);
  552. vset_.MarkFileNumberUsed(next_file_number_ - 1);
  553. mutex_.Lock();
  554. Status status = vset_.LogAndApply(
  555. cfd, *cfd->GetLatestMutableCFOptions(), &edit, &mutex_,
  556. nullptr /* db_directory */, false /* new_descriptor_log */);
  557. mutex_.Unlock();
  558. if (!status.ok()) {
  559. return status;
  560. }
  561. }
  562. return Status::OK();
  563. }
  564. void ArchiveFile(const std::string& fname) {
  565. // Move into another directory. E.g., for
  566. // dir/foo
  567. // rename to
  568. // dir/lost/foo
  569. const char* slash = strrchr(fname.c_str(), '/');
  570. std::string new_dir;
  571. if (slash != nullptr) {
  572. new_dir.assign(fname.data(), slash - fname.data());
  573. }
  574. new_dir.append("/lost");
  575. env_->CreateDir(new_dir); // Ignore error
  576. std::string new_file = new_dir;
  577. new_file.append("/");
  578. new_file.append((slash == nullptr) ? fname.c_str() : slash + 1);
  579. Status s = env_->RenameFile(fname, new_file);
  580. ROCKS_LOG_INFO(db_options_.info_log, "Archiving %s: %s\n", fname.c_str(),
  581. s.ToString().c_str());
  582. }
  583. };
  584. Status GetDefaultCFOptions(
  585. const std::vector<ColumnFamilyDescriptor>& column_families,
  586. ColumnFamilyOptions* res) {
  587. assert(res != nullptr);
  588. auto iter = std::find_if(column_families.begin(), column_families.end(),
  589. [](const ColumnFamilyDescriptor& cfd) {
  590. return cfd.name == kDefaultColumnFamilyName;
  591. });
  592. if (iter == column_families.end()) {
  593. return Status::InvalidArgument(
  594. "column_families", "Must contain entry for default column family");
  595. }
  596. *res = iter->options;
  597. return Status::OK();
  598. }
  599. } // anonymous namespace
  600. Status RepairDB(const std::string& dbname, const DBOptions& db_options,
  601. const std::vector<ColumnFamilyDescriptor>& column_families
  602. ) {
  603. ColumnFamilyOptions default_cf_opts;
  604. Status status = GetDefaultCFOptions(column_families, &default_cf_opts);
  605. if (status.ok()) {
  606. Repairer repairer(dbname, db_options, column_families,
  607. default_cf_opts,
  608. ColumnFamilyOptions() /* unknown_cf_opts */,
  609. false /* create_unknown_cfs */);
  610. status = repairer.Run();
  611. }
  612. return status;
  613. }
  614. Status RepairDB(const std::string& dbname, const DBOptions& db_options,
  615. const std::vector<ColumnFamilyDescriptor>& column_families,
  616. const ColumnFamilyOptions& unknown_cf_opts) {
  617. ColumnFamilyOptions default_cf_opts;
  618. Status status = GetDefaultCFOptions(column_families, &default_cf_opts);
  619. if (status.ok()) {
  620. Repairer repairer(dbname, db_options,
  621. column_families, default_cf_opts,
  622. unknown_cf_opts, true /* create_unknown_cfs */);
  623. status = repairer.Run();
  624. }
  625. return status;
  626. }
  627. Status RepairDB(const std::string& dbname, const Options& options) {
  628. Options opts(options);
  629. if (opts.file_system == nullptr) {
  630. opts.file_system.reset(new LegacyFileSystemWrapper(opts.env));
  631. ;
  632. }
  633. DBOptions db_options(opts);
  634. ColumnFamilyOptions cf_options(opts);
  635. Repairer repairer(dbname, db_options,
  636. {}, cf_options /* default_cf_opts */,
  637. cf_options /* unknown_cf_opts */,
  638. true /* create_unknown_cfs */);
  639. return repairer.Run();
  640. }
  641. } // namespace ROCKSDB_NAMESPACE
  642. #endif // ROCKSDB_LITE