block_fetcher_test.cc 21 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542
  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/block_fetcher.h"
  6. #include "db/table_properties_collector.h"
  7. #include "file/file_util.h"
  8. #include "options/options_helper.h"
  9. #include "port/port.h"
  10. #include "port/stack_trace.h"
  11. #include "rocksdb/db.h"
  12. #include "rocksdb/file_system.h"
  13. #include "table/block_based/binary_search_index_reader.h"
  14. #include "table/block_based/block_based_table_builder.h"
  15. #include "table/block_based/block_based_table_factory.h"
  16. #include "table/block_based/block_based_table_reader.h"
  17. #include "table/format.h"
  18. #include "test_util/testharness.h"
  19. #include "utilities/memory_allocators.h"
  20. namespace ROCKSDB_NAMESPACE {
  21. namespace {
  22. struct MemcpyStats {
  23. int num_stack_buf_memcpy;
  24. int num_heap_buf_memcpy;
  25. int num_compressed_buf_memcpy;
  26. };
  27. struct BufAllocationStats {
  28. int num_heap_buf_allocations;
  29. int num_compressed_buf_allocations;
  30. };
  31. struct TestStats {
  32. MemcpyStats memcpy_stats;
  33. BufAllocationStats buf_allocation_stats;
  34. };
  35. class BlockFetcherTest : public testing::Test {
  36. public:
  37. enum class Mode {
  38. kBufferedRead = 0,
  39. kBufferedMmap,
  40. kDirectRead,
  41. kNumModes,
  42. };
  43. // use NumModes as array size to avoid "size of array '...' has non-integral
  44. // type" errors.
  45. const static int NumModes = static_cast<int>(Mode::kNumModes);
  46. protected:
  47. void SetUp() override {
  48. SetupSyncPointsToMockDirectIO();
  49. test_dir_ = test::PerThreadDBPath("block_fetcher_test");
  50. env_ = Env::Default();
  51. fs_ = FileSystem::Default();
  52. ASSERT_OK(fs_->CreateDir(test_dir_, IOOptions(), nullptr));
  53. }
  54. void TearDown() override { EXPECT_OK(DestroyDir(env_, test_dir_)); }
  55. void AssertSameBlock(const std::string& block1, const std::string& block2) {
  56. ASSERT_EQ(block1, block2);
  57. }
  58. // Creates a table with kv pairs (i, i) where i ranges from 0 to 9, inclusive.
  59. void CreateTable(const std::string& table_name,
  60. const CompressionType& compression_type) {
  61. std::unique_ptr<WritableFileWriter> writer;
  62. NewFileWriter(table_name, &writer);
  63. // Create table builder.
  64. ImmutableOptions ioptions(options_);
  65. InternalKeyComparator comparator(options_.comparator);
  66. ColumnFamilyOptions cf_options(options_);
  67. MutableCFOptions moptions(cf_options);
  68. InternalTblPropCollFactories factories;
  69. const ReadOptions read_options;
  70. const WriteOptions write_options;
  71. std::unique_ptr<TableBuilder> table_builder(table_factory_.NewTableBuilder(
  72. TableBuilderOptions(ioptions, moptions, read_options, write_options,
  73. comparator, &factories, compression_type,
  74. CompressionOptions(), 0 /* column_family_id */,
  75. kDefaultColumnFamilyName, -1 /* level */,
  76. kUnknownNewestKeyTime),
  77. writer.get()));
  78. // Build table.
  79. for (int i = 0; i < 9; i++) {
  80. std::string key = ToInternalKey(std::to_string(i));
  81. // Append "00000000" to string value to enhance compression ratio
  82. std::string value = "00000000" + std::to_string(i);
  83. table_builder->Add(key, value);
  84. }
  85. ASSERT_OK(table_builder->Finish());
  86. }
  87. void FetchIndexBlock(const std::string& table_name,
  88. CountedMemoryAllocator* heap_buf_allocator,
  89. CountedMemoryAllocator* compressed_buf_allocator,
  90. MemcpyStats* memcpy_stats, BlockContents* index_block,
  91. std::string* result) {
  92. FileOptions fopt(options_);
  93. std::unique_ptr<RandomAccessFileReader> file;
  94. NewFileReader(table_name, fopt, &file);
  95. // Get handle of the index block.
  96. Footer footer;
  97. uint64_t file_size = 0;
  98. ReadFooter(file.get(), &footer, &file_size);
  99. // Index handle comes from metaindex for format_version >= 6
  100. ASSERT_TRUE(footer.index_handle().IsNull());
  101. BlockHandle index_handle;
  102. ASSERT_OK(FindMetaBlockInFile(
  103. file.get(), file_size, kBlockBasedTableMagicNumber,
  104. ImmutableOptions(options_), {}, kIndexBlockName, &index_handle));
  105. CompressionType compression_type;
  106. FetchBlock(file.get(), index_handle, BlockType::kIndex,
  107. false /* compressed */, false /* do_uncompress */,
  108. heap_buf_allocator, compressed_buf_allocator, index_block,
  109. memcpy_stats, &compression_type);
  110. ASSERT_EQ(compression_type, CompressionType::kNoCompression);
  111. result->assign(index_block->data.ToString());
  112. }
  113. // Fetches the first data block in both direct IO and non-direct IO mode.
  114. //
  115. // compressed: whether the data blocks are compressed;
  116. // do_uncompress: whether the data blocks should be uncompressed on fetching.
  117. // compression_type: the expected compression type.
  118. //
  119. // Expects:
  120. // Block contents are the same.
  121. // Bufferr allocation and memory copy statistics are expected.
  122. void TestFetchDataBlock(
  123. const std::string& table_name_prefix, bool compressed, bool do_uncompress,
  124. std::array<TestStats, NumModes> expected_stats_by_mode) {
  125. for (CompressionType compression_type : GetSupportedCompressions()) {
  126. bool do_compress = compression_type != kNoCompression;
  127. if (compressed != do_compress) {
  128. continue;
  129. }
  130. std::string compression_type_str =
  131. CompressionTypeToString(compression_type);
  132. std::string table_name = table_name_prefix + compression_type_str;
  133. CreateTable(table_name, compression_type);
  134. CompressionType expected_compression_type_after_fetch =
  135. (compressed && !do_uncompress) ? compression_type : kNoCompression;
  136. BlockContents blocks[NumModes];
  137. std::string block_datas[NumModes];
  138. MemcpyStats memcpy_stats[NumModes];
  139. CountedMemoryAllocator heap_buf_allocators[NumModes];
  140. CountedMemoryAllocator compressed_buf_allocators[NumModes];
  141. for (int i = 0; i < NumModes; ++i) {
  142. SetMode(static_cast<Mode>(i));
  143. FetchFirstDataBlock(table_name, compressed, do_uncompress,
  144. expected_compression_type_after_fetch,
  145. &heap_buf_allocators[i],
  146. &compressed_buf_allocators[i], &blocks[i],
  147. &block_datas[i], &memcpy_stats[i]);
  148. }
  149. for (int i = 0; i < NumModes - 1; ++i) {
  150. AssertSameBlock(block_datas[i], block_datas[i + 1]);
  151. }
  152. // Check memcpy and buffer allocation statistics.
  153. for (int i = 0; i < NumModes; ++i) {
  154. const TestStats& expected_stats = expected_stats_by_mode[i];
  155. ASSERT_EQ(memcpy_stats[i].num_stack_buf_memcpy,
  156. expected_stats.memcpy_stats.num_stack_buf_memcpy);
  157. ASSERT_EQ(memcpy_stats[i].num_heap_buf_memcpy,
  158. expected_stats.memcpy_stats.num_heap_buf_memcpy);
  159. ASSERT_EQ(memcpy_stats[i].num_compressed_buf_memcpy,
  160. expected_stats.memcpy_stats.num_compressed_buf_memcpy);
  161. if (kXpressCompression == compression_type) {
  162. // XPRESS allocates memory internally, thus does not support for
  163. // custom allocator verification
  164. continue;
  165. } else {
  166. ASSERT_EQ(
  167. heap_buf_allocators[i].GetNumAllocations(),
  168. expected_stats.buf_allocation_stats.num_heap_buf_allocations);
  169. ASSERT_EQ(compressed_buf_allocators[i].GetNumAllocations(),
  170. expected_stats.buf_allocation_stats
  171. .num_compressed_buf_allocations);
  172. // The allocated buffers are not deallocated until
  173. // the block content is deleted.
  174. ASSERT_EQ(heap_buf_allocators[i].GetNumDeallocations(), 0);
  175. ASSERT_EQ(compressed_buf_allocators[i].GetNumDeallocations(), 0);
  176. blocks[i].allocation.reset();
  177. ASSERT_EQ(
  178. heap_buf_allocators[i].GetNumDeallocations(),
  179. expected_stats.buf_allocation_stats.num_heap_buf_allocations);
  180. ASSERT_EQ(compressed_buf_allocators[i].GetNumDeallocations(),
  181. expected_stats.buf_allocation_stats
  182. .num_compressed_buf_allocations);
  183. }
  184. }
  185. }
  186. }
  187. void SetMode(Mode mode) {
  188. switch (mode) {
  189. case Mode::kBufferedRead:
  190. options_.use_direct_reads = false;
  191. options_.allow_mmap_reads = false;
  192. break;
  193. case Mode::kBufferedMmap:
  194. options_.use_direct_reads = false;
  195. options_.allow_mmap_reads = true;
  196. break;
  197. case Mode::kDirectRead:
  198. options_.use_direct_reads = true;
  199. options_.allow_mmap_reads = false;
  200. break;
  201. case Mode::kNumModes:
  202. assert(false);
  203. }
  204. }
  205. private:
  206. std::string test_dir_;
  207. Env* env_;
  208. std::shared_ptr<FileSystem> fs_;
  209. BlockBasedTableFactory table_factory_;
  210. Options options_;
  211. std::string Path(const std::string& fname) { return test_dir_ + "/" + fname; }
  212. void WriteToFile(const std::string& content, const std::string& filename) {
  213. std::unique_ptr<FSWritableFile> f;
  214. ASSERT_OK(fs_->NewWritableFile(Path(filename), FileOptions(), &f, nullptr));
  215. ASSERT_OK(f->Append(content, IOOptions(), nullptr));
  216. ASSERT_OK(f->Close(IOOptions(), nullptr));
  217. }
  218. void NewFileWriter(const std::string& filename,
  219. std::unique_ptr<WritableFileWriter>* writer) {
  220. std::string path = Path(filename);
  221. FileOptions file_options;
  222. ASSERT_OK(WritableFileWriter::Create(env_->GetFileSystem(), path,
  223. file_options, writer, nullptr));
  224. }
  225. void NewFileReader(const std::string& filename, const FileOptions& opt,
  226. std::unique_ptr<RandomAccessFileReader>* reader) {
  227. std::string path = Path(filename);
  228. std::unique_ptr<FSRandomAccessFile> f;
  229. ASSERT_OK(fs_->NewRandomAccessFile(path, opt, &f, nullptr));
  230. reader->reset(new RandomAccessFileReader(std::move(f), path,
  231. env_->GetSystemClock().get()));
  232. }
  233. void NewTableReader(const ImmutableOptions& ioptions,
  234. const FileOptions& foptions,
  235. const InternalKeyComparator& comparator,
  236. const std::string& table_name,
  237. std::unique_ptr<BlockBasedTable>* table) {
  238. std::unique_ptr<RandomAccessFileReader> file;
  239. NewFileReader(table_name, foptions, &file);
  240. uint64_t file_size = 0;
  241. ASSERT_OK(env_->GetFileSize(Path(table_name), &file_size));
  242. std::unique_ptr<TableReader> table_reader;
  243. ReadOptions ro;
  244. const auto* table_options =
  245. table_factory_.GetOptions<BlockBasedTableOptions>();
  246. ASSERT_NE(table_options, nullptr);
  247. ASSERT_OK(BlockBasedTable::Open(ro, ioptions, EnvOptions(), *table_options,
  248. comparator, std::move(file), file_size,
  249. 0 /* block_protection_bytes_per_key */,
  250. &table_reader, 0 /* tail_size */));
  251. table->reset(static_cast<BlockBasedTable*>(table_reader.release()));
  252. }
  253. std::string ToInternalKey(const std::string& key) {
  254. InternalKey internal_key(key, 0, ValueType::kTypeValue);
  255. return internal_key.Encode().ToString();
  256. }
  257. void ReadFooter(RandomAccessFileReader* file, Footer* footer,
  258. uint64_t* file_size_out = nullptr) {
  259. uint64_t file_size = 0;
  260. ASSERT_OK(env_->GetFileSize(file->file_name(), &file_size));
  261. IOOptions opts;
  262. ASSERT_OK(ReadFooterFromFile(opts, file, *fs_,
  263. nullptr /* prefetch_buffer */, file_size,
  264. footer, kBlockBasedTableMagicNumber));
  265. if (file_size_out) {
  266. *file_size_out = file_size;
  267. }
  268. }
  269. // NOTE: compression_type returns the compression type of the fetched block
  270. // contents, so if the block is fetched and uncompressed, then it's
  271. // kNoCompression.
  272. void FetchBlock(RandomAccessFileReader* file, const BlockHandle& block,
  273. BlockType block_type, bool compressed, bool do_uncompress,
  274. MemoryAllocator* heap_buf_allocator,
  275. MemoryAllocator* compressed_buf_allocator,
  276. BlockContents* contents, MemcpyStats* stats,
  277. CompressionType* compression_type) {
  278. ImmutableOptions ioptions(options_);
  279. ReadOptions roptions;
  280. PersistentCacheOptions persistent_cache_options;
  281. Footer footer;
  282. ReadFooter(file, &footer);
  283. auto mgr = GetBuiltinCompressionManager(
  284. GetCompressFormatForVersion(footer.format_version()));
  285. std::unique_ptr<BlockFetcher> fetcher(new BlockFetcher(
  286. file, nullptr /* prefetch_buffer */, footer, roptions, block, contents,
  287. ioptions, do_uncompress, compressed, block_type,
  288. mgr->GetDecompressor().get(), persistent_cache_options,
  289. heap_buf_allocator, compressed_buf_allocator));
  290. ASSERT_OK(fetcher->ReadBlockContents());
  291. stats->num_stack_buf_memcpy = fetcher->TEST_GetNumStackBufMemcpy();
  292. stats->num_heap_buf_memcpy = fetcher->TEST_GetNumHeapBufMemcpy();
  293. stats->num_compressed_buf_memcpy =
  294. fetcher->TEST_GetNumCompressedBufMemcpy();
  295. if (do_uncompress) {
  296. *compression_type = kNoCompression;
  297. } else {
  298. *compression_type = fetcher->compression_type();
  299. }
  300. }
  301. // NOTE: expected_compression_type is the expected compression
  302. // type of the fetched block content, if the block is uncompressed,
  303. // then the expected compression type is kNoCompression.
  304. void FetchFirstDataBlock(const std::string& table_name, bool compressed,
  305. bool do_uncompress,
  306. CompressionType expected_compression_type,
  307. MemoryAllocator* heap_buf_allocator,
  308. MemoryAllocator* compressed_buf_allocator,
  309. BlockContents* block, std::string* result,
  310. MemcpyStats* memcpy_stats) {
  311. ImmutableOptions ioptions(options_);
  312. InternalKeyComparator comparator(options_.comparator);
  313. FileOptions foptions(options_);
  314. // Get block handle for the first data block.
  315. std::unique_ptr<BlockBasedTable> table;
  316. NewTableReader(ioptions, foptions, comparator, table_name, &table);
  317. std::unique_ptr<BlockBasedTable::IndexReader> index_reader;
  318. ReadOptions ro;
  319. ASSERT_OK(BinarySearchIndexReader::Create(
  320. table.get(), ro, nullptr /* prefetch_buffer */, false /* use_cache */,
  321. false /* prefetch */, false /* pin */, nullptr /* lookup_context */,
  322. &index_reader));
  323. std::unique_ptr<InternalIteratorBase<IndexValue>> iter(
  324. index_reader->NewIterator(
  325. ReadOptions(), false /* disable_prefix_seek */, nullptr /* iter */,
  326. nullptr /* get_context */, nullptr /* lookup_context */));
  327. ASSERT_OK(iter->status());
  328. iter->SeekToFirst();
  329. BlockHandle first_block_handle = iter->value().handle;
  330. // Fetch first data block.
  331. std::unique_ptr<RandomAccessFileReader> file;
  332. NewFileReader(table_name, foptions, &file);
  333. CompressionType compression_type;
  334. FetchBlock(file.get(), first_block_handle, BlockType::kData, compressed,
  335. do_uncompress, heap_buf_allocator, compressed_buf_allocator,
  336. block, memcpy_stats, &compression_type);
  337. ASSERT_EQ(compression_type, expected_compression_type);
  338. result->assign(block->data.ToString());
  339. }
  340. };
  341. // Skip the following tests in lite mode since direct I/O is unsupported.
  342. // Fetch index block under both direct IO and non-direct IO.
  343. // Expects:
  344. // the index block contents are the same for both read modes.
  345. TEST_F(BlockFetcherTest, FetchIndexBlock) {
  346. for (CompressionType compression : GetSupportedCompressions()) {
  347. std::string table_name =
  348. "FetchIndexBlock" + CompressionTypeToString(compression);
  349. CreateTable(table_name, compression);
  350. CountedMemoryAllocator allocator;
  351. MemcpyStats memcpy_stats;
  352. BlockContents indexes[NumModes];
  353. std::string index_datas[NumModes];
  354. for (int i = 0; i < NumModes; ++i) {
  355. SetMode(static_cast<Mode>(i));
  356. FetchIndexBlock(table_name, &allocator, &allocator, &memcpy_stats,
  357. &indexes[i], &index_datas[i]);
  358. }
  359. for (int i = 0; i < NumModes - 1; ++i) {
  360. AssertSameBlock(index_datas[i], index_datas[i + 1]);
  361. }
  362. }
  363. }
  364. // Data blocks are not compressed,
  365. // fetch data block under direct IO, mmap IO,and non-direct IO.
  366. // Expects:
  367. // 1. in non-direct IO mode, allocate a heap buffer and memcpy the block
  368. // into the buffer;
  369. // 2. in direct IO mode, allocate a heap buffer and memcpy from the
  370. // direct IO buffer to the heap buffer.
  371. TEST_F(BlockFetcherTest, FetchUncompressedDataBlock) {
  372. TestStats expected_non_mmap_stats = {
  373. {
  374. 0 /* num_stack_buf_memcpy */,
  375. 1 /* num_heap_buf_memcpy */,
  376. 0 /* num_compressed_buf_memcpy */,
  377. },
  378. {
  379. 1 /* num_heap_buf_allocations */,
  380. 0 /* num_compressed_buf_allocations */,
  381. }};
  382. TestStats expected_mmap_stats = {{
  383. 0 /* num_stack_buf_memcpy */,
  384. 0 /* num_heap_buf_memcpy */,
  385. 0 /* num_compressed_buf_memcpy */,
  386. },
  387. {
  388. 0 /* num_heap_buf_allocations */,
  389. 0 /* num_compressed_buf_allocations */,
  390. }};
  391. std::array<TestStats, NumModes> expected_stats_by_mode{{
  392. expected_non_mmap_stats /* kBufferedRead */,
  393. expected_mmap_stats /* kBufferedMmap */,
  394. expected_non_mmap_stats /* kDirectRead */,
  395. }};
  396. TestFetchDataBlock("FetchUncompressedDataBlock", false, false,
  397. expected_stats_by_mode);
  398. }
  399. // Data blocks are compressed,
  400. // fetch data block under both direct IO and non-direct IO,
  401. // but do not uncompress.
  402. // Expects:
  403. // 1. in non-direct IO mode, allocate a compressed buffer and memcpy the block
  404. // into the buffer;
  405. // 2. in direct IO mode, allocate a compressed buffer and memcpy from the
  406. // direct IO buffer to the compressed buffer.
  407. TEST_F(BlockFetcherTest, FetchCompressedDataBlock) {
  408. TestStats expected_non_mmap_stats = {
  409. {
  410. 0 /* num_stack_buf_memcpy */,
  411. 0 /* num_heap_buf_memcpy */,
  412. 1 /* num_compressed_buf_memcpy */,
  413. },
  414. {
  415. 0 /* num_heap_buf_allocations */,
  416. 1 /* num_compressed_buf_allocations */,
  417. }};
  418. TestStats expected_mmap_stats = {{
  419. 0 /* num_stack_buf_memcpy */,
  420. 0 /* num_heap_buf_memcpy */,
  421. 0 /* num_compressed_buf_memcpy */,
  422. },
  423. {
  424. 0 /* num_heap_buf_allocations */,
  425. 0 /* num_compressed_buf_allocations */,
  426. }};
  427. std::array<TestStats, NumModes> expected_stats_by_mode{{
  428. expected_non_mmap_stats /* kBufferedRead */,
  429. expected_mmap_stats /* kBufferedMmap */,
  430. expected_non_mmap_stats /* kDirectRead */,
  431. }};
  432. TestFetchDataBlock("FetchCompressedDataBlock", true, false,
  433. expected_stats_by_mode);
  434. }
  435. // Data blocks are compressed,
  436. // fetch and uncompress data block under both direct IO and non-direct IO.
  437. // Expects:
  438. // 1. in non-direct IO mode, since the block is small, so it's first memcpyed
  439. // to the stack buffer, then a heap buffer is allocated and the block is
  440. // uncompressed into the heap.
  441. // 2. in direct IO mode mode, allocate a heap buffer, then directly uncompress
  442. // and memcpy from the direct IO buffer to the heap buffer.
  443. TEST_F(BlockFetcherTest, FetchAndUncompressCompressedDataBlock) {
  444. TestStats expected_buffered_read_stats = {
  445. {
  446. 1 /* num_stack_buf_memcpy */,
  447. 1 /* num_heap_buf_memcpy */,
  448. 0 /* num_compressed_buf_memcpy */,
  449. },
  450. {
  451. 1 /* num_heap_buf_allocations */,
  452. 0 /* num_compressed_buf_allocations */,
  453. }};
  454. TestStats expected_mmap_stats = {{
  455. 0 /* num_stack_buf_memcpy */,
  456. 1 /* num_heap_buf_memcpy */,
  457. 0 /* num_compressed_buf_memcpy */,
  458. },
  459. {
  460. 1 /* num_heap_buf_allocations */,
  461. 0 /* num_compressed_buf_allocations */,
  462. }};
  463. TestStats expected_direct_read_stats = {
  464. {
  465. 0 /* num_stack_buf_memcpy */,
  466. 1 /* num_heap_buf_memcpy */,
  467. 0 /* num_compressed_buf_memcpy */,
  468. },
  469. {
  470. 1 /* num_heap_buf_allocations */,
  471. 0 /* num_compressed_buf_allocations */,
  472. }};
  473. std::array<TestStats, NumModes> expected_stats_by_mode{{
  474. expected_buffered_read_stats,
  475. expected_mmap_stats,
  476. expected_direct_read_stats,
  477. }};
  478. TestFetchDataBlock("FetchAndUncompressCompressedDataBlock", true, true,
  479. expected_stats_by_mode);
  480. }
  481. } // namespace
  482. } // namespace ROCKSDB_NAMESPACE
  483. int main(int argc, char** argv) {
  484. ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
  485. ::testing::InitGoogleTest(&argc, argv);
  486. return RUN_ALL_TESTS();
  487. }