plain_table_builder.cc 12 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356
  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 "table/plain/plain_table_builder.h"
  6. #include <cassert>
  7. #include <limits>
  8. #include <map>
  9. #include <string>
  10. #include "db/dbformat.h"
  11. #include "file/writable_file_writer.h"
  12. #include "logging/logging.h"
  13. #include "rocksdb/comparator.h"
  14. #include "rocksdb/env.h"
  15. #include "rocksdb/filter_policy.h"
  16. #include "rocksdb/options.h"
  17. #include "rocksdb/table.h"
  18. #include "table/block_based/block_builder.h"
  19. #include "table/format.h"
  20. #include "table/meta_blocks.h"
  21. #include "table/plain/plain_table_bloom.h"
  22. #include "table/plain/plain_table_factory.h"
  23. #include "table/plain/plain_table_index.h"
  24. #include "util/coding.h"
  25. #include "util/crc32c.h"
  26. #include "util/stop_watch.h"
  27. namespace ROCKSDB_NAMESPACE {
  28. namespace {
  29. // a utility that helps writing block content to the file
  30. // @offset will advance if @block_contents was successfully written.
  31. // @block_handle the block handle this particular block.
  32. IOStatus WriteBlock(const Slice& block_contents, WritableFileWriter* file,
  33. uint64_t* offset, BlockHandle* block_handle) {
  34. block_handle->set_offset(*offset);
  35. block_handle->set_size(block_contents.size());
  36. IOStatus io_s = file->Append(IOOptions(), block_contents);
  37. if (io_s.ok()) {
  38. *offset += block_contents.size();
  39. }
  40. return io_s;
  41. }
  42. } // namespace
  43. // kPlainTableMagicNumber was picked by running
  44. // echo rocksdb.table.plain | sha1sum
  45. // and taking the leading 64 bits.
  46. const uint64_t kPlainTableMagicNumber = 0x8242229663bf9564ull;
  47. const uint64_t kLegacyPlainTableMagicNumber = 0x4f3418eb7a8f13b8ull;
  48. PlainTableBuilder::PlainTableBuilder(
  49. const ImmutableOptions& ioptions, const MutableCFOptions& moptions,
  50. const InternalTblPropCollFactories* internal_tbl_prop_coll_factories,
  51. uint32_t column_family_id, int level_at_creation, WritableFileWriter* file,
  52. uint32_t user_key_len, EncodingType encoding_type, size_t index_sparseness,
  53. uint32_t bloom_bits_per_key, const std::string& column_family_name,
  54. uint32_t num_probes, size_t huge_page_tlb_size, double hash_table_ratio,
  55. bool store_index_in_file, const std::string& db_id,
  56. const std::string& db_session_id, uint64_t file_number)
  57. : ioptions_(ioptions),
  58. moptions_(moptions),
  59. bloom_block_(num_probes),
  60. file_(file),
  61. bloom_bits_per_key_(bloom_bits_per_key),
  62. huge_page_tlb_size_(huge_page_tlb_size),
  63. encoder_(encoding_type, user_key_len, moptions.prefix_extractor.get(),
  64. index_sparseness),
  65. store_index_in_file_(store_index_in_file),
  66. prefix_extractor_(moptions.prefix_extractor.get()) {
  67. // Build index block and save it in the file if hash_table_ratio > 0
  68. if (store_index_in_file_) {
  69. assert(hash_table_ratio > 0 || IsTotalOrderMode());
  70. index_builder_.reset(new PlainTableIndexBuilder(
  71. &arena_, ioptions, moptions.prefix_extractor.get(), index_sparseness,
  72. hash_table_ratio, huge_page_tlb_size_));
  73. properties_
  74. .user_collected_properties[PlainTablePropertyNames::kBloomVersion] =
  75. "1"; // For future use
  76. }
  77. properties_.fixed_key_len = user_key_len;
  78. // for plain table, we put all the data in a big chuck.
  79. properties_.num_data_blocks = 1;
  80. // Fill it later if store_index_in_file_ == true
  81. properties_.index_size = 0;
  82. properties_.filter_size = 0;
  83. // To support roll-back to previous version, now still use version 0 for
  84. // plain encoding.
  85. properties_.format_version = (encoding_type == kPlain) ? 0 : 1;
  86. properties_.column_family_id = column_family_id;
  87. properties_.column_family_name = column_family_name;
  88. properties_.db_id = db_id;
  89. properties_.db_session_id = db_session_id;
  90. properties_.db_host_id = ioptions.db_host_id;
  91. if (!ReifyDbHostIdProperty(ioptions_.env, &properties_.db_host_id).ok()) {
  92. ROCKS_LOG_INFO(ioptions_.logger, "db_host_id property will not be set");
  93. }
  94. properties_.orig_file_number = file_number;
  95. properties_.prefix_extractor_name =
  96. moptions_.prefix_extractor != nullptr
  97. ? moptions_.prefix_extractor->AsString()
  98. : "nullptr";
  99. std::string val;
  100. PutFixed32(&val, static_cast<uint32_t>(encoder_.GetEncodingType()));
  101. properties_
  102. .user_collected_properties[PlainTablePropertyNames::kEncodingType] = val;
  103. assert(internal_tbl_prop_coll_factories);
  104. for (auto& factory : *internal_tbl_prop_coll_factories) {
  105. assert(factory);
  106. std::unique_ptr<InternalTblPropColl> collector{
  107. factory->CreateInternalTblPropColl(column_family_id, level_at_creation,
  108. ioptions.num_levels)};
  109. if (collector) {
  110. table_properties_collectors_.emplace_back(std::move(collector));
  111. }
  112. }
  113. }
  114. PlainTableBuilder::~PlainTableBuilder() {
  115. // They are supposed to have been passed to users through Finish()
  116. // if the file succeeds.
  117. status_.PermitUncheckedError();
  118. io_status_.PermitUncheckedError();
  119. }
  120. void PlainTableBuilder::Add(const Slice& key, const Slice& value) {
  121. // temp buffer for metadata bytes between key and value.
  122. char meta_bytes_buf[6];
  123. size_t meta_bytes_buf_size = 0;
  124. const IOOptions opts;
  125. ParsedInternalKey internal_key;
  126. if (!ParseInternalKey(key, &internal_key, false /* log_err_key */)
  127. .ok()) { // TODO
  128. assert(false);
  129. return;
  130. }
  131. if (internal_key.type == kTypeRangeDeletion) {
  132. status_ = Status::NotSupported("Range deletion unsupported");
  133. return;
  134. }
  135. #ifndef NDEBUG
  136. bool skip = false;
  137. TEST_SYNC_POINT_CALLBACK("PlainTableBuilder::Add::skip", (void*)&skip);
  138. if (skip) {
  139. return;
  140. }
  141. #endif // !NDEBUG
  142. // Store key hash
  143. if (store_index_in_file_) {
  144. if (moptions_.prefix_extractor == nullptr) {
  145. keys_or_prefixes_hashes_.push_back(GetSliceHash(internal_key.user_key));
  146. } else {
  147. Slice prefix =
  148. moptions_.prefix_extractor->Transform(internal_key.user_key);
  149. keys_or_prefixes_hashes_.push_back(GetSliceHash(prefix));
  150. }
  151. }
  152. // Write value
  153. assert(offset_ <= std::numeric_limits<uint32_t>::max());
  154. auto prev_offset = static_cast<uint32_t>(offset_);
  155. // Write out the key
  156. io_status_ = encoder_.AppendKey(key, file_, &offset_, meta_bytes_buf,
  157. &meta_bytes_buf_size);
  158. if (SaveIndexInFile()) {
  159. index_builder_->AddKeyPrefix(GetPrefix(internal_key), prev_offset);
  160. }
  161. // Write value length
  162. uint32_t value_size = static_cast<uint32_t>(value.size());
  163. if (io_status_.ok()) {
  164. char* end_ptr =
  165. EncodeVarint32(meta_bytes_buf + meta_bytes_buf_size, value_size);
  166. assert(end_ptr <= meta_bytes_buf + sizeof(meta_bytes_buf));
  167. meta_bytes_buf_size = end_ptr - meta_bytes_buf;
  168. io_status_ =
  169. file_->Append(opts, Slice(meta_bytes_buf, meta_bytes_buf_size));
  170. }
  171. // Write value
  172. if (io_status_.ok()) {
  173. io_status_ = file_->Append(opts, value);
  174. offset_ += value_size + meta_bytes_buf_size;
  175. }
  176. if (io_status_.ok()) {
  177. properties_.num_entries++;
  178. properties_.raw_key_size += key.size();
  179. properties_.raw_value_size += value.size();
  180. if (internal_key.type == kTypeDeletion ||
  181. internal_key.type == kTypeSingleDeletion) {
  182. properties_.num_deletions++;
  183. } else if (internal_key.type == kTypeMerge) {
  184. properties_.num_merge_operands++;
  185. }
  186. }
  187. // notify property collectors
  188. NotifyCollectTableCollectorsOnAdd(
  189. key, value, offset_, table_properties_collectors_, ioptions_.logger);
  190. status_ = io_status_;
  191. }
  192. Status PlainTableBuilder::Finish() {
  193. assert(!closed_);
  194. closed_ = true;
  195. properties_.data_size = offset_;
  196. // Write the following blocks
  197. // 1. [meta block: bloom] - optional
  198. // 2. [meta block: index] - optional
  199. // 3. [meta block: properties]
  200. // 4. [metaindex block]
  201. // 5. [footer]
  202. MetaIndexBuilder meta_index_builer;
  203. if (store_index_in_file_ && (properties_.num_entries > 0)) {
  204. assert(properties_.num_entries <= std::numeric_limits<uint32_t>::max());
  205. BlockHandle bloom_block_handle;
  206. if (bloom_bits_per_key_ > 0) {
  207. bloom_block_.SetTotalBits(
  208. &arena_,
  209. static_cast<uint32_t>(properties_.num_entries) * bloom_bits_per_key_,
  210. ioptions_.bloom_locality, huge_page_tlb_size_, ioptions_.logger);
  211. PutVarint32(&properties_.user_collected_properties
  212. [PlainTablePropertyNames::kNumBloomBlocks],
  213. bloom_block_.GetNumBlocks());
  214. bloom_block_.AddKeysHashes(keys_or_prefixes_hashes_);
  215. Slice bloom_finish_result = bloom_block_.Finish();
  216. properties_.filter_size = bloom_finish_result.size();
  217. io_status_ =
  218. WriteBlock(bloom_finish_result, file_, &offset_, &bloom_block_handle);
  219. if (!io_status_.ok()) {
  220. status_ = io_status_;
  221. return status_;
  222. }
  223. meta_index_builer.Add(BloomBlockBuilder::kBloomBlock, bloom_block_handle);
  224. }
  225. BlockHandle index_block_handle;
  226. Slice index_finish_result = index_builder_->Finish();
  227. properties_.index_size = index_finish_result.size();
  228. io_status_ =
  229. WriteBlock(index_finish_result, file_, &offset_, &index_block_handle);
  230. if (!io_status_.ok()) {
  231. status_ = io_status_;
  232. return status_;
  233. }
  234. meta_index_builer.Add(PlainTableIndexBuilder::kPlainTableIndexBlock,
  235. index_block_handle);
  236. }
  237. // Calculate bloom block size and index block size
  238. PropertyBlockBuilder property_block_builder;
  239. // -- Add basic properties
  240. property_block_builder.AddTableProperty(properties_);
  241. // -- Add eixsting user collected properties
  242. property_block_builder.Add(properties_.user_collected_properties);
  243. // -- Add more user collected properties
  244. UserCollectedProperties more_user_collected_properties;
  245. NotifyCollectTableCollectorsOnFinish(
  246. table_properties_collectors_, ioptions_.logger, &property_block_builder,
  247. more_user_collected_properties, properties_.readable_properties);
  248. properties_.user_collected_properties.insert(
  249. more_user_collected_properties.begin(),
  250. more_user_collected_properties.end());
  251. // -- Write property block
  252. BlockHandle property_block_handle;
  253. io_status_ = WriteBlock(property_block_builder.Finish(), file_, &offset_,
  254. &property_block_handle);
  255. if (!io_status_.ok()) {
  256. status_ = io_status_;
  257. return status_;
  258. }
  259. meta_index_builer.Add(kPropertiesBlockName, property_block_handle);
  260. // -- write metaindex block
  261. BlockHandle metaindex_block_handle;
  262. io_status_ = WriteBlock(meta_index_builer.Finish(), file_, &offset_,
  263. &metaindex_block_handle);
  264. if (!io_status_.ok()) {
  265. status_ = io_status_;
  266. return status_;
  267. }
  268. // Write Footer
  269. // no need to write out new footer if we're using default checksum
  270. FooterBuilder footer;
  271. Status s = footer.Build(kPlainTableMagicNumber, /* format_version */ 0,
  272. offset_, kNoChecksum, metaindex_block_handle);
  273. if (!s.ok()) {
  274. status_ = s;
  275. return status_;
  276. }
  277. io_status_ = file_->Append(IOOptions(), footer.GetSlice());
  278. if (io_status_.ok()) {
  279. offset_ += footer.GetSlice().size();
  280. }
  281. status_ = io_status_;
  282. return status_;
  283. }
  284. void PlainTableBuilder::Abandon() { closed_ = true; }
  285. uint64_t PlainTableBuilder::NumEntries() const {
  286. return properties_.num_entries;
  287. }
  288. uint64_t PlainTableBuilder::FileSize() const { return offset_; }
  289. std::string PlainTableBuilder::GetFileChecksum() const {
  290. if (file_ != nullptr) {
  291. return file_->GetFileChecksum();
  292. } else {
  293. return kUnknownFileChecksum;
  294. }
  295. }
  296. const char* PlainTableBuilder::GetFileChecksumFuncName() const {
  297. if (file_ != nullptr) {
  298. return file_->GetFileChecksumFuncName();
  299. } else {
  300. return kUnknownFileChecksumFuncName;
  301. }
  302. }
  303. void PlainTableBuilder::SetSeqnoTimeTableProperties(
  304. const SeqnoToTimeMapping& relevant_mapping, uint64_t uint_64) {
  305. // TODO: storing seqno to time mapping is not yet support for plain table.
  306. TableBuilder::SetSeqnoTimeTableProperties(relevant_mapping, uint_64);
  307. }
  308. } // namespace ROCKSDB_NAMESPACE