table_properties_collector_test.cc 19 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515
  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 <map>
  6. #include <memory>
  7. #include <string>
  8. #include <utility>
  9. #include <vector>
  10. #include "db/db_impl/db_impl.h"
  11. #include "db/dbformat.h"
  12. #include "db/table_properties_collector.h"
  13. #include "env/composite_env_wrapper.h"
  14. #include "file/sequence_file_reader.h"
  15. #include "file/writable_file_writer.h"
  16. #include "options/cf_options.h"
  17. #include "rocksdb/table.h"
  18. #include "table/block_based/block_based_table_factory.h"
  19. #include "table/meta_blocks.h"
  20. #include "table/plain/plain_table_factory.h"
  21. #include "table/table_builder.h"
  22. #include "test_util/testharness.h"
  23. #include "test_util/testutil.h"
  24. #include "util/coding.h"
  25. namespace ROCKSDB_NAMESPACE {
  26. class TablePropertiesTest : public testing::Test,
  27. public testing::WithParamInterface<bool> {
  28. public:
  29. void SetUp() override { backward_mode_ = GetParam(); }
  30. bool backward_mode_;
  31. };
  32. // Utilities test functions
  33. namespace {
  34. static const uint32_t kTestColumnFamilyId = 66;
  35. static const std::string kTestColumnFamilyName = "test_column_fam";
  36. void MakeBuilder(const Options& options, const ImmutableCFOptions& ioptions,
  37. const MutableCFOptions& moptions,
  38. const InternalKeyComparator& internal_comparator,
  39. const std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
  40. int_tbl_prop_collector_factories,
  41. std::unique_ptr<WritableFileWriter>* writable,
  42. std::unique_ptr<TableBuilder>* builder) {
  43. std::unique_ptr<WritableFile> wf(new test::StringSink);
  44. writable->reset(
  45. new WritableFileWriter(NewLegacyWritableFileWrapper(std::move(wf)),
  46. "" /* don't care */, EnvOptions()));
  47. int unknown_level = -1;
  48. builder->reset(NewTableBuilder(
  49. ioptions, moptions, internal_comparator, int_tbl_prop_collector_factories,
  50. kTestColumnFamilyId, kTestColumnFamilyName, writable->get(),
  51. options.compression, options.sample_for_compression,
  52. options.compression_opts, unknown_level));
  53. }
  54. } // namespace
  55. // Collects keys that starts with "A" in a table.
  56. class RegularKeysStartWithA: public TablePropertiesCollector {
  57. public:
  58. const char* Name() const override { return "RegularKeysStartWithA"; }
  59. Status Finish(UserCollectedProperties* properties) override {
  60. std::string encoded;
  61. std::string encoded_num_puts;
  62. std::string encoded_num_deletes;
  63. std::string encoded_num_single_deletes;
  64. std::string encoded_num_size_changes;
  65. PutVarint32(&encoded, count_);
  66. PutVarint32(&encoded_num_puts, num_puts_);
  67. PutVarint32(&encoded_num_deletes, num_deletes_);
  68. PutVarint32(&encoded_num_single_deletes, num_single_deletes_);
  69. PutVarint32(&encoded_num_size_changes, num_size_changes_);
  70. *properties = UserCollectedProperties{
  71. {"TablePropertiesTest", message_},
  72. {"Count", encoded},
  73. {"NumPuts", encoded_num_puts},
  74. {"NumDeletes", encoded_num_deletes},
  75. {"NumSingleDeletes", encoded_num_single_deletes},
  76. {"NumSizeChanges", encoded_num_size_changes},
  77. };
  78. return Status::OK();
  79. }
  80. Status AddUserKey(const Slice& user_key, const Slice& /*value*/,
  81. EntryType type, SequenceNumber /*seq*/,
  82. uint64_t file_size) override {
  83. // simply asssume all user keys are not empty.
  84. if (user_key.data()[0] == 'A') {
  85. ++count_;
  86. }
  87. if (type == kEntryPut) {
  88. num_puts_++;
  89. } else if (type == kEntryDelete) {
  90. num_deletes_++;
  91. } else if (type == kEntrySingleDelete) {
  92. num_single_deletes_++;
  93. }
  94. if (file_size < file_size_) {
  95. message_ = "File size should not decrease.";
  96. } else if (file_size != file_size_) {
  97. num_size_changes_++;
  98. }
  99. return Status::OK();
  100. }
  101. UserCollectedProperties GetReadableProperties() const override {
  102. return UserCollectedProperties{};
  103. }
  104. private:
  105. std::string message_ = "Rocksdb";
  106. uint32_t count_ = 0;
  107. uint32_t num_puts_ = 0;
  108. uint32_t num_deletes_ = 0;
  109. uint32_t num_single_deletes_ = 0;
  110. uint32_t num_size_changes_ = 0;
  111. uint64_t file_size_ = 0;
  112. };
  113. // Collects keys that starts with "A" in a table. Backward compatible mode
  114. // It is also used to test internal key table property collector
  115. class RegularKeysStartWithABackwardCompatible
  116. : public TablePropertiesCollector {
  117. public:
  118. const char* Name() const override { return "RegularKeysStartWithA"; }
  119. Status Finish(UserCollectedProperties* properties) override {
  120. std::string encoded;
  121. PutVarint32(&encoded, count_);
  122. *properties = UserCollectedProperties{{"TablePropertiesTest", "Rocksdb"},
  123. {"Count", encoded}};
  124. return Status::OK();
  125. }
  126. Status Add(const Slice& user_key, const Slice& /*value*/) override {
  127. // simply asssume all user keys are not empty.
  128. if (user_key.data()[0] == 'A') {
  129. ++count_;
  130. }
  131. return Status::OK();
  132. }
  133. UserCollectedProperties GetReadableProperties() const override {
  134. return UserCollectedProperties{};
  135. }
  136. private:
  137. uint32_t count_ = 0;
  138. };
  139. class RegularKeysStartWithAInternal : public IntTblPropCollector {
  140. public:
  141. const char* Name() const override { return "RegularKeysStartWithA"; }
  142. Status Finish(UserCollectedProperties* properties) override {
  143. std::string encoded;
  144. PutVarint32(&encoded, count_);
  145. *properties = UserCollectedProperties{{"TablePropertiesTest", "Rocksdb"},
  146. {"Count", encoded}};
  147. return Status::OK();
  148. }
  149. Status InternalAdd(const Slice& user_key, const Slice& /*value*/,
  150. uint64_t /*file_size*/) override {
  151. // simply asssume all user keys are not empty.
  152. if (user_key.data()[0] == 'A') {
  153. ++count_;
  154. }
  155. return Status::OK();
  156. }
  157. void BlockAdd(uint64_t /* blockRawBytes */,
  158. uint64_t /* blockCompressedBytesFast */,
  159. uint64_t /* blockCompressedBytesSlow */) override {
  160. // Nothing to do.
  161. return;
  162. }
  163. UserCollectedProperties GetReadableProperties() const override {
  164. return UserCollectedProperties{};
  165. }
  166. private:
  167. uint32_t count_ = 0;
  168. };
  169. class RegularKeysStartWithAFactory : public IntTblPropCollectorFactory,
  170. public TablePropertiesCollectorFactory {
  171. public:
  172. explicit RegularKeysStartWithAFactory(bool backward_mode)
  173. : backward_mode_(backward_mode) {}
  174. TablePropertiesCollector* CreateTablePropertiesCollector(
  175. TablePropertiesCollectorFactory::Context context) override {
  176. EXPECT_EQ(kTestColumnFamilyId, context.column_family_id);
  177. if (!backward_mode_) {
  178. return new RegularKeysStartWithA();
  179. } else {
  180. return new RegularKeysStartWithABackwardCompatible();
  181. }
  182. }
  183. IntTblPropCollector* CreateIntTblPropCollector(
  184. uint32_t /*column_family_id*/) override {
  185. return new RegularKeysStartWithAInternal();
  186. }
  187. const char* Name() const override { return "RegularKeysStartWithA"; }
  188. bool backward_mode_;
  189. };
  190. class FlushBlockEveryThreePolicy : public FlushBlockPolicy {
  191. public:
  192. bool Update(const Slice& /*key*/, const Slice& /*value*/) override {
  193. return (++count_ % 3U == 0);
  194. }
  195. private:
  196. uint64_t count_ = 0;
  197. };
  198. class FlushBlockEveryThreePolicyFactory : public FlushBlockPolicyFactory {
  199. public:
  200. explicit FlushBlockEveryThreePolicyFactory() {}
  201. const char* Name() const override {
  202. return "FlushBlockEveryThreePolicyFactory";
  203. }
  204. FlushBlockPolicy* NewFlushBlockPolicy(
  205. const BlockBasedTableOptions& /*table_options*/,
  206. const BlockBuilder& /*data_block_builder*/) const override {
  207. return new FlushBlockEveryThreePolicy;
  208. }
  209. };
  210. extern const uint64_t kBlockBasedTableMagicNumber;
  211. extern const uint64_t kPlainTableMagicNumber;
  212. namespace {
  213. void TestCustomizedTablePropertiesCollector(
  214. bool backward_mode, uint64_t magic_number, bool test_int_tbl_prop_collector,
  215. const Options& options, const InternalKeyComparator& internal_comparator) {
  216. // make sure the entries will be inserted with order.
  217. std::map<std::pair<std::string, ValueType>, std::string> kvs = {
  218. {{"About ", kTypeValue}, "val5"}, // starts with 'A'
  219. {{"Abstract", kTypeValue}, "val2"}, // starts with 'A'
  220. {{"Around ", kTypeValue}, "val7"}, // starts with 'A'
  221. {{"Beyond ", kTypeValue}, "val3"},
  222. {{"Builder ", kTypeValue}, "val1"},
  223. {{"Love ", kTypeDeletion}, ""},
  224. {{"Cancel ", kTypeValue}, "val4"},
  225. {{"Find ", kTypeValue}, "val6"},
  226. {{"Rocks ", kTypeDeletion}, ""},
  227. {{"Foo ", kTypeSingleDeletion}, ""},
  228. };
  229. // -- Step 1: build table
  230. std::unique_ptr<TableBuilder> builder;
  231. std::unique_ptr<WritableFileWriter> writer;
  232. const ImmutableCFOptions ioptions(options);
  233. const MutableCFOptions moptions(options);
  234. std::vector<std::unique_ptr<IntTblPropCollectorFactory>>
  235. int_tbl_prop_collector_factories;
  236. if (test_int_tbl_prop_collector) {
  237. int_tbl_prop_collector_factories.emplace_back(
  238. new RegularKeysStartWithAFactory(backward_mode));
  239. } else {
  240. GetIntTblPropCollectorFactory(ioptions, &int_tbl_prop_collector_factories);
  241. }
  242. MakeBuilder(options, ioptions, moptions, internal_comparator,
  243. &int_tbl_prop_collector_factories, &writer, &builder);
  244. SequenceNumber seqNum = 0U;
  245. for (const auto& kv : kvs) {
  246. InternalKey ikey(kv.first.first, seqNum++, kv.first.second);
  247. builder->Add(ikey.Encode(), kv.second);
  248. }
  249. ASSERT_OK(builder->Finish());
  250. writer->Flush();
  251. // -- Step 2: Read properties
  252. LegacyWritableFileWrapper* file =
  253. static_cast<LegacyWritableFileWrapper*>(writer->writable_file());
  254. test::StringSink* fwf = static_cast<test::StringSink*>(file->target());
  255. std::unique_ptr<RandomAccessFileReader> fake_file_reader(
  256. test::GetRandomAccessFileReader(
  257. new test::StringSource(fwf->contents())));
  258. TableProperties* props;
  259. Status s = ReadTableProperties(fake_file_reader.get(), fwf->contents().size(),
  260. magic_number, ioptions, &props,
  261. true /* compression_type_missing */);
  262. std::unique_ptr<TableProperties> props_guard(props);
  263. ASSERT_OK(s);
  264. auto user_collected = props->user_collected_properties;
  265. ASSERT_NE(user_collected.find("TablePropertiesTest"), user_collected.end());
  266. ASSERT_EQ("Rocksdb", user_collected.at("TablePropertiesTest"));
  267. uint32_t starts_with_A = 0;
  268. ASSERT_NE(user_collected.find("Count"), user_collected.end());
  269. Slice key(user_collected.at("Count"));
  270. ASSERT_TRUE(GetVarint32(&key, &starts_with_A));
  271. ASSERT_EQ(3u, starts_with_A);
  272. if (!backward_mode && !test_int_tbl_prop_collector) {
  273. uint32_t num_puts;
  274. ASSERT_NE(user_collected.find("NumPuts"), user_collected.end());
  275. Slice key_puts(user_collected.at("NumPuts"));
  276. ASSERT_TRUE(GetVarint32(&key_puts, &num_puts));
  277. ASSERT_EQ(7u, num_puts);
  278. uint32_t num_deletes;
  279. ASSERT_NE(user_collected.find("NumDeletes"), user_collected.end());
  280. Slice key_deletes(user_collected.at("NumDeletes"));
  281. ASSERT_TRUE(GetVarint32(&key_deletes, &num_deletes));
  282. ASSERT_EQ(2u, num_deletes);
  283. uint32_t num_single_deletes;
  284. ASSERT_NE(user_collected.find("NumSingleDeletes"), user_collected.end());
  285. Slice key_single_deletes(user_collected.at("NumSingleDeletes"));
  286. ASSERT_TRUE(GetVarint32(&key_single_deletes, &num_single_deletes));
  287. ASSERT_EQ(1u, num_single_deletes);
  288. uint32_t num_size_changes;
  289. ASSERT_NE(user_collected.find("NumSizeChanges"), user_collected.end());
  290. Slice key_size_changes(user_collected.at("NumSizeChanges"));
  291. ASSERT_TRUE(GetVarint32(&key_size_changes, &num_size_changes));
  292. ASSERT_GE(num_size_changes, 2u);
  293. }
  294. }
  295. } // namespace
  296. TEST_P(TablePropertiesTest, CustomizedTablePropertiesCollector) {
  297. // Test properties collectors with internal keys or regular keys
  298. // for block based table
  299. for (bool encode_as_internal : { true, false }) {
  300. Options options;
  301. BlockBasedTableOptions table_options;
  302. table_options.flush_block_policy_factory =
  303. std::make_shared<FlushBlockEveryThreePolicyFactory>();
  304. options.table_factory.reset(NewBlockBasedTableFactory(table_options));
  305. test::PlainInternalKeyComparator ikc(options.comparator);
  306. std::shared_ptr<TablePropertiesCollectorFactory> collector_factory(
  307. new RegularKeysStartWithAFactory(backward_mode_));
  308. options.table_properties_collector_factories.resize(1);
  309. options.table_properties_collector_factories[0] = collector_factory;
  310. TestCustomizedTablePropertiesCollector(backward_mode_,
  311. kBlockBasedTableMagicNumber,
  312. encode_as_internal, options, ikc);
  313. #ifndef ROCKSDB_LITE // PlainTable is not supported in Lite
  314. // test plain table
  315. PlainTableOptions plain_table_options;
  316. plain_table_options.user_key_len = 8;
  317. plain_table_options.bloom_bits_per_key = 8;
  318. plain_table_options.hash_table_ratio = 0;
  319. options.table_factory =
  320. std::make_shared<PlainTableFactory>(plain_table_options);
  321. TestCustomizedTablePropertiesCollector(backward_mode_,
  322. kPlainTableMagicNumber,
  323. encode_as_internal, options, ikc);
  324. #endif // !ROCKSDB_LITE
  325. }
  326. }
  327. namespace {
  328. void TestInternalKeyPropertiesCollector(
  329. bool backward_mode, uint64_t magic_number, bool sanitized,
  330. std::shared_ptr<TableFactory> table_factory) {
  331. InternalKey keys[] = {
  332. InternalKey("A ", 0, ValueType::kTypeValue),
  333. InternalKey("B ", 1, ValueType::kTypeValue),
  334. InternalKey("C ", 2, ValueType::kTypeValue),
  335. InternalKey("W ", 3, ValueType::kTypeDeletion),
  336. InternalKey("X ", 4, ValueType::kTypeDeletion),
  337. InternalKey("Y ", 5, ValueType::kTypeDeletion),
  338. InternalKey("Z ", 6, ValueType::kTypeDeletion),
  339. InternalKey("a ", 7, ValueType::kTypeSingleDeletion),
  340. InternalKey("b ", 8, ValueType::kTypeMerge),
  341. InternalKey("c ", 9, ValueType::kTypeMerge),
  342. };
  343. std::unique_ptr<TableBuilder> builder;
  344. std::unique_ptr<WritableFileWriter> writable;
  345. Options options;
  346. test::PlainInternalKeyComparator pikc(options.comparator);
  347. std::vector<std::unique_ptr<IntTblPropCollectorFactory>>
  348. int_tbl_prop_collector_factories;
  349. options.table_factory = table_factory;
  350. if (sanitized) {
  351. options.table_properties_collector_factories.emplace_back(
  352. new RegularKeysStartWithAFactory(backward_mode));
  353. // with sanitization, even regular properties collector will be able to
  354. // handle internal keys.
  355. auto comparator = options.comparator;
  356. // HACK: Set options.info_log to avoid writing log in
  357. // SanitizeOptions().
  358. options.info_log = std::make_shared<test::NullLogger>();
  359. options = SanitizeOptions("db", // just a place holder
  360. options);
  361. ImmutableCFOptions ioptions(options);
  362. GetIntTblPropCollectorFactory(ioptions, &int_tbl_prop_collector_factories);
  363. options.comparator = comparator;
  364. }
  365. const ImmutableCFOptions ioptions(options);
  366. MutableCFOptions moptions(options);
  367. for (int iter = 0; iter < 2; ++iter) {
  368. MakeBuilder(options, ioptions, moptions, pikc,
  369. &int_tbl_prop_collector_factories, &writable, &builder);
  370. for (const auto& k : keys) {
  371. builder->Add(k.Encode(), "val");
  372. }
  373. ASSERT_OK(builder->Finish());
  374. writable->Flush();
  375. LegacyWritableFileWrapper* file =
  376. static_cast<LegacyWritableFileWrapper*>(writable->writable_file());
  377. test::StringSink* fwf = static_cast<test::StringSink*>(file->target());
  378. std::unique_ptr<RandomAccessFileReader> reader(
  379. test::GetRandomAccessFileReader(
  380. new test::StringSource(fwf->contents())));
  381. TableProperties* props;
  382. Status s =
  383. ReadTableProperties(reader.get(), fwf->contents().size(), magic_number,
  384. ioptions, &props, true /* compression_type_missing */);
  385. ASSERT_OK(s);
  386. std::unique_ptr<TableProperties> props_guard(props);
  387. auto user_collected = props->user_collected_properties;
  388. uint64_t deleted = GetDeletedKeys(user_collected);
  389. ASSERT_EQ(5u, deleted); // deletes + single-deletes
  390. bool property_present;
  391. uint64_t merges = GetMergeOperands(user_collected, &property_present);
  392. ASSERT_TRUE(property_present);
  393. ASSERT_EQ(2u, merges);
  394. if (sanitized) {
  395. uint32_t starts_with_A = 0;
  396. ASSERT_NE(user_collected.find("Count"), user_collected.end());
  397. Slice key(user_collected.at("Count"));
  398. ASSERT_TRUE(GetVarint32(&key, &starts_with_A));
  399. ASSERT_EQ(1u, starts_with_A);
  400. if (!backward_mode) {
  401. uint32_t num_puts;
  402. ASSERT_NE(user_collected.find("NumPuts"), user_collected.end());
  403. Slice key_puts(user_collected.at("NumPuts"));
  404. ASSERT_TRUE(GetVarint32(&key_puts, &num_puts));
  405. ASSERT_EQ(3u, num_puts);
  406. uint32_t num_deletes;
  407. ASSERT_NE(user_collected.find("NumDeletes"), user_collected.end());
  408. Slice key_deletes(user_collected.at("NumDeletes"));
  409. ASSERT_TRUE(GetVarint32(&key_deletes, &num_deletes));
  410. ASSERT_EQ(4u, num_deletes);
  411. uint32_t num_single_deletes;
  412. ASSERT_NE(user_collected.find("NumSingleDeletes"),
  413. user_collected.end());
  414. Slice key_single_deletes(user_collected.at("NumSingleDeletes"));
  415. ASSERT_TRUE(GetVarint32(&key_single_deletes, &num_single_deletes));
  416. ASSERT_EQ(1u, num_single_deletes);
  417. }
  418. }
  419. }
  420. }
  421. } // namespace
  422. TEST_P(TablePropertiesTest, InternalKeyPropertiesCollector) {
  423. TestInternalKeyPropertiesCollector(
  424. backward_mode_, kBlockBasedTableMagicNumber, true /* sanitize */,
  425. std::make_shared<BlockBasedTableFactory>());
  426. if (backward_mode_) {
  427. TestInternalKeyPropertiesCollector(
  428. backward_mode_, kBlockBasedTableMagicNumber, false /* not sanitize */,
  429. std::make_shared<BlockBasedTableFactory>());
  430. }
  431. #ifndef ROCKSDB_LITE // PlainTable is not supported in Lite
  432. PlainTableOptions plain_table_options;
  433. plain_table_options.user_key_len = 8;
  434. plain_table_options.bloom_bits_per_key = 8;
  435. plain_table_options.hash_table_ratio = 0;
  436. TestInternalKeyPropertiesCollector(
  437. backward_mode_, kPlainTableMagicNumber, false /* not sanitize */,
  438. std::make_shared<PlainTableFactory>(plain_table_options));
  439. #endif // !ROCKSDB_LITE
  440. }
  441. INSTANTIATE_TEST_CASE_P(InternalKeyPropertiesCollector, TablePropertiesTest,
  442. ::testing::Bool());
  443. INSTANTIATE_TEST_CASE_P(CustomizedTablePropertiesCollector, TablePropertiesTest,
  444. ::testing::Bool());
  445. } // namespace ROCKSDB_NAMESPACE
  446. int main(int argc, char** argv) {
  447. ::testing::InitGoogleTest(&argc, argv);
  448. return RUN_ALL_TESTS();
  449. }