compaction.h 27 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676
  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. #pragma once
  10. #include "db/snapshot_checker.h"
  11. #include "db/version_set.h"
  12. #include "memory/arena.h"
  13. #include "options/cf_options.h"
  14. #include "rocksdb/sst_partitioner.h"
  15. #include "util/autovector.h"
  16. namespace ROCKSDB_NAMESPACE {
  17. // The file contains class Compaction, as well as some helper functions
  18. // and data structures used by the class.
  19. // Utility for comparing sstable boundary keys. Returns -1 if either a or b is
  20. // null which provides the property that a==null indicates a key that is less
  21. // than any key and b==null indicates a key that is greater than any key. Note
  22. // that the comparison is performed primarily on the user-key portion of the
  23. // key. If the user-keys compare equal, an additional test is made to sort
  24. // range tombstone sentinel keys before other keys with the same user-key. The
  25. // result is that 2 user-keys will compare equal if they differ purely on
  26. // their sequence number and value, but the range tombstone sentinel for that
  27. // user-key will compare not equal. This is necessary because the range
  28. // tombstone sentinel key is set as the largest key for an sstable even though
  29. // that key never appears in the database. We don't want adjacent sstables to
  30. // be considered overlapping if they are separated by the range tombstone
  31. // sentinel.
  32. int sstableKeyCompare(const Comparator* user_cmp, const Slice&, const Slice&);
  33. inline int sstableKeyCompare(const Comparator* user_cmp, const Slice& a,
  34. const InternalKey& b) {
  35. return sstableKeyCompare(user_cmp, a, b.Encode());
  36. }
  37. inline int sstableKeyCompare(const Comparator* user_cmp, const InternalKey& a,
  38. const Slice& b) {
  39. return sstableKeyCompare(user_cmp, a.Encode(), b);
  40. }
  41. inline int sstableKeyCompare(const Comparator* user_cmp, const InternalKey& a,
  42. const InternalKey& b) {
  43. return sstableKeyCompare(user_cmp, a.Encode(), b.Encode());
  44. }
  45. int sstableKeyCompare(const Comparator* user_cmp, const InternalKey* a,
  46. const InternalKey& b);
  47. int sstableKeyCompare(const Comparator* user_cmp, const InternalKey& a,
  48. const InternalKey* b);
  49. // An AtomicCompactionUnitBoundary represents a range of keys [smallest,
  50. // largest] that exactly spans one ore more neighbouring SSTs on the same
  51. // level. Every pair of SSTs in this range "overlap" (i.e., the largest
  52. // user key of one file is the smallest user key of the next file). These
  53. // boundaries are propagated down to RangeDelAggregator during compaction
  54. // to provide safe truncation boundaries for range tombstones.
  55. struct AtomicCompactionUnitBoundary {
  56. const InternalKey* smallest = nullptr;
  57. const InternalKey* largest = nullptr;
  58. };
  59. // The structure that manages compaction input files associated
  60. // with the same physical level.
  61. struct CompactionInputFiles {
  62. int level;
  63. std::vector<FileMetaData*> files;
  64. std::vector<AtomicCompactionUnitBoundary> atomic_compaction_unit_boundaries;
  65. inline bool empty() const { return files.empty(); }
  66. inline size_t size() const { return files.size(); }
  67. inline void clear() { files.clear(); }
  68. inline FileMetaData* operator[](size_t i) const { return files[i]; }
  69. };
  70. class Version;
  71. class ColumnFamilyData;
  72. class VersionStorageInfo;
  73. class CompactionFilter;
  74. // A Compaction encapsulates metadata about a compaction.
  75. class Compaction {
  76. public:
  77. Compaction(VersionStorageInfo* input_version,
  78. const ImmutableOptions& immutable_options,
  79. const MutableCFOptions& mutable_cf_options,
  80. const MutableDBOptions& mutable_db_options,
  81. std::vector<CompactionInputFiles> inputs, int output_level,
  82. uint64_t target_file_size, uint64_t max_compaction_bytes,
  83. uint32_t output_path_id, CompressionType compression,
  84. CompressionOptions compression_opts,
  85. Temperature output_temperature_override,
  86. uint32_t max_subcompactions,
  87. std::vector<FileMetaData*> grandparents,
  88. std::optional<SequenceNumber> earliest_snapshot,
  89. const SnapshotChecker* snapshot_checker,
  90. CompactionReason compaction_reason,
  91. const std::string& trim_ts = "", double score = -1,
  92. bool l0_files_might_overlap = true,
  93. BlobGarbageCollectionPolicy blob_garbage_collection_policy =
  94. BlobGarbageCollectionPolicy::kUseDefault,
  95. double blob_garbage_collection_age_cutoff = -1);
  96. // The type of the proximal level output range
  97. enum class ProximalOutputRangeType : int {
  98. kNotSupported, // it cannot output to the proximal level
  99. kFullRange, // any data could be output to the proximal level
  100. kNonLastRange, // only the keys within non_last_level compaction inputs can
  101. // be outputted to the proximal level
  102. kDisabled, // no data can be outputted to the proximal level
  103. };
  104. // No copying allowed
  105. Compaction(const Compaction&) = delete;
  106. void operator=(const Compaction&) = delete;
  107. ~Compaction();
  108. // Returns the level associated to the specified compaction input level.
  109. // If compaction_input_level is not specified, then input_level is set to 0.
  110. int level(size_t compaction_input_level = 0) const {
  111. return inputs_[compaction_input_level].level;
  112. }
  113. int start_level() const { return start_level_; }
  114. // Outputs will go to this level
  115. int output_level() const { return output_level_; }
  116. // Returns the number of input levels in this compaction.
  117. size_t num_input_levels() const { return inputs_.size(); }
  118. // Return the object that holds the edits to the descriptor done
  119. // by this compaction.
  120. VersionEdit* edit() { return &edit_; }
  121. // Returns the number of input files associated to the specified
  122. // compaction input level.
  123. // The function will return 0 if when "compaction_input_level" < 0
  124. // or "compaction_input_level" >= "num_input_levels()".
  125. size_t num_input_files(size_t compaction_input_level) const {
  126. if (compaction_input_level < inputs_.size()) {
  127. return inputs_[compaction_input_level].size();
  128. }
  129. return 0;
  130. }
  131. // Returns input version of the compaction
  132. Version* input_version() const { return input_version_; }
  133. // Returns the ColumnFamilyData associated with the compaction.
  134. ColumnFamilyData* column_family_data() const { return cfd_; }
  135. // Returns the file meta data of the 'i'th input file at the
  136. // specified compaction input level.
  137. // REQUIREMENT: "compaction_input_level" must be >= 0 and
  138. // < "input_levels()"
  139. FileMetaData* input(size_t compaction_input_level, size_t i) const {
  140. assert(compaction_input_level < inputs_.size());
  141. return inputs_[compaction_input_level][i];
  142. }
  143. const std::vector<AtomicCompactionUnitBoundary>* boundaries(
  144. size_t compaction_input_level) const {
  145. assert(compaction_input_level < inputs_.size());
  146. return &inputs_[compaction_input_level].atomic_compaction_unit_boundaries;
  147. }
  148. // Returns the list of file meta data of the specified compaction
  149. // input level.
  150. // REQUIREMENT: "compaction_input_level" must be >= 0 and
  151. // < "input_levels()"
  152. const std::vector<FileMetaData*>* inputs(
  153. size_t compaction_input_level) const {
  154. assert(compaction_input_level < inputs_.size());
  155. return &inputs_[compaction_input_level].files;
  156. }
  157. const std::vector<CompactionInputFiles>* inputs() { return &inputs_; }
  158. // Returns the LevelFilesBrief of the specified compaction input level.
  159. // Note that if the compaction includes standalone range deletion file,
  160. // this function returns the result after filtering out input files covered
  161. // by the range deletion file.
  162. // Use inputs() if you want to get the original input files.
  163. const LevelFilesBrief* input_levels(size_t compaction_input_level) const {
  164. return &input_levels_[compaction_input_level];
  165. }
  166. // Returns the filtered input files of the specified compaction input level.
  167. // For now, only non start level is filtered.
  168. const std::vector<FileMetaData*>& filtered_input_levels(
  169. size_t compaction_input_level) const {
  170. const std::vector<FileMetaData*>& filtered_input_level =
  171. filtered_input_levels_[compaction_input_level];
  172. assert(compaction_input_level != 0 || filtered_input_level.size() == 0);
  173. return filtered_input_level;
  174. }
  175. // Maximum size of files to build during this compaction.
  176. uint64_t max_output_file_size() const { return max_output_file_size_; }
  177. // Target output file size for this compaction
  178. uint64_t target_output_file_size() const { return target_output_file_size_; }
  179. // What compression for output
  180. CompressionType output_compression() const { return output_compression_; }
  181. // What compression options for output
  182. const CompressionOptions& output_compression_opts() const {
  183. return output_compression_opts_;
  184. }
  185. // Whether need to write output file to second DB path.
  186. uint32_t output_path_id() const { return output_path_id_; }
  187. // Is this a trivial compaction that can be implemented by just
  188. // moving a single input file to the next level (no merging or splitting)
  189. bool IsTrivialMove() const;
  190. // The split user key in the output level if this compaction is required to
  191. // split the output files according to the existing cursor in the output
  192. // level under round-robin compaction policy. Empty indicates no required
  193. // splitting key
  194. const InternalKey* GetOutputSplitKey() const { return output_split_key_; }
  195. // If true, then the compaction can be done by simply deleting input files.
  196. bool deletion_compaction() const { return deletion_compaction_; }
  197. // Add all inputs to this compaction as delete operations to *edit.
  198. void AddInputDeletions(VersionEdit* edit);
  199. // Returns true if the available information we have guarantees that
  200. // the input "user_key" does not exist in any level beyond `output_level()`.
  201. bool KeyNotExistsBeyondOutputLevel(const Slice& user_key,
  202. std::vector<size_t>* level_ptrs) const;
  203. // Returns true if the user key range [begin_key, end_key) does not exist
  204. // in any level beyond `output_level()`.
  205. // Used for checking range tombstones, so we assume begin_key < end_key.
  206. // begin_key and end_key should include timestamp if enabled.
  207. bool KeyRangeNotExistsBeyondOutputLevel(
  208. const Slice& begin_key, const Slice& end_key,
  209. std::vector<size_t>* level_ptrs) const;
  210. // Clear all files to indicate that they are not being compacted
  211. // Delete this compaction from the list of running compactions.
  212. //
  213. // Requirement: DB mutex held
  214. void ReleaseCompactionFiles(const Status& status);
  215. // Returns the summary of the compaction in "output" with maximum "len"
  216. // in bytes. The caller is responsible for the memory management of
  217. // "output".
  218. void Summary(char* output, int len);
  219. // Return the score that was used to pick this compaction run.
  220. double score() const { return score_; }
  221. // Is this compaction creating a file in the bottom most level?
  222. bool bottommost_level() const { return bottommost_level_; }
  223. // Is the compaction compact to the last level
  224. bool is_last_level() const {
  225. return output_level_ == immutable_options_.num_levels - 1;
  226. }
  227. // Does this compaction include all sst files?
  228. bool is_full_compaction() const { return is_full_compaction_; }
  229. // Was this compaction triggered manually by the client?
  230. bool is_manual_compaction() const { return is_manual_compaction_; }
  231. std::string trim_ts() const { return trim_ts_; }
  232. // Used when allow_trivial_move option is set in
  233. // Universal compaction. If all the input files are
  234. // non overlapping, then is_trivial_move_ variable
  235. // will be set true, else false
  236. void set_is_trivial_move(bool trivial_move) {
  237. is_trivial_move_ = trivial_move;
  238. }
  239. // Used when allow_trivial_move option is set in
  240. // Universal compaction. Returns true, if the input files
  241. // are non-overlapping and can be trivially moved.
  242. bool is_trivial_move() const { return is_trivial_move_; }
  243. bool is_trivial_copy_compaction() const {
  244. return immutable_options_.compaction_style == kCompactionStyleFIFO &&
  245. compaction_reason_ == CompactionReason::kChangeTemperature &&
  246. mutable_cf_options_.compaction_options_fifo
  247. .allow_trivial_copy_when_change_temperature;
  248. }
  249. // How many total levels are there?
  250. int number_levels() const { return number_levels_; }
  251. // Return the ImmutableOptions that should be used throughout the compaction
  252. // procedure
  253. const ImmutableOptions& immutable_options() const {
  254. return immutable_options_;
  255. }
  256. // Return the MutableCFOptions that should be used throughout the compaction
  257. // procedure
  258. const MutableCFOptions& mutable_cf_options() const {
  259. return mutable_cf_options_;
  260. }
  261. // Returns the size in bytes that the output file should be preallocated to.
  262. // In level compaction, that is max_file_size_. In universal compaction, that
  263. // is the sum of all input file sizes.
  264. uint64_t OutputFilePreallocationSize() const;
  265. void FinalizeInputInfo(Version* input_version);
  266. struct InputLevelSummaryBuffer {
  267. char buffer[128];
  268. };
  269. const char* InputLevelSummary(InputLevelSummaryBuffer* scratch) const;
  270. uint64_t CalculateTotalInputSize() const;
  271. // In case of compaction error, reset the nextIndex that is used
  272. // to pick up the next file to be compacted from files_by_size_
  273. void ResetNextCompactionIndex();
  274. // Create a CompactionFilter from compaction_filter_factory
  275. std::unique_ptr<CompactionFilter> CreateCompactionFilter() const;
  276. // Create a SstPartitioner from sst_partitioner_factory
  277. std::unique_ptr<SstPartitioner> CreateSstPartitioner() const;
  278. // Is the input level corresponding to output_level_ empty?
  279. bool IsOutputLevelEmpty() const;
  280. // Should this compaction be broken up into smaller ones run in parallel?
  281. bool ShouldFormSubcompactions() const;
  282. // Returns true iff at least one input file references a blob file.
  283. //
  284. // PRE: input version has been set.
  285. bool DoesInputReferenceBlobFiles() const;
  286. // test function to validate the functionality of IsBottommostLevel()
  287. // function -- determines if compaction with inputs and storage is bottommost
  288. static bool TEST_IsBottommostLevel(
  289. int output_level, VersionStorageInfo* vstorage,
  290. const std::vector<CompactionInputFiles>& inputs);
  291. // TODO(hx235): eventually we should consider `InitInputTableProperties()`'s
  292. // status and fail the compaction if needed
  293. //
  294. // May open and read table files for table property.
  295. // Should not be called while holding mutex_.
  296. const TablePropertiesCollection& GetOrInitInputTableProperties() {
  297. InitInputTableProperties().PermitUncheckedError();
  298. return input_table_properties_;
  299. }
  300. const TablePropertiesCollection& GetInputTableProperties() const {
  301. return input_table_properties_;
  302. }
  303. // TODO(hx235): consider making this function symmetric to
  304. // InitInputTableProperties()
  305. void SetOutputTableProperties(
  306. const std::string& file_name,
  307. const std::shared_ptr<const TableProperties>& tp) {
  308. output_table_properties_[file_name] = tp;
  309. }
  310. const TablePropertiesCollection& GetOutputTableProperties() const {
  311. return output_table_properties_;
  312. }
  313. Slice GetSmallestUserKey() const { return smallest_user_key_; }
  314. Slice GetLargestUserKey() const { return largest_user_key_; }
  315. ProximalOutputRangeType GetProximalOutputRangeType() const {
  316. return proximal_output_range_type_;
  317. }
  318. // Return true if the compaction supports per_key_placement
  319. bool SupportsPerKeyPlacement() const;
  320. // Get per_key_placement proximal output level, which is `last_level - 1`
  321. // if per_key_placement feature is supported. Otherwise, return -1.
  322. int GetProximalLevel() const;
  323. // Return true if the given range is overlap with proximal level output
  324. // range.
  325. // Both smallest_key and largest_key include timestamps if user-defined
  326. // timestamp is enabled.
  327. bool OverlapProximalLevelOutputRange(const Slice& smallest_key,
  328. const Slice& largest_key) const;
  329. // For testing purposes, check that a key is within proximal level
  330. // output range for per_key_placement feature, which is safe to place the key
  331. // to the proximal level. Different compaction strategies have different
  332. // rules. `user_key` includes timestamp if user-defined timestamp is enabled.
  333. void TEST_AssertWithinProximalLevelOutputRange(
  334. const Slice& user_key, bool expect_failure = false) const;
  335. CompactionReason compaction_reason() const { return compaction_reason_; }
  336. const std::vector<FileMetaData*>& grandparents() const {
  337. return grandparents_;
  338. }
  339. uint64_t max_compaction_bytes() const { return max_compaction_bytes_; }
  340. // Order of precedence for temperature:
  341. // 1. Override temp if not kUnknown
  342. // 2. Temperature of the last level files if applicable
  343. // 3. Default write temperature
  344. Temperature GetOutputTemperature(bool is_proximal_level = false) const;
  345. uint32_t max_subcompactions() const { return max_subcompactions_; }
  346. bool enable_blob_garbage_collection() const {
  347. return enable_blob_garbage_collection_;
  348. }
  349. double blob_garbage_collection_age_cutoff() const {
  350. return blob_garbage_collection_age_cutoff_;
  351. }
  352. // start and end are sub compact range. Null if no boundary.
  353. // This is used to calculate the newest_key_time table property after
  354. // compaction.
  355. uint64_t MaxInputFileNewestKeyTime(const InternalKey* start,
  356. const InternalKey* end) const;
  357. // start and end are sub compact range. Null if no boundary.
  358. // This is used to filter out some input files' ancester's time range.
  359. uint64_t MinInputFileOldestAncesterTime(const InternalKey* start,
  360. const InternalKey* end) const;
  361. // Return the minimum epoch number among
  362. // input files' associated with this compaction
  363. uint64_t MinInputFileEpochNumber() const;
  364. // Called by DBImpl::NotifyOnCompactionCompleted to make sure number of
  365. // compaction begin and compaction completion callbacks match.
  366. void SetNotifyOnCompactionCompleted() {
  367. notify_on_compaction_completion_ = true;
  368. }
  369. bool ShouldNotifyOnCompactionCompleted() const {
  370. return notify_on_compaction_completion_;
  371. }
  372. static constexpr int kInvalidLevel = -1;
  373. // Evaluate proximal output level. If the compaction supports
  374. // per_key_placement feature, it returns the proximal level number.
  375. // Otherwise, it's set to kInvalidLevel (-1), which means
  376. // output_to_proximal_level is not supported.
  377. // Note: even the proximal level output is supported (ProximalLevel !=
  378. // kInvalidLevel), some key range maybe unsafe to be outputted to the
  379. // proximal level. The safe key range is populated by
  380. // `PopulateProximalLevelOutputRange()`.
  381. // Which could potentially disable all proximal level output.
  382. static int EvaluateProximalLevel(const VersionStorageInfo* vstorage,
  383. const MutableCFOptions& mutable_cf_options,
  384. const ImmutableOptions& immutable_options,
  385. const int start_level,
  386. const int output_level);
  387. static bool OutputToNonZeroMaxOutputLevel(int output_level,
  388. int max_output_level) {
  389. return output_level > 0 && output_level == max_output_level;
  390. }
  391. // If some data cannot be safely migrated "up" the LSM tree due to a change
  392. // in the preclude_last_level_data_seconds setting, this indicates a sequence
  393. // number for the newest data that must be kept in the last level.
  394. SequenceNumber GetKeepInLastLevelThroughSeqno() const {
  395. return keep_in_last_level_through_seqno_;
  396. }
  397. // mark (or clear) all files that are being compacted
  398. void MarkFilesBeingCompacted(bool being_compacted) const;
  399. private:
  400. Status InitInputTableProperties();
  401. // get the smallest and largest key present in files to be compacted
  402. static void GetBoundaryKeys(VersionStorageInfo* vstorage,
  403. const std::vector<CompactionInputFiles>& inputs,
  404. Slice* smallest_key, Slice* largest_key,
  405. int exclude_level = -1);
  406. // get the smallest and largest internal key present in files to be compacted
  407. static void GetBoundaryInternalKeys(
  408. VersionStorageInfo* vstorage,
  409. const std::vector<CompactionInputFiles>& inputs,
  410. InternalKey* smallest_key, InternalKey* largest_key,
  411. int exclude_level = -1);
  412. // populate proximal level output range, which will be used to determine if
  413. // a key is safe to output to the proximal level (details see
  414. // `Compaction::WithinProximalLevelOutputRange()`.
  415. void PopulateProximalLevelOutputRange();
  416. // If oldest snapshot is specified at Compaction construction time, we have
  417. // an opportunity to optimize inputs for compaction iterator for this case:
  418. // When a standalone range deletion file on the start level is recognized and
  419. // can be determined to completely shadow some input files on non-start level.
  420. // These files will be filtered out and later not feed to compaction iterator.
  421. void FilterInputsForCompactionIterator();
  422. // Get the atomic file boundaries for all files in the compaction. Necessary
  423. // in order to avoid the scenario described in
  424. // https://github.com/facebook/rocksdb/pull/4432#discussion_r221072219 and
  425. // plumb down appropriate key boundaries to RangeDelAggregator during
  426. // compaction.
  427. static std::vector<CompactionInputFiles> PopulateWithAtomicBoundaries(
  428. VersionStorageInfo* vstorage, std::vector<CompactionInputFiles> inputs);
  429. // helper function to determine if compaction with inputs and storage is
  430. // bottommost
  431. static bool IsBottommostLevel(
  432. int output_level, VersionStorageInfo* vstorage,
  433. const std::vector<CompactionInputFiles>& inputs);
  434. static bool IsFullCompaction(VersionStorageInfo* vstorage,
  435. const std::vector<CompactionInputFiles>& inputs);
  436. VersionStorageInfo* input_vstorage_;
  437. const int start_level_; // the lowest level to be compacted
  438. const int output_level_; // levels to which output files are stored
  439. uint64_t target_output_file_size_;
  440. uint64_t max_output_file_size_;
  441. uint64_t max_compaction_bytes_;
  442. uint32_t max_subcompactions_;
  443. const ImmutableOptions immutable_options_;
  444. const MutableCFOptions mutable_cf_options_;
  445. Version* input_version_;
  446. VersionEdit edit_;
  447. const int number_levels_;
  448. ColumnFamilyData* cfd_;
  449. Arena arena_; // Arena used to allocate space for file_levels_
  450. const uint32_t output_path_id_;
  451. CompressionType output_compression_;
  452. CompressionOptions output_compression_opts_;
  453. Temperature output_temperature_override_;
  454. // If true, then the compaction can be done by simply deleting input files.
  455. const bool deletion_compaction_;
  456. // should it split the output file using the compact cursor?
  457. const InternalKey* output_split_key_;
  458. // L0 files in LSM-tree might be overlapping. But the compaction picking
  459. // logic might pick a subset of the files that aren't overlapping. if
  460. // that is the case, set the value to false. Otherwise, set it true.
  461. bool l0_files_might_overlap_;
  462. // Compaction input files organized by level. Constant after construction
  463. const std::vector<CompactionInputFiles> inputs_;
  464. // All files from inputs_ that are not filtered and will be fed to compaction
  465. // iterator, organized more closely in memory.
  466. autovector<LevelFilesBrief, 2> input_levels_;
  467. // State used to check for number of overlapping grandparent files
  468. // (grandparent == "output_level_ + 1")
  469. std::vector<FileMetaData*> grandparents_;
  470. // The earliest snapshot and snapshot checker at compaction picking time.
  471. // These fields are only set for deletion triggered compactions picked in
  472. // universal compaction. And when user-defined timestamp is not enabled.
  473. // It will be used to possibly filter out some non start level input files.
  474. std::optional<SequenceNumber> earliest_snapshot_;
  475. const SnapshotChecker* snapshot_checker_;
  476. // Markers for which non start level input files are filtered out if
  477. // applicable. Only applicable if earliest_snapshot_ is provided and input
  478. // start level has a standalone range deletion file. Filtered files are
  479. // tracked in `filtered_input_levels_`.
  480. std::vector<std::vector<bool>> non_start_level_input_files_filtered_;
  481. // All files from inputs_ that are filtered.
  482. std::vector<std::vector<FileMetaData*>> filtered_input_levels_;
  483. const double score_; // score that was used to pick this compaction.
  484. // Is this compaction creating a file in the bottom most level?
  485. const bool bottommost_level_;
  486. // Does this compaction include all sst files?
  487. const bool is_full_compaction_;
  488. // Is this compaction requested by the client?
  489. const bool is_manual_compaction_;
  490. // The data with timestamp > trim_ts_ will be removed
  491. const std::string trim_ts_;
  492. // True if we can do trivial move in Universal multi level
  493. // compaction
  494. bool is_trivial_move_;
  495. // Does input compression match the output compression?
  496. bool InputCompressionMatchesOutput() const;
  497. TablePropertiesCollection input_table_properties_;
  498. TablePropertiesCollection output_table_properties_;
  499. // smallest user keys in compaction
  500. // includes timestamp if user-defined timestamp is enabled.
  501. Slice smallest_user_key_;
  502. // largest user keys in compaction
  503. // includes timestamp if user-defined timestamp is enabled.
  504. Slice largest_user_key_;
  505. // Reason for compaction
  506. CompactionReason compaction_reason_;
  507. // Notify on compaction completion only if listener was notified on compaction
  508. // begin.
  509. bool notify_on_compaction_completion_;
  510. // Enable/disable GC collection for blobs during compaction.
  511. bool enable_blob_garbage_collection_;
  512. // Blob garbage collection age cutoff.
  513. double blob_garbage_collection_age_cutoff_;
  514. SequenceNumber keep_in_last_level_through_seqno_ = kMaxSequenceNumber;
  515. // only set when per_key_placement feature is enabled, -1 (kInvalidLevel)
  516. // means not supported.
  517. const int proximal_level_;
  518. // Key range for proximal level output
  519. // includes timestamp if user-defined timestamp is enabled.
  520. // proximal_output_range_type_ shows the range type
  521. InternalKey proximal_level_smallest_;
  522. InternalKey proximal_level_largest_;
  523. ProximalOutputRangeType proximal_output_range_type_ =
  524. ProximalOutputRangeType::kNotSupported;
  525. };
  526. #ifndef NDEBUG
  527. // Helper struct only for tests, which contains the data to decide if a key
  528. // should be output to the proximal level.
  529. // TODO: remove this when the public feature knob is available
  530. struct PerKeyPlacementContext {
  531. const int level;
  532. const Slice key;
  533. const Slice value;
  534. const SequenceNumber seq_num;
  535. bool& output_to_proximal_level;
  536. PerKeyPlacementContext(int _level, Slice _key, Slice _value,
  537. SequenceNumber _seq_num,
  538. bool& _output_to_proximal_level)
  539. : level(_level),
  540. key(_key),
  541. value(_value),
  542. seq_num(_seq_num),
  543. output_to_proximal_level(_output_to_proximal_level) {}
  544. };
  545. #endif /* !NDEBUG */
  546. // Return sum of sizes of all files in `files`.
  547. uint64_t TotalFileSize(const std::vector<FileMetaData*>& files);
  548. } // namespace ROCKSDB_NAMESPACE