sst_file_writer.cc 21 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544
  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. #include "rocksdb/sst_file_writer.h"
  6. #include <vector>
  7. #include "db/db_impl/db_impl.h"
  8. #include "db/dbformat.h"
  9. #include "db/wide/wide_column_serialization.h"
  10. #include "db/wide/wide_columns_helper.h"
  11. #include "file/writable_file_writer.h"
  12. #include "rocksdb/file_system.h"
  13. #include "rocksdb/table.h"
  14. #include "table/block_based/block_based_table_builder.h"
  15. #include "table/sst_file_writer_collectors.h"
  16. #include "test_util/sync_point.h"
  17. namespace ROCKSDB_NAMESPACE {
  18. const std::string ExternalSstFilePropertyNames::kVersion =
  19. "rocksdb.external_sst_file.version";
  20. const std::string ExternalSstFilePropertyNames::kGlobalSeqno =
  21. "rocksdb.external_sst_file.global_seqno";
  22. const size_t kFadviseTrigger = 1024 * 1024; // 1MB
  23. struct SstFileWriter::Rep {
  24. Rep(const EnvOptions& _env_options, const Options& options,
  25. Env::IOPriority _io_priority, const Comparator* _user_comparator,
  26. ColumnFamilyHandle* _cfh, bool _invalidate_page_cache, bool _skip_filters,
  27. std::string _db_session_id)
  28. : env_options(_env_options),
  29. ioptions(options),
  30. mutable_cf_options(options),
  31. io_priority(_io_priority),
  32. internal_comparator(_user_comparator),
  33. cfh(_cfh),
  34. invalidate_page_cache(_invalidate_page_cache),
  35. skip_filters(_skip_filters),
  36. db_session_id(_db_session_id),
  37. ts_sz(_user_comparator->timestamp_size()),
  38. strip_timestamp(ts_sz > 0 &&
  39. !ioptions.persist_user_defined_timestamps) {
  40. // TODO (hx235): pass in `WriteOptions` instead of `rate_limiter_priority`
  41. // during construction
  42. write_options.rate_limiter_priority = io_priority;
  43. }
  44. std::unique_ptr<WritableFileWriter> file_writer;
  45. std::unique_ptr<TableBuilder> builder;
  46. EnvOptions env_options;
  47. ImmutableOptions ioptions;
  48. MutableCFOptions mutable_cf_options;
  49. Env::IOPriority io_priority;
  50. WriteOptions write_options;
  51. InternalKeyComparator internal_comparator;
  52. ExternalSstFileInfo file_info;
  53. InternalKey ikey;
  54. std::string column_family_name;
  55. ColumnFamilyHandle* cfh;
  56. // If true, We will give the OS a hint that this file pages is not needed
  57. // every time we write 1MB to the file.
  58. bool invalidate_page_cache;
  59. // The size of the file during the last time we called Fadvise to remove
  60. // cached pages from page cache.
  61. uint64_t last_fadvise_size = 0;
  62. bool skip_filters;
  63. std::string db_session_id;
  64. uint64_t next_file_number = 1;
  65. size_t ts_sz;
  66. bool strip_timestamp;
  67. Status AddImpl(const Slice& user_key, const Slice& value,
  68. ValueType value_type) {
  69. if (!builder) {
  70. return Status::InvalidArgument("File is not opened");
  71. }
  72. if (!builder->status().ok()) {
  73. return builder->status();
  74. }
  75. assert(user_key.size() >= ts_sz);
  76. if (strip_timestamp) {
  77. // In this mode, we expect users to always provide a min timestamp.
  78. if (internal_comparator.user_comparator()->CompareTimestamp(
  79. Slice(user_key.data() + user_key.size() - ts_sz, ts_sz),
  80. MinU64Ts()) != 0) {
  81. return Status::InvalidArgument(
  82. "persist_user_defined_timestamps flag is set to false, only "
  83. "minimum timestamp is accepted.");
  84. }
  85. }
  86. if (file_info.num_entries == 0) {
  87. file_info.smallest_key.assign(user_key.data(), user_key.size());
  88. } else {
  89. if (internal_comparator.user_comparator()->Compare(
  90. user_key, file_info.largest_key) <= 0) {
  91. // Make sure that keys are added in order
  92. return Status::InvalidArgument(
  93. "Keys must be added in strict ascending order.");
  94. }
  95. }
  96. assert(value_type == kTypeValue || value_type == kTypeMerge ||
  97. value_type == kTypeDeletion ||
  98. value_type == kTypeDeletionWithTimestamp ||
  99. value_type == kTypeWideColumnEntity);
  100. constexpr SequenceNumber sequence_number = 0;
  101. ikey.Set(user_key, sequence_number, value_type);
  102. builder->Add(ikey.Encode(), value);
  103. // update file info
  104. file_info.num_entries++;
  105. file_info.largest_key.assign(user_key.data(), user_key.size());
  106. file_info.file_size = builder->FileSize();
  107. InvalidatePageCache(false /* closing */).PermitUncheckedError();
  108. return builder->status();
  109. }
  110. Status Add(const Slice& user_key, const Slice& value, ValueType value_type) {
  111. if (internal_comparator.user_comparator()->timestamp_size() != 0) {
  112. return Status::InvalidArgument("Timestamp size mismatch");
  113. }
  114. return AddImpl(user_key, value, value_type);
  115. }
  116. Status Add(const Slice& user_key, const Slice& timestamp, const Slice& value,
  117. ValueType value_type) {
  118. const size_t timestamp_size = timestamp.size();
  119. if (internal_comparator.user_comparator()->timestamp_size() !=
  120. timestamp_size) {
  121. return Status::InvalidArgument("Timestamp size mismatch");
  122. }
  123. const size_t user_key_size = user_key.size();
  124. if (user_key.data() + user_key_size == timestamp.data()) {
  125. Slice user_key_with_ts(user_key.data(), user_key_size + timestamp_size);
  126. return AddImpl(user_key_with_ts, value, value_type);
  127. }
  128. std::string user_key_with_ts;
  129. user_key_with_ts.reserve(user_key_size + timestamp_size);
  130. user_key_with_ts.append(user_key.data(), user_key_size);
  131. user_key_with_ts.append(timestamp.data(), timestamp_size);
  132. return AddImpl(user_key_with_ts, value, value_type);
  133. }
  134. Status AddEntity(const Slice& user_key, const WideColumns& columns) {
  135. WideColumns sorted_columns(columns);
  136. WideColumnsHelper::SortColumns(sorted_columns);
  137. std::string entity;
  138. const Status s = WideColumnSerialization::Serialize(sorted_columns, entity);
  139. if (!s.ok()) {
  140. return s;
  141. }
  142. if (entity.size() > size_t{std::numeric_limits<uint32_t>::max()}) {
  143. return Status::InvalidArgument("wide column entity is too large");
  144. }
  145. return Add(user_key, entity, kTypeWideColumnEntity);
  146. }
  147. Status DeleteRangeImpl(const Slice& begin_key, const Slice& end_key) {
  148. if (!builder) {
  149. return Status::InvalidArgument("File is not opened");
  150. }
  151. int cmp = internal_comparator.user_comparator()->CompareWithoutTimestamp(
  152. begin_key, end_key);
  153. if (cmp > 0) {
  154. // It's an empty range where endpoints appear mistaken. Don't bother
  155. // applying it to the DB, and return an error to the user.
  156. return Status::InvalidArgument("end key comes before start key");
  157. } else if (cmp == 0) {
  158. // It's an empty range. Don't bother applying it to the DB.
  159. return Status::OK();
  160. }
  161. assert(begin_key.size() >= ts_sz);
  162. assert(end_key.size() >= ts_sz);
  163. Slice begin_key_ts =
  164. Slice(begin_key.data() + begin_key.size() - ts_sz, ts_sz);
  165. Slice end_key_ts = Slice(end_key.data() + end_key.size() - ts_sz, ts_sz);
  166. assert(begin_key_ts.compare(end_key_ts) == 0);
  167. if (strip_timestamp) {
  168. // In this mode, we expect users to always provide a min timestamp.
  169. if (internal_comparator.user_comparator()->CompareTimestamp(
  170. begin_key_ts, MinU64Ts()) != 0) {
  171. return Status::InvalidArgument(
  172. "persist_user_defined_timestamps flag is set to false, only "
  173. "minimum timestamp is accepted for start key.");
  174. }
  175. if (internal_comparator.user_comparator()->CompareTimestamp(
  176. end_key_ts, MinU64Ts()) != 0) {
  177. return Status::InvalidArgument(
  178. "persist_user_defined_timestamps flag is set to false, only "
  179. "minimum timestamp is accepted for end key.");
  180. }
  181. }
  182. RangeTombstone tombstone(begin_key, end_key, 0 /* Sequence Number */);
  183. if (file_info.num_range_del_entries == 0) {
  184. file_info.smallest_range_del_key.assign(tombstone.start_key_.data(),
  185. tombstone.start_key_.size());
  186. file_info.largest_range_del_key.assign(tombstone.end_key_.data(),
  187. tombstone.end_key_.size());
  188. } else {
  189. if (internal_comparator.user_comparator()->Compare(
  190. tombstone.start_key_, file_info.smallest_range_del_key) < 0) {
  191. file_info.smallest_range_del_key.assign(tombstone.start_key_.data(),
  192. tombstone.start_key_.size());
  193. }
  194. if (internal_comparator.user_comparator()->Compare(
  195. tombstone.end_key_, file_info.largest_range_del_key) > 0) {
  196. file_info.largest_range_del_key.assign(tombstone.end_key_.data(),
  197. tombstone.end_key_.size());
  198. }
  199. }
  200. auto ikey_and_end_key = tombstone.Serialize();
  201. builder->Add(ikey_and_end_key.first.Encode(), ikey_and_end_key.second);
  202. // update file info
  203. file_info.num_range_del_entries++;
  204. file_info.file_size = builder->FileSize();
  205. InvalidatePageCache(false /* closing */).PermitUncheckedError();
  206. return Status::OK();
  207. }
  208. Status DeleteRange(const Slice& begin_key, const Slice& end_key) {
  209. if (internal_comparator.user_comparator()->timestamp_size() != 0) {
  210. return Status::InvalidArgument("Timestamp size mismatch");
  211. }
  212. return DeleteRangeImpl(begin_key, end_key);
  213. }
  214. // begin_key and end_key should be users keys without timestamp.
  215. Status DeleteRange(const Slice& begin_key, const Slice& end_key,
  216. const Slice& timestamp) {
  217. const size_t timestamp_size = timestamp.size();
  218. if (internal_comparator.user_comparator()->timestamp_size() !=
  219. timestamp_size) {
  220. return Status::InvalidArgument("Timestamp size mismatch");
  221. }
  222. const size_t begin_key_size = begin_key.size();
  223. const size_t end_key_size = end_key.size();
  224. if (begin_key.data() + begin_key_size == timestamp.data() ||
  225. end_key.data() + begin_key_size == timestamp.data()) {
  226. assert(memcmp(begin_key.data() + begin_key_size,
  227. end_key.data() + end_key_size, timestamp_size) == 0);
  228. Slice begin_key_with_ts(begin_key.data(),
  229. begin_key_size + timestamp_size);
  230. Slice end_key_with_ts(end_key.data(), end_key.size() + timestamp_size);
  231. return DeleteRangeImpl(begin_key_with_ts, end_key_with_ts);
  232. }
  233. std::string begin_key_with_ts;
  234. begin_key_with_ts.reserve(begin_key_size + timestamp_size);
  235. begin_key_with_ts.append(begin_key.data(), begin_key_size);
  236. begin_key_with_ts.append(timestamp.data(), timestamp_size);
  237. std::string end_key_with_ts;
  238. end_key_with_ts.reserve(end_key_size + timestamp_size);
  239. end_key_with_ts.append(end_key.data(), end_key_size);
  240. end_key_with_ts.append(timestamp.data(), timestamp_size);
  241. return DeleteRangeImpl(begin_key_with_ts, end_key_with_ts);
  242. }
  243. Status InvalidatePageCache(bool closing) {
  244. Status s = Status::OK();
  245. if (invalidate_page_cache == false) {
  246. // Fadvise disabled
  247. return s;
  248. }
  249. uint64_t bytes_since_last_fadvise = builder->FileSize() - last_fadvise_size;
  250. if (bytes_since_last_fadvise > kFadviseTrigger || closing) {
  251. TEST_SYNC_POINT_CALLBACK("SstFileWriter::Rep::InvalidatePageCache",
  252. &(bytes_since_last_fadvise));
  253. // Tell the OS that we don't need this file in page cache
  254. s = file_writer->InvalidateCache(0, 0);
  255. if (s.IsNotSupported()) {
  256. // NotSupported is fine as it could be a file type that doesn't use page
  257. // cache.
  258. s = Status::OK();
  259. }
  260. last_fadvise_size = builder->FileSize();
  261. }
  262. return s;
  263. }
  264. };
  265. SstFileWriter::SstFileWriter(const EnvOptions& env_options,
  266. const Options& options,
  267. const Comparator* user_comparator,
  268. ColumnFamilyHandle* column_family,
  269. bool invalidate_page_cache,
  270. Env::IOPriority io_priority, bool skip_filters)
  271. : rep_(new Rep(env_options, options, io_priority, user_comparator,
  272. column_family, invalidate_page_cache, skip_filters,
  273. DBImpl::GenerateDbSessionId(options.env))) {
  274. // SstFileWriter is used to create sst files that can be added to database
  275. // later. Therefore, no real db_id and db_session_id are associated with it.
  276. // Here we mimic the way db_session_id behaves by getting a db_session_id
  277. // for each SstFileWriter, and (later below) assign unique file numbers
  278. // in the table properties. The db_id is set to be "SST Writer" for clarity.
  279. rep_->file_info.file_size = 0;
  280. }
  281. SstFileWriter::~SstFileWriter() {
  282. if (rep_->builder) {
  283. // User did not call Finish() or Finish() failed, we need to
  284. // abandon the builder.
  285. rep_->builder->Abandon();
  286. }
  287. }
  288. Status SstFileWriter::Open(const std::string& file_path, Temperature temp) {
  289. Rep* r = rep_.get();
  290. Status s;
  291. std::unique_ptr<FSWritableFile> sst_file;
  292. FileOptions cur_file_opts(r->env_options);
  293. cur_file_opts.temperature = temp;
  294. s = r->ioptions.env->GetFileSystem()->NewWritableFile(
  295. file_path, cur_file_opts, &sst_file, nullptr);
  296. if (!s.ok()) {
  297. return s;
  298. }
  299. sst_file->SetIOPriority(r->io_priority);
  300. CompressionType compression_type;
  301. CompressionOptions compression_opts;
  302. if (r->mutable_cf_options.bottommost_compression !=
  303. kDisableCompressionOption) {
  304. compression_type = r->mutable_cf_options.bottommost_compression;
  305. if (r->mutable_cf_options.bottommost_compression_opts.enabled) {
  306. compression_opts = r->mutable_cf_options.bottommost_compression_opts;
  307. } else {
  308. compression_opts = r->mutable_cf_options.compression_opts;
  309. }
  310. } else if (!r->mutable_cf_options.compression_per_level.empty()) {
  311. // Use the compression of the last level if we have per level compression
  312. compression_type = *(r->mutable_cf_options.compression_per_level.rbegin());
  313. compression_opts = r->mutable_cf_options.compression_opts;
  314. } else {
  315. compression_type = r->mutable_cf_options.compression;
  316. compression_opts = r->mutable_cf_options.compression_opts;
  317. }
  318. InternalTblPropCollFactories internal_tbl_prop_coll_factories;
  319. // SstFileWriter properties collector to add SstFileWriter version.
  320. internal_tbl_prop_coll_factories.emplace_back(
  321. new SstFileWriterPropertiesCollectorFactory(2 /* version */,
  322. 0 /* global_seqno*/));
  323. // User collector factories
  324. auto user_collector_factories =
  325. r->ioptions.table_properties_collector_factories;
  326. for (size_t i = 0; i < user_collector_factories.size(); i++) {
  327. internal_tbl_prop_coll_factories.emplace_back(
  328. new UserKeyTablePropertiesCollectorFactory(
  329. user_collector_factories[i]));
  330. }
  331. int unknown_level = -1;
  332. uint32_t cf_id;
  333. if (r->cfh != nullptr) {
  334. // user explicitly specified that this file will be ingested into cfh,
  335. // we can persist this information in the file.
  336. cf_id = r->cfh->GetID();
  337. r->column_family_name = r->cfh->GetName();
  338. } else {
  339. r->column_family_name = "";
  340. cf_id = TablePropertiesCollectorFactory::Context::kUnknownColumnFamily;
  341. }
  342. // TODO: it would be better to set oldest_key_time to be used for getting the
  343. // approximate time of ingested keys.
  344. // TODO: plumb Env::IOActivity, Env::IOPriority
  345. TableBuilderOptions table_builder_options(
  346. r->ioptions, r->mutable_cf_options, ReadOptions(), r->write_options,
  347. r->internal_comparator, &internal_tbl_prop_coll_factories,
  348. compression_type, compression_opts, cf_id, r->column_family_name,
  349. unknown_level, kUnknownNewestKeyTime, false /* is_bottommost */,
  350. TableFileCreationReason::kMisc, 0 /* oldest_key_time */,
  351. 0 /* file_creation_time */, "SST Writer" /* db_id */, r->db_session_id,
  352. 0 /* target_file_size */, r->next_file_number);
  353. // External SST files used to each get a unique session id. Now for
  354. // slightly better uniqueness probability in constructing cache keys, we
  355. // assign fake file numbers to each file (into table properties) and keep
  356. // the same session id for the life of the SstFileWriter.
  357. r->next_file_number++;
  358. // XXX: when we can remove skip_filters from the SstFileWriter public API
  359. // we can remove it from TableBuilderOptions.
  360. table_builder_options.skip_filters = r->skip_filters;
  361. FileTypeSet tmp_set = r->ioptions.checksum_handoff_file_types;
  362. r->file_writer.reset(new WritableFileWriter(
  363. std::move(sst_file), file_path, r->env_options, r->ioptions.clock,
  364. nullptr /* io_tracer */, r->ioptions.stats, Histograms::SST_WRITE_MICROS,
  365. r->ioptions.listeners, r->ioptions.file_checksum_gen_factory.get(),
  366. tmp_set.Contains(FileType::kTableFile), false));
  367. // TODO(tec) : If table_factory is using compressed block cache, we will
  368. // be adding the external sst file blocks into it, which is wasteful.
  369. r->builder.reset(r->mutable_cf_options.table_factory->NewTableBuilder(
  370. table_builder_options, r->file_writer.get()));
  371. r->file_info = ExternalSstFileInfo();
  372. r->file_info.file_path = file_path;
  373. r->file_info.version = 2;
  374. return s;
  375. }
  376. Status SstFileWriter::Add(const Slice& user_key, const Slice& value) {
  377. return rep_->Add(user_key, value, ValueType::kTypeValue);
  378. }
  379. Status SstFileWriter::Put(const Slice& user_key, const Slice& value) {
  380. return rep_->Add(user_key, value, ValueType::kTypeValue);
  381. }
  382. Status SstFileWriter::Put(const Slice& user_key, const Slice& timestamp,
  383. const Slice& value) {
  384. return rep_->Add(user_key, timestamp, value, ValueType::kTypeValue);
  385. }
  386. Status SstFileWriter::PutEntity(const Slice& user_key,
  387. const WideColumns& columns) {
  388. return rep_->AddEntity(user_key, columns);
  389. }
  390. Status SstFileWriter::Merge(const Slice& user_key, const Slice& value) {
  391. return rep_->Add(user_key, value, ValueType::kTypeMerge);
  392. }
  393. Status SstFileWriter::Delete(const Slice& user_key) {
  394. return rep_->Add(user_key, Slice(), ValueType::kTypeDeletion);
  395. }
  396. Status SstFileWriter::Delete(const Slice& user_key, const Slice& timestamp) {
  397. return rep_->Add(user_key, timestamp, Slice(),
  398. ValueType::kTypeDeletionWithTimestamp);
  399. }
  400. Status SstFileWriter::DeleteRange(const Slice& begin_key,
  401. const Slice& end_key) {
  402. return rep_->DeleteRange(begin_key, end_key);
  403. }
  404. Status SstFileWriter::DeleteRange(const Slice& begin_key, const Slice& end_key,
  405. const Slice& timestamp) {
  406. return rep_->DeleteRange(begin_key, end_key, timestamp);
  407. }
  408. Status SstFileWriter::Finish(ExternalSstFileInfo* file_info) {
  409. Rep* r = rep_.get();
  410. if (!r->builder) {
  411. return Status::InvalidArgument("File is not opened");
  412. }
  413. if (r->file_info.num_entries == 0 &&
  414. r->file_info.num_range_del_entries == 0) {
  415. r->builder->status().PermitUncheckedError();
  416. return Status::InvalidArgument("Cannot create sst file with no entries");
  417. }
  418. Status s = r->builder->Finish();
  419. r->file_info.file_size = r->builder->FileSize();
  420. IOOptions opts;
  421. if (s.ok()) {
  422. s = WritableFileWriter::PrepareIOOptions(r->write_options, opts);
  423. }
  424. if (s.ok()) {
  425. s = r->file_writer->Sync(opts, r->ioptions.use_fsync);
  426. r->InvalidatePageCache(true /* closing */).PermitUncheckedError();
  427. if (s.ok()) {
  428. s = r->file_writer->Close(opts);
  429. }
  430. }
  431. if (s.ok()) {
  432. r->file_info.file_checksum = r->file_writer->GetFileChecksum();
  433. r->file_info.file_checksum_func_name =
  434. r->file_writer->GetFileChecksumFuncName();
  435. }
  436. if (!s.ok()) {
  437. Status status = r->ioptions.env->DeleteFile(r->file_info.file_path);
  438. // Silence ASSERT_STATUS_CHECKED warning, since DeleteFile may fail under
  439. // some error injection, and we can just ignore the failure
  440. status.PermitUncheckedError();
  441. }
  442. if (file_info != nullptr) {
  443. *file_info = r->file_info;
  444. Slice smallest_key = r->file_info.smallest_key;
  445. Slice largest_key = r->file_info.largest_key;
  446. Slice smallest_range_del_key = r->file_info.smallest_range_del_key;
  447. Slice largest_range_del_key = r->file_info.largest_range_del_key;
  448. assert(smallest_key.empty() == largest_key.empty());
  449. assert(smallest_range_del_key.empty() == largest_range_del_key.empty());
  450. // Remove user-defined timestamps from external file metadata too when they
  451. // should not be persisted.
  452. if (r->strip_timestamp) {
  453. if (!smallest_key.empty()) {
  454. assert(smallest_key.size() >= r->ts_sz);
  455. assert(largest_key.size() >= r->ts_sz);
  456. file_info->smallest_key.resize(smallest_key.size() - r->ts_sz);
  457. file_info->largest_key.resize(largest_key.size() - r->ts_sz);
  458. }
  459. if (!smallest_range_del_key.empty()) {
  460. assert(smallest_range_del_key.size() >= r->ts_sz);
  461. assert(largest_range_del_key.size() >= r->ts_sz);
  462. file_info->smallest_range_del_key.resize(smallest_range_del_key.size() -
  463. r->ts_sz);
  464. file_info->largest_range_del_key.resize(largest_range_del_key.size() -
  465. r->ts_sz);
  466. }
  467. }
  468. }
  469. r->builder.reset();
  470. return s;
  471. }
  472. uint64_t SstFileWriter::FileSize() { return rep_->file_info.file_size; }
  473. bool SstFileWriter::CreatedBySstFileWriter(const TableProperties& tp) {
  474. const auto& uprops = tp.user_collected_properties;
  475. return uprops.find(ExternalSstFilePropertyNames::kVersion) != uprops.end();
  476. }
  477. } // namespace ROCKSDB_NAMESPACE