diff --git a/HISTORY.md b/HISTORY.md index ca65e8c18..a46a51a32 100644 --- a/HISTORY.md +++ b/HISTORY.md @@ -4,12 +4,11 @@ ### Public API changes * Added _LEVEL to all InfoLogLevel enums +* Deprecated ReadOptions.prefix and ReadOptions.prefix_seek. Seek() defaults to prefix-based seek when Options.prefix_extractor is supplied. More detail is documented in https://github.com/facebook/rocksdb/wiki/Prefix-Seek-API-Changes ### New Features * Column family support - -### Public API changes -* Deprecated ReadOptions.prefix and ReadOptions.prefix_seek. Seek() defaults to prefix-based seek when Options.prefix_extractor is supplied. More detail is documented in https://github.com/facebook/rocksdb/wiki/Prefix-Seek-API-Changes +* Added an option to use different checksum functions in BlockBasedTableOptions ## 2.8.0 (04/04/2014) diff --git a/db/db_bench.cc b/db/db_bench.cc index 650ec5d1e..2e8da9e48 100644 --- a/db/db_bench.cc +++ b/db/db_bench.cc @@ -36,6 +36,7 @@ #include "util/string_util.h" #include "util/statistics.h" #include "util/testutil.h" +#include "util/xxhash.h" #include "hdfs/env_hdfs.h" #include "utilities/merge_operators.h" @@ -64,6 +65,7 @@ DEFINE_string(benchmarks, "randomwithverify," "fill100K," "crc32c," + "xxhash," "compress," "uncompress," "acquireload,", @@ -107,6 +109,7 @@ DEFINE_string(benchmarks, "\tseekrandom -- N random seeks\n" "\tseekrandom -- 1 writer, N threads doing random seeks\n" "\tcrc32c -- repeated crc32c of 4K of data\n" + "\txxhash -- repeated xxHash of 4K of data\n" "\tacquireload -- load N*1000 times\n" "Meta operations:\n" "\tcompact -- Compact the entire DB\n" @@ -1234,6 +1237,8 @@ class Benchmark { method = &Benchmark::Compact; } else if (name == Slice("crc32c")) { method = &Benchmark::Crc32c; + } else if (name == Slice("xxhash")) { + method = &Benchmark::xxHash; } else if (name == Slice("acquireload")) { method = &Benchmark::AcquireLoad; } else if (name == Slice("compress")) { @@ -1382,6 +1387,25 @@ class Benchmark { thread->stats.AddMessage(label); } + void xxHash(ThreadState* thread) { + // Checksum about 500MB of data total + const int size = 4096; + const char* label = "(4K per op)"; + std::string data(size, 'x'); + int64_t bytes = 0; + unsigned int xxh32 = 0; + while (bytes < 500 * 1048576) { + xxh32 = XXH32(data.data(), size, 0); + thread->stats.FinishedSingleOp(nullptr); + bytes += size; + } + // Print so result is not dead + fprintf(stderr, "... xxh32=0x%x\r", static_cast(xxh32)); + + thread->stats.AddBytes(bytes); + thread->stats.AddMessage(label); + } + void AcquireLoad(ThreadState* thread) { int dummy; port::AtomicPointer ap(&dummy); diff --git a/db/db_test.cc b/db/db_test.cc index 638649755..350160af6 100644 --- a/db/db_test.cc +++ b/db/db_test.cc @@ -316,6 +316,7 @@ class DBTest { kUniversalCompaction, kCompressedBlockCache, kInfiniteMaxOpenFiles, + kxxHashChecksum, kEnd }; int option_config_; @@ -496,6 +497,12 @@ class DBTest { case kInfiniteMaxOpenFiles: options.max_open_files = -1; break; + case kxxHashChecksum: { + BlockBasedTableOptions table_options; + table_options.checksum = kxxHash; + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + break; + } case kBlockBasedTableWithPrefixHashIndex: { BlockBasedTableOptions table_options; table_options.index_type = BlockBasedTableOptions::kHashSearch; @@ -6778,7 +6785,40 @@ TEST(DBTest, TailingIteratorPrefixSeek) { ASSERT_TRUE(!iter->Valid()); } +TEST(DBTest, ChecksumTest) { + BlockBasedTableOptions table_options; + Options options = CurrentOptions(); + table_options.checksum = kCRC32c; + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + Reopen(&options); + ASSERT_OK(Put("a", "b")); + ASSERT_OK(Put("c", "d")); + ASSERT_OK(Flush()); // table with crc checksum + + table_options.checksum = kxxHash; + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + Reopen(&options); + ASSERT_OK(Put("e", "f")); + ASSERT_OK(Put("g", "h")); + ASSERT_OK(Flush()); // table with xxhash checksum + + table_options.checksum = kCRC32c; + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + Reopen(&options); + ASSERT_EQ("b", Get("a")); + ASSERT_EQ("d", Get("c")); + ASSERT_EQ("f", Get("e")); + ASSERT_EQ("h", Get("g")); + + table_options.checksum = kCRC32c; + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + Reopen(&options); + ASSERT_EQ("b", Get("a")); + ASSERT_EQ("d", Get("c")); + ASSERT_EQ("f", Get("e")); + ASSERT_EQ("h", Get("g")); +} } // namespace rocksdb int main(int argc, char** argv) { diff --git a/include/rocksdb/table.h b/include/rocksdb/table.h index aad480575..11adfec8c 100644 --- a/include/rocksdb/table.h +++ b/include/rocksdb/table.h @@ -38,6 +38,12 @@ struct Options; using std::unique_ptr; +enum ChecksumType : char { + kNoChecksum = 0x0, // not yet supported. Will fail + kCRC32c = 0x1, + kxxHash = 0x2, +}; + // For advanced user only struct BlockBasedTableOptions { // @flush_block_policy_factory creates the instances of flush block policy. @@ -67,6 +73,11 @@ struct BlockBasedTableOptions { }; IndexType index_type = kBinarySearch; + + // Use the specified checksum type. Newly created table files will be + // protected with this checksum type. Old table files will still be readable, + // even though they have different checksum type. + ChecksumType checksum = kCRC32c; }; // Table Properties that are specific to block-based table properties. diff --git a/table/block_based_table_builder.cc b/table/block_based_table_builder.cc index 3dd51cd65..c6469a270 100644 --- a/table/block_based_table_builder.cc +++ b/table/block_based_table_builder.cc @@ -37,6 +37,7 @@ #include "util/coding.h" #include "util/crc32c.h" #include "util/stop_watch.h" +#include "util/xxhash.h" namespace rocksdb { @@ -231,12 +232,14 @@ Slice CompressBlock(const Slice& raw, } // anonymous namespace // kBlockBasedTableMagicNumber was picked by running -// echo http://code.google.com/p/leveldb/ | sha1sum +// echo rocksdb.table.block_based | sha1sum // and taking the leading 64 bits. // Please note that kBlockBasedTableMagicNumber may also be accessed by // other .cc files so it have to be explicitly declared with "extern". -extern const uint64_t kBlockBasedTableMagicNumber - = 0xdb4775248b80fb57ull; +extern const uint64_t kBlockBasedTableMagicNumber = 0x88e241b785f4cff7ull; +// We also support reading and writing legacy block based table format (for +// backwards compatibility) +extern const uint64_t kLegacyBlockBasedTableMagicNumber = 0xdb4775248b80fb57ull; // A collector that collects properties of interest to block-based table. // For now this class looks heavy-weight since we only write one additional @@ -289,6 +292,7 @@ struct BlockBasedTableBuilder::Rep { std::string last_key; CompressionType compression_type; + ChecksumType checksum_type; TableProperties props; bool closed = false; // Either Finish() or Abandon() has been called. @@ -303,7 +307,8 @@ struct BlockBasedTableBuilder::Rep { Rep(const Options& opt, const InternalKeyComparator& icomparator, WritableFile* f, FlushBlockPolicyFactory* flush_block_policy_factory, - CompressionType compression_type, IndexType index_block_type) + CompressionType compression_type, IndexType index_block_type, + ChecksumType checksum_type) : options(opt), internal_comparator(icomparator), file(f), @@ -311,6 +316,7 @@ struct BlockBasedTableBuilder::Rep { index_builder( CreateIndexBuilder(index_block_type, &internal_comparator)), compression_type(compression_type), + checksum_type(checksum_type), filter_block(opt.filter_policy == nullptr ? nullptr : new FilterBlockBuilder(opt, &internal_comparator)), @@ -330,7 +336,8 @@ BlockBasedTableBuilder::BlockBasedTableBuilder( : rep_(new Rep(options, internal_comparator, file, table_options.flush_block_policy_factory.get(), compression_type, - BlockBasedTableOptions::IndexType::kBinarySearch)) { + BlockBasedTableOptions::IndexType::kBinarySearch, + table_options.checksum)) { if (rep_->filter_block != nullptr) { rep_->filter_block->StartBlock(0); } @@ -443,9 +450,27 @@ void BlockBasedTableBuilder::WriteRawBlock(const Slice& block_contents, if (r->status.ok()) { char trailer[kBlockTrailerSize]; trailer[0] = type; - uint32_t crc = crc32c::Value(block_contents.data(), block_contents.size()); - crc = crc32c::Extend(crc, trailer, 1); // Extend crc to cover block type - EncodeFixed32(trailer+1, crc32c::Mask(crc)); + char* trailer_without_type = trailer + 1; + switch (r->checksum_type) { + case kNoChecksum: + // we don't support no checksum yet + assert(false); + // intentional fallthrough in release binary + case kCRC32c: { + auto crc = crc32c::Value(block_contents.data(), block_contents.size()); + crc = crc32c::Extend(crc, trailer, 1); // Extend to cover block type + EncodeFixed32(trailer_without_type, crc32c::Mask(crc)); + break; + } + case kxxHash: { + void* xxh = XXH32_init(0); + XXH32_update(xxh, block_contents.data(), block_contents.size()); + XXH32_update(xxh, trailer, 1); // Extend to cover block type + EncodeFixed32(trailer_without_type, XXH32_digest(xxh)); + break; + } + } + r->status = r->file->Append(Slice(trailer, kBlockTrailerSize)); if (r->status.ok()) { r->status = InsertBlockInCache(block_contents, type, handle); @@ -596,9 +621,19 @@ Status BlockBasedTableBuilder::Finish() { // Write footer if (ok()) { - Footer footer(kBlockBasedTableMagicNumber); + // No need to write out new footer if we're using default checksum. + // We're writing legacy magic number because we want old versions of RocksDB + // be able to read files generated with new release (just in case if + // somebody wants to roll back after an upgrade) + // TODO(icanadi) at some point in the future, when we're absolutely sure + // nobody will roll back to RocksDB 2.x versions, retire the legacy magic + // number and always write new table files with new magic number + bool legacy = (r->checksum_type == kCRC32c); + Footer footer(legacy ? kLegacyBlockBasedTableMagicNumber + : kBlockBasedTableMagicNumber); footer.set_metaindex_handle(metaindex_block_handle); footer.set_index_handle(index_block_handle); + footer.set_checksum(r->checksum_type); std::string footer_encoding; footer.EncodeTo(&footer_encoding); r->status = r->file->Append(footer_encoding); diff --git a/table/block_based_table_reader.cc b/table/block_based_table_reader.cc index f75726108..953800787 100644 --- a/table/block_based_table_reader.cc +++ b/table/block_based_table_reader.cc @@ -37,7 +37,7 @@ namespace rocksdb { -extern uint64_t kBlockBasedTableMagicNumber; +extern const uint64_t kBlockBasedTableMagicNumber; using std::unique_ptr; typedef BlockBasedTable::IndexReader IndexReader; @@ -56,12 +56,13 @@ const size_t kMaxCacheKeyPrefixSize __attribute__((unused)) = // Set *didIO to true if didIO is not null. // On failure return non-OK. // On success fill *result and return OK - caller owns *result -Status ReadBlockFromFile(RandomAccessFile* file, const ReadOptions& options, - const BlockHandle& handle, Block** result, Env* env, - bool* didIO = nullptr, bool do_uncompress = true) { +Status ReadBlockFromFile(RandomAccessFile* file, const Footer& footer, + const ReadOptions& options, const BlockHandle& handle, + Block** result, Env* env, bool* didIO = nullptr, + bool do_uncompress = true) { BlockContents contents; - Status s = - ReadBlockContents(file, options, handle, &contents, env, do_uncompress); + Status s = ReadBlockContents(file, footer, options, handle, &contents, env, + do_uncompress); if (s.ok()) { *result = new Block(contents); } @@ -154,11 +155,12 @@ class BinarySearchIndexReader : public IndexReader { // `BinarySearchIndexReader`. // On success, index_reader will be populated; otherwise it will remain // unmodified. - static Status Create(RandomAccessFile* file, const BlockHandle& index_handle, - Env* env, const Comparator* comparator, + static Status Create(RandomAccessFile* file, const Footer& footer, + const BlockHandle& index_handle, Env* env, + const Comparator* comparator, IndexReader** index_reader) { Block* index_block = nullptr; - auto s = ReadBlockFromFile(file, ReadOptions(), index_handle, + auto s = ReadBlockFromFile(file, footer, ReadOptions(), index_handle, &index_block, env); if (s.ok()) { @@ -190,15 +192,16 @@ class BinarySearchIndexReader : public IndexReader { // without the table to be fully initialized. class HashIndexReader : public IndexReader { public: - static Status Create(RandomAccessFile* file, const BlockHandle& index_handle, - Env* env, const Comparator* comparator, + static Status Create(RandomAccessFile* file, const Footer& footer, + const BlockHandle& index_handle, Env* env, + const Comparator* comparator, std::function data_iter_gen, const SliceTransform* prefix_extractor, IndexReader** index_reader) { assert(prefix_extractor); Block* index_block = nullptr; - auto s = - ReadBlockFromFile(file, ReadOptions(), index_handle, &index_block, env); + auto s = ReadBlockFromFile(file, footer, ReadOptions(), index_handle, + &index_block, env); if (!s.ok()) { return s; @@ -245,10 +248,8 @@ struct BlockBasedTable::Rep { char compressed_cache_key_prefix[kMaxCacheKeyPrefixSize]; size_t compressed_cache_key_prefix_size = 0; - // Handle to metaindex_block: saved from footer - BlockHandle metaindex_handle; - // Handle to index: saved from footer - BlockHandle index_handle; + // Footer contains the fixed table information + Footer footer; // index_reader and filter will be populated and used only when // options.block_cache is nullptr; otherwise we will get the index block via // the block cache. @@ -354,8 +355,7 @@ Status BlockBasedTable::Open(const Options& options, const EnvOptions& soptions, Rep* rep = new BlockBasedTable::Rep(soptions, internal_comparator); rep->options = options; rep->file = std::move(file); - rep->metaindex_handle = footer.metaindex_handle(); - rep->index_handle = footer.index_handle(); + rep->footer = footer; rep->index_type = table_options.index_type; SetupCacheKeyPrefix(rep); unique_ptr new_table(new BlockBasedTable(rep)); @@ -373,8 +373,9 @@ Status BlockBasedTable::Open(const Options& options, const EnvOptions& soptions, s = meta_iter->status(); TableProperties* table_properties = nullptr; if (s.ok()) { - s = ReadProperties(meta_iter->value(), rep->file.get(), rep->options.env, - rep->options.info_log.get(), &table_properties); + s = ReadProperties(meta_iter->value(), rep->file.get(), rep->footer, + rep->options.env, rep->options.info_log.get(), + &table_properties); } if (!s.ok()) { @@ -470,8 +471,9 @@ Status BlockBasedTable::ReadMetaBlock( Block* meta = nullptr; Status s = ReadBlockFromFile( rep->file.get(), + rep->footer, ReadOptions(), - rep->metaindex_handle, + rep->footer.metaindex_handle(), &meta, rep->options.env); @@ -629,8 +631,8 @@ FilterBlockReader* BlockBasedTable::ReadFilter ( // requiring checksum verification in Table::Open. ReadOptions opt; BlockContents block; - if (!ReadBlockContents(rep->file.get(), opt, filter_handle, &block, - rep->options.env, false).ok()) { + if (!ReadBlockContents(rep->file.get(), rep->footer, opt, filter_handle, + &block, rep->options.env, false).ok()) { return nullptr; } @@ -642,7 +644,6 @@ FilterBlockReader* BlockBasedTable::ReadFilter ( rep->options, block.data, block.heap_allocated); } - BlockBasedTable::CachableEntry BlockBasedTable::GetFilter( bool no_io) const { // filter pre-populated @@ -661,7 +662,7 @@ BlockBasedTable::CachableEntry BlockBasedTable::GetFilter( auto key = GetCacheKey( rep_->cache_key_prefix, rep_->cache_key_prefix_size, - rep_->metaindex_handle, + rep_->footer.metaindex_handle(), cache_key ); @@ -713,7 +714,7 @@ Iterator* BlockBasedTable::NewIndexIterator(const ReadOptions& read_options) { Cache* block_cache = rep_->options.block_cache.get(); char cache_key[kMaxCacheKeyPrefixSize + kMaxVarint64Length]; auto key = GetCacheKey(rep_->cache_key_prefix, rep_->cache_key_prefix_size, - rep_->index_handle, cache_key); + rep_->footer.index_handle(), cache_key); Statistics* statistics = rep_->options.statistics.get(); auto cache_handle = GetEntryFromCache(block_cache, key, BLOCK_CACHE_INDEX_MISS, @@ -798,7 +799,7 @@ Iterator* BlockBasedTable::NewDataBlockIterator(Rep* rep, Block* raw_block = nullptr; { StopWatch sw(rep->options.env, statistics, histogram); - s = ReadBlockFromFile(rep->file.get(), ro, handle, + s = ReadBlockFromFile(rep->file.get(), rep->footer, ro, handle, &raw_block, rep->options.env, didIO, block_cache_compressed == nullptr); } @@ -816,7 +817,7 @@ Iterator* BlockBasedTable::NewDataBlockIterator(Rep* rep, // Could not read from block_cache and can't do IO return NewErrorIterator(Status::Incomplete("no blocking io")); } - s = ReadBlockFromFile(rep->file.get(), ro, handle, + s = ReadBlockFromFile(rep->file.get(), rep->footer, ro, handle, &block.value, rep->options.env, didIO); } @@ -1052,14 +1053,14 @@ Status BlockBasedTable::CreateIndexReader(IndexReader** index_reader) { } auto file = rep_->file.get(); - const auto& index_handle = rep_->index_handle; auto env = rep_->options.env; auto comparator = &rep_->internal_comparator; + const Footer& footer = rep_->footer; switch (rep_->index_type) { case BlockBasedTableOptions::kBinarySearch: { - return BinarySearchIndexReader::Create(file, index_handle, env, - comparator, index_reader); + return BinarySearchIndexReader::Create( + file, footer, footer.index_handle(), env, comparator, index_reader); } case BlockBasedTableOptions::kHashSearch: { // We need to wrap data with internal_prefix_transform to make sure it can @@ -1067,7 +1068,7 @@ Status BlockBasedTable::CreateIndexReader(IndexReader** index_reader) { rep_->internal_prefix_transform.reset( new InternalKeySliceTransform(rep_->options.prefix_extractor.get())); return HashIndexReader::Create( - file, index_handle, env, comparator, + file, footer, footer.index_handle(), env, comparator, [&](Iterator* index_iter) { return NewTwoLevelIterator(new BlockEntryIteratorState(this, ReadOptions(), nullptr), index_iter); @@ -1099,7 +1100,7 @@ uint64_t BlockBasedTable::ApproximateOffsetOf(const Slice& key) { // Strange: we can't decode the block handle in the index block. // We'll just return the offset of the metaindex block, which is // close to the whole file size for this case. - result = rep_->metaindex_handle.offset(); + result = rep_->footer.metaindex_handle().offset(); } } else { // key is past the last key in the file. If table_properties is not @@ -1111,7 +1112,7 @@ uint64_t BlockBasedTable::ApproximateOffsetOf(const Slice& key) { } // table_properties is not present in the table. if (result == 0) { - result = rep_->metaindex_handle.offset(); + result = rep_->footer.metaindex_handle().offset(); } } return result; diff --git a/table/format.cc b/table/format.cc index 0d93cb93f..8514f7ac3 100644 --- a/table/format.cc +++ b/table/format.cc @@ -18,9 +18,15 @@ #include "util/coding.h" #include "util/crc32c.h" #include "util/perf_context_imp.h" +#include "util/xxhash.h" namespace rocksdb { +extern const uint64_t kLegacyBlockBasedTableMagicNumber; +extern const uint64_t kLegacyPlainTableMagicNumber; +extern const uint64_t kBlockBasedTableMagicNumber; +extern const uint64_t kPlainTableMagicNumber; + void BlockHandle::EncodeTo(std::string* dst) const { // Sanity check that all fields have been set assert(offset_ != ~static_cast(0)); @@ -39,28 +45,81 @@ Status BlockHandle::DecodeFrom(Slice* input) { } const BlockHandle BlockHandle::kNullBlockHandle(0, 0); +// legacy footer format: +// metaindex handle (varint64 offset, varint64 size) +// index handle (varint64 offset, varint64 size) +// to make the total size 2 * BlockHandle::kMaxEncodedLength +// table_magic_number (8 bytes) +// new footer format: +// checksum (char, 1 byte) +// metaindex handle (varint64 offset, varint64 size) +// index handle (varint64 offset, varint64 size) +// to make the total size 2 * BlockHandle::kMaxEncodedLength + 1 +// footer version (4 bytes) +// table_magic_number (8 bytes) void Footer::EncodeTo(std::string* dst) const { -#ifndef NDEBUG - const size_t original_size = dst->size(); -#endif - metaindex_handle_.EncodeTo(dst); - index_handle_.EncodeTo(dst); - dst->resize(2 * BlockHandle::kMaxEncodedLength); // Padding - PutFixed32(dst, static_cast(table_magic_number() & 0xffffffffu)); - PutFixed32(dst, static_cast(table_magic_number() >> 32)); - assert(dst->size() == original_size + kEncodedLength); + if (version() == kLegacyFooter) { + // has to be default checksum with legacy footer + assert(checksum_ == kCRC32c); + const size_t original_size = dst->size(); + metaindex_handle_.EncodeTo(dst); + index_handle_.EncodeTo(dst); + dst->resize(original_size + 2 * BlockHandle::kMaxEncodedLength); // Padding + PutFixed32(dst, static_cast(table_magic_number() & 0xffffffffu)); + PutFixed32(dst, static_cast(table_magic_number() >> 32)); + assert(dst->size() == original_size + kVersion0EncodedLength); + } else { + const size_t original_size = dst->size(); + dst->push_back(static_cast(checksum_)); + metaindex_handle_.EncodeTo(dst); + index_handle_.EncodeTo(dst); + dst->resize(original_size + kVersion1EncodedLength - 12); // Padding + PutFixed32(dst, kFooterVersion); + PutFixed32(dst, static_cast(table_magic_number() & 0xffffffffu)); + PutFixed32(dst, static_cast(table_magic_number() >> 32)); + assert(dst->size() == original_size + kVersion1EncodedLength); + } } +namespace { +inline bool IsLegacyFooterFormat(uint64_t magic_number) { + return magic_number == kLegacyBlockBasedTableMagicNumber || + magic_number == kLegacyPlainTableMagicNumber; +} + +inline uint64_t UpconvertLegacyFooterFormat(uint64_t magic_number) { + if (magic_number == kLegacyBlockBasedTableMagicNumber) { + return kBlockBasedTableMagicNumber; + } + if (magic_number == kLegacyPlainTableMagicNumber) { + return kPlainTableMagicNumber; + } + assert(false); +} +} // namespace + +Footer::Footer(uint64_t table_magic_number) + : version_(IsLegacyFooterFormat(table_magic_number) ? kLegacyFooter + : kFooterVersion), + checksum_(kCRC32c), + table_magic_number_(table_magic_number) {} + Status Footer::DecodeFrom(Slice* input) { assert(input != nullptr); - assert(input->size() >= kEncodedLength); + assert(input->size() >= kMinEncodedLength); - const char* magic_ptr = - input->data() + kEncodedLength - kMagicNumberLengthByte; + const char *magic_ptr = + input->data() + input->size() - kMagicNumberLengthByte; const uint32_t magic_lo = DecodeFixed32(magic_ptr); const uint32_t magic_hi = DecodeFixed32(magic_ptr + 4); - const uint64_t magic = ((static_cast(magic_hi) << 32) | - (static_cast(magic_lo))); + uint64_t magic = ((static_cast(magic_hi) << 32) | + (static_cast(magic_lo))); + + // We check for legacy formats here and silently upconvert them + bool legacy = IsLegacyFooterFormat(magic); + if (legacy) { + magic = UpconvertLegacyFooterFormat(magic); + } if (HasInitializedTableMagicNumber()) { if (magic != table_magic_number()) { char buffer[80]; @@ -73,13 +132,39 @@ Status Footer::DecodeFrom(Slice* input) { set_table_magic_number(magic); } + if (legacy) { + // The size is already asserted to be at least kMinEncodedLength + // at the beginning of the function + input->remove_prefix(input->size() - kVersion0EncodedLength); + version_ = kLegacyFooter; + checksum_ = kCRC32c; + } else { + version_ = DecodeFixed32(magic_ptr - 4); + if (version_ != kFooterVersion) { + return Status::Corruption("bad footer version"); + } + // Footer version 1 will always occupy exactly this many bytes. + // It consists of the checksum type, two block handles, padding, + // a version number, and a magic number + if (input->size() < kVersion1EncodedLength) { + return Status::InvalidArgument("input is too short to be an sstable"); + } else { + input->remove_prefix(input->size() - kVersion1EncodedLength); + } + uint32_t checksum; + if (!GetVarint32(input, &checksum)) { + return Status::Corruption("bad checksum type"); + } + checksum_ = static_cast(checksum); + } + Status result = metaindex_handle_.DecodeFrom(input); if (result.ok()) { result = index_handle_.DecodeFrom(input); } if (result.ok()) { // We skip over any leftover data (just padding for now) in "input" - const char* end = magic_ptr + 8; + const char* end = magic_ptr + kMagicNumberLengthByte; *input = Slice(end, input->data() + input->size() - end); } return result; @@ -88,21 +173,22 @@ Status Footer::DecodeFrom(Slice* input) { Status ReadFooterFromFile(RandomAccessFile* file, uint64_t file_size, Footer* footer) { - if (file_size < Footer::kEncodedLength) { + if (file_size < Footer::kMinEncodedLength) { return Status::InvalidArgument("file is too short to be an sstable"); } - char footer_space[Footer::kEncodedLength]; + char footer_space[Footer::kMaxEncodedLength]; Slice footer_input; - Status s = file->Read(file_size - Footer::kEncodedLength, - Footer::kEncodedLength, - &footer_input, + size_t read_offset = (file_size > Footer::kMaxEncodedLength) + ? (file_size - Footer::kMaxEncodedLength) + : 0; + Status s = file->Read(read_offset, Footer::kMaxEncodedLength, &footer_input, footer_space); if (!s.ok()) return s; // Check that we actually read the whole footer from the file. It may be // that size isn't correct. - if (footer_input.size() != Footer::kEncodedLength) { + if (footer_input.size() < Footer::kMinEncodedLength) { return Status::InvalidArgument("file is too short to be an sstable"); } @@ -110,6 +196,7 @@ Status ReadFooterFromFile(RandomAccessFile* file, } Status ReadBlockContents(RandomAccessFile* file, + const Footer& footer, const ReadOptions& options, const BlockHandle& handle, BlockContents* result, @@ -143,18 +230,33 @@ Status ReadBlockContents(RandomAccessFile* file, // Check the crc of the type and the block contents const char* data = contents.data(); // Pointer to where Read put the data if (options.verify_checksums) { - const uint32_t crc = crc32c::Unmask(DecodeFixed32(data + n + 1)); - const uint32_t actual = crc32c::Value(data, n + 1); - if (actual != crc) { - delete[] buf; + uint32_t value = DecodeFixed32(data + n + 1); + uint32_t actual; + switch (footer.checksum()) { + case kCRC32c: + value = crc32c::Unmask(value); + actual = crc32c::Value(data, n + 1); + break; + case kxxHash: + actual = XXH32(data, n + 1, 0); + break; + default: + s = Status::Corruption("unknown checksum type"); + } + if (s.ok() && actual != value) { s = Status::Corruption("block checksum mismatch"); + } + if (!s.ok()) { + delete[] buf; return s; } PERF_TIMER_MEASURE(block_checksum_time); } + rocksdb::CompressionType compression_type = + static_cast(data[n]); // If the caller has requested that the block not be uncompressed - if (!do_uncompress || data[n] == kNoCompression) { + if (!do_uncompress || compression_type == kNoCompression) { if (data != buf) { // File implementation gave us pointer to some other data. // Use it directly under the assumption that it will be live @@ -168,8 +270,8 @@ Status ReadBlockContents(RandomAccessFile* file, result->heap_allocated = true; result->cachable = true; } - result->compression_type = (rocksdb::CompressionType)data[n]; - s = Status::OK(); + result->compression_type = compression_type; + s = Status::OK(); } else { s = UncompressBlockContents(data, n, result); delete[] buf; diff --git a/table/format.h b/table/format.h index f05fbf890..a971c1a67 100644 --- a/table/format.h +++ b/table/format.h @@ -74,24 +74,32 @@ class Footer { // @table_magic_number serves two purposes: // 1. Identify different types of the tables. // 2. Help us to identify if a given file is a valid sst. - explicit Footer(uint64_t table_magic_number) - : table_magic_number_(table_magic_number) {} + explicit Footer(uint64_t table_magic_number); + + // The version of the footer in this file + uint32_t version() const { return version_; } + + // The checksum type used in this file + ChecksumType checksum() const { return checksum_; } + void set_checksum(const ChecksumType c) { checksum_ = c; } // The block handle for the metaindex block of the table const BlockHandle& metaindex_handle() const { return metaindex_handle_; } void set_metaindex_handle(const BlockHandle& h) { metaindex_handle_ = h; } // The block handle for the index block of the table - const BlockHandle& index_handle() const { - return index_handle_; - } + const BlockHandle& index_handle() const { return index_handle_; } - void set_index_handle(const BlockHandle& h) { - index_handle_ = h; - } + void set_index_handle(const BlockHandle& h) { index_handle_ = h; } uint64_t table_magic_number() const { return table_magic_number_; } + // The version of Footer we encode + enum { + kLegacyFooter = 0, + kFooterVersion = 1, + }; + void EncodeTo(std::string* dst) const; // Set the current footer based on the input slice. If table_magic_number_ @@ -102,11 +110,21 @@ class Footer { // when the test passes. Status DecodeFrom(Slice* input); - // Encoded length of a Footer. Note that the serialization of a - // Footer will always occupy exactly this many bytes. It consists - // of two block handles and a magic number. + // Encoded length of a Footer. Note that the serialization of a Footer will + // always occupy at least kMinEncodedLength bytes. If fields are changed + // the version number should be incremented and kMaxEncodedLength should be + // increased accordingly. enum { - kEncodedLength = 2 * BlockHandle::kMaxEncodedLength + 8 + // Footer version 0 (legacy) will always occupy exactly this many bytes. + // It consists of two block handles, padding, and a magic number. + kVersion0EncodedLength = 2 * BlockHandle::kMaxEncodedLength + 8, + // Footer version 1 will always occupy exactly this many bytes. + // It consists of the checksum type, two block handles, padding, + // a version number, and a magic number + kVersion1EncodedLength = 1 + 2 * BlockHandle::kMaxEncodedLength + 4 + 8, + + kMinEncodedLength = kVersion0EncodedLength, + kMaxEncodedLength = kVersion1EncodedLength }; static const uint64_t kInvalidTableMagicNumber = 0; @@ -124,6 +142,8 @@ class Footer { return (table_magic_number_ != kInvalidTableMagicNumber); } + uint32_t version_; + ChecksumType checksum_; BlockHandle metaindex_handle_; BlockHandle index_handle_; uint64_t table_magic_number_ = 0; @@ -147,6 +167,7 @@ struct BlockContents { // Read the block identified by "handle" from "file". On failure // return non-OK. On success fill *result and return OK. extern Status ReadBlockContents(RandomAccessFile* file, + const Footer& footer, const ReadOptions& options, const BlockHandle& handle, BlockContents* result, diff --git a/table/meta_blocks.cc b/table/meta_blocks.cc index 5775c2a14..f28b44d62 100644 --- a/table/meta_blocks.cc +++ b/table/meta_blocks.cc @@ -133,9 +133,9 @@ bool NotifyCollectTableCollectorsOnFinish( return all_succeeded; } -Status ReadProperties(const Slice& handle_value, RandomAccessFile* file, - Env* env, Logger* logger, - TableProperties** table_properties) { +Status ReadProperties(const Slice &handle_value, RandomAccessFile *file, + const Footer &footer, Env *env, Logger *logger, + TableProperties **table_properties) { assert(table_properties); Slice v = handle_value; @@ -147,8 +147,8 @@ Status ReadProperties(const Slice& handle_value, RandomAccessFile* file, BlockContents block_contents; ReadOptions read_options; read_options.verify_checksums = false; - Status s = ReadBlockContents(file, read_options, handle, &block_contents, env, - false); + Status s = ReadBlockContents(file, footer, read_options, handle, + &block_contents, env, false); if (!s.ok()) { return s; @@ -234,7 +234,7 @@ Status ReadTableProperties(RandomAccessFile* file, uint64_t file_size, BlockContents metaindex_contents; ReadOptions read_options; read_options.verify_checksums = false; - s = ReadBlockContents(file, read_options, metaindex_handle, + s = ReadBlockContents(file, footer, read_options, metaindex_handle, &metaindex_contents, env, false); if (!s.ok()) { return s; @@ -252,7 +252,8 @@ Status ReadTableProperties(RandomAccessFile* file, uint64_t file_size, TableProperties table_properties; if (found_properties_block == true) { - s = ReadProperties(meta_iter->value(), file, env, info_log, properties); + s = ReadProperties(meta_iter->value(), file, footer, env, info_log, + properties); } else { s = Status::Corruption("Unable to read the property block."); Log(WARN_LEVEL, info_log, @@ -262,38 +263,4 @@ Status ReadTableProperties(RandomAccessFile* file, uint64_t file_size, return s; } -Status ReadTableMagicNumber(const std::string& file_path, - const Options& options, - const EnvOptions& env_options, - uint64_t* table_magic_number) { - unique_ptr file; - Status s = options.env->NewRandomAccessFile(file_path, &file, env_options); - if (!s.ok()) { - return s; - } - - uint64_t file_size; - options.env->GetFileSize(file_path, &file_size); - return ReadTableMagicNumber(file.get(), file_size, options, env_options, - table_magic_number); -} - -Status ReadTableMagicNumber(RandomAccessFile* file, uint64_t file_size, - const Options& options, - const EnvOptions& env_options, - uint64_t* table_magic_number) { - if (file_size < Footer::kEncodedLength) { - return Status::InvalidArgument("file is too short to be an sstable"); - } - - Footer footer; - auto s = ReadFooterFromFile(file, file_size, &footer); - if (!s.ok()) { - return s; - } - - *table_magic_number = footer.table_magic_number(); - return Status::OK(); -} - } // namespace rocksdb diff --git a/table/meta_blocks.h b/table/meta_blocks.h index 9371409cb..2ac890345 100644 --- a/table/meta_blocks.h +++ b/table/meta_blocks.h @@ -20,6 +20,7 @@ namespace rocksdb { class BlockBuilder; class BlockHandle; class Env; +class Footer; class Logger; class RandomAccessFile; struct TableProperties; @@ -106,9 +107,9 @@ bool NotifyCollectTableCollectorsOnFinish( // @returns a status to indicate if the operation succeeded. On success, // *table_properties will point to a heap-allocated TableProperties // object, otherwise value of `table_properties` will not be modified. -Status ReadProperties(const Slice& handle_value, RandomAccessFile* file, - Env* env, Logger* logger, - TableProperties** table_properties); +Status ReadProperties(const Slice &handle_value, RandomAccessFile *file, + const Footer &footer, Env *env, Logger *logger, + TableProperties **table_properties); // Directly read the properties from the properties block of a plain table. // @returns a status to indicate if the operation succeeded. On success, @@ -118,18 +119,6 @@ Status ReadTableProperties(RandomAccessFile* file, uint64_t file_size, uint64_t table_magic_number, Env* env, Logger* info_log, TableProperties** properties); -// Read the magic number of the specified file directly. The magic number -// of a valid sst table the last 8-byte of the file. -Status ReadTableMagicNumber(const std::string& file_path, - const Options& options, - const EnvOptions& env_options, - uint64_t* table_magic_number); - -Status ReadTableMagicNumber(RandomAccessFile* file, uint64_t file_size, - const Options& options, - const EnvOptions& env_options, - uint64_t* table_magic_number); - // Seek to the properties block. // If it successfully seeks to the properties block, "is_found" will be // set to true. diff --git a/table/plain_table_builder.cc b/table/plain_table_builder.cc index 57bfd3bcf..d76f0b2df 100644 --- a/table/plain_table_builder.cc +++ b/table/plain_table_builder.cc @@ -47,9 +47,10 @@ Status WriteBlock( } // namespace // kPlainTableMagicNumber was picked by running -// echo rocksdb.plain.table | sha1sum +// echo rocksdb.table.plain | sha1sum // and taking the leading 64 bits. -extern const uint64_t kPlainTableMagicNumber = 0x4f3418eb7a8f13b8ull; +extern const uint64_t kPlainTableMagicNumber = 0x8242229663bf9564ull; +extern const uint64_t kLegacyPlainTableMagicNumber = 0x4f3418eb7a8f13b8ull; PlainTableBuilder::PlainTableBuilder(const Options& options, WritableFile* file, @@ -180,7 +181,8 @@ Status PlainTableBuilder::Finish() { } // Write Footer - Footer footer(kPlainTableMagicNumber); + // no need to write out new footer if we're using default checksum + Footer footer(kLegacyPlainTableMagicNumber); footer.set_metaindex_handle(metaindex_block_handle); footer.set_index_handle(BlockHandle::NullBlockHandle()); std::string footer_encoding; diff --git a/table/table_test.cc b/table/table_test.cc index 2e21c5064..dd81baea8 100644 --- a/table/table_test.cc +++ b/table/table_test.cc @@ -44,6 +44,11 @@ namespace rocksdb { +extern const uint64_t kLegacyBlockBasedTableMagicNumber; +extern const uint64_t kLegacyPlainTableMagicNumber; +extern const uint64_t kBlockBasedTableMagicNumber; +extern const uint64_t kPlainTableMagicNumber; + namespace { // Return reverse of "key". @@ -1470,7 +1475,6 @@ TEST(BlockBasedTableTest, BlockCacheLeak) { } } -extern const uint64_t kPlainTableMagicNumber; TEST(PlainTableTest, BasicPlainTableProperties) { PlainTableFactory factory(8, 8, 0); StringSink sink; @@ -1717,6 +1721,83 @@ TEST(Harness, SimpleSpecialKey) { } } +TEST(Harness, FooterTests) { + { + // upconvert legacy block based + std::string encoded; + Footer footer(kLegacyBlockBasedTableMagicNumber); + BlockHandle meta_index(10, 5), index(20, 15); + footer.set_metaindex_handle(meta_index); + footer.set_index_handle(index); + footer.EncodeTo(&encoded); + Footer decoded_footer; + Slice encoded_slice(encoded); + decoded_footer.DecodeFrom(&encoded_slice); + ASSERT_EQ(decoded_footer.table_magic_number(), kBlockBasedTableMagicNumber); + ASSERT_EQ(decoded_footer.checksum(), kCRC32c); + ASSERT_EQ(decoded_footer.metaindex_handle().offset(), meta_index.offset()); + ASSERT_EQ(decoded_footer.metaindex_handle().size(), meta_index.size()); + ASSERT_EQ(decoded_footer.index_handle().offset(), index.offset()); + ASSERT_EQ(decoded_footer.index_handle().size(), index.size()); + } + { + // xxhash block based + std::string encoded; + Footer footer(kBlockBasedTableMagicNumber); + BlockHandle meta_index(10, 5), index(20, 15); + footer.set_metaindex_handle(meta_index); + footer.set_index_handle(index); + footer.set_checksum(kxxHash); + footer.EncodeTo(&encoded); + Footer decoded_footer; + Slice encoded_slice(encoded); + decoded_footer.DecodeFrom(&encoded_slice); + ASSERT_EQ(decoded_footer.table_magic_number(), kBlockBasedTableMagicNumber); + ASSERT_EQ(decoded_footer.checksum(), kxxHash); + ASSERT_EQ(decoded_footer.metaindex_handle().offset(), meta_index.offset()); + ASSERT_EQ(decoded_footer.metaindex_handle().size(), meta_index.size()); + ASSERT_EQ(decoded_footer.index_handle().offset(), index.offset()); + ASSERT_EQ(decoded_footer.index_handle().size(), index.size()); + } + { + // upconvert legacy plain table + std::string encoded; + Footer footer(kLegacyPlainTableMagicNumber); + BlockHandle meta_index(10, 5), index(20, 15); + footer.set_metaindex_handle(meta_index); + footer.set_index_handle(index); + footer.EncodeTo(&encoded); + Footer decoded_footer; + Slice encoded_slice(encoded); + decoded_footer.DecodeFrom(&encoded_slice); + ASSERT_EQ(decoded_footer.table_magic_number(), kPlainTableMagicNumber); + ASSERT_EQ(decoded_footer.checksum(), kCRC32c); + ASSERT_EQ(decoded_footer.metaindex_handle().offset(), meta_index.offset()); + ASSERT_EQ(decoded_footer.metaindex_handle().size(), meta_index.size()); + ASSERT_EQ(decoded_footer.index_handle().offset(), index.offset()); + ASSERT_EQ(decoded_footer.index_handle().size(), index.size()); + } + { + // xxhash block based + std::string encoded; + Footer footer(kPlainTableMagicNumber); + BlockHandle meta_index(10, 5), index(20, 15); + footer.set_metaindex_handle(meta_index); + footer.set_index_handle(index); + footer.set_checksum(kxxHash); + footer.EncodeTo(&encoded); + Footer decoded_footer; + Slice encoded_slice(encoded); + decoded_footer.DecodeFrom(&encoded_slice); + ASSERT_EQ(decoded_footer.table_magic_number(), kPlainTableMagicNumber); + ASSERT_EQ(decoded_footer.checksum(), kxxHash); + ASSERT_EQ(decoded_footer.metaindex_handle().offset(), meta_index.offset()); + ASSERT_EQ(decoded_footer.metaindex_handle().size(), meta_index.size()); + ASSERT_EQ(decoded_footer.index_handle().offset(), index.offset()); + ASSERT_EQ(decoded_footer.index_handle().size(), index.size()); + } +} + } // namespace rocksdb int main(int argc, char** argv) { diff --git a/tools/sst_dump.cc b/tools/sst_dump.cc index 8e00d6913..9a144bb0b 100644 --- a/tools/sst_dump.cc +++ b/tools/sst_dump.cc @@ -20,9 +20,9 @@ #include "rocksdb/table_properties.h" #include "table/block_based_table_factory.h" #include "table/plain_table_factory.h" +#include "table/meta_blocks.h" #include "table/block.h" #include "table/block_builder.h" -#include "table/meta_blocks.h" #include "table/format.h" #include "util/ldb_cmd.h" #include "util/random.h" @@ -84,30 +84,36 @@ extern uint64_t kPlainTableMagicNumber; Status SstFileReader::NewTableReader(const std::string& file_path) { uint64_t magic_number; - Status s = - ReadTableMagicNumber(file_path, options_, soptions_, &magic_number); - if (!s.ok()) { - return s; - } - if (magic_number == kPlainTableMagicNumber) { - soptions_.use_mmap_reads = true; - } - options_.comparator = &internal_comparator_; - s = options_.env->NewRandomAccessFile(file_path, &file_, soptions_); - if (!s.ok()) { - return s; - } + // read table magic number + Footer footer; + + unique_ptr file; uint64_t file_size; - options_.env->GetFileSize(file_path, &file_size); - s = SetTableOptionsByMagicNumber(magic_number, file_.get(), file_size); - if (!s.ok()) { - return s; + Status s = options_.env->NewRandomAccessFile(file_path, &file_, soptions_); + if (s.ok()) { + s = options_.env->GetFileSize(file_path, &file_size); + } + if (s.ok()) { + s = ReadFooterFromFile(file_.get(), file_size, &footer); + } + if (s.ok()) { + magic_number = footer.table_magic_number(); } - s = options_.table_factory->NewTableReader( - options_, soptions_, internal_comparator_, std::move(file_), file_size, - &table_reader_); + if (s.ok()) { + if (magic_number == kPlainTableMagicNumber) { + soptions_.use_mmap_reads = true; + } + options_.comparator = &internal_comparator_; + s = SetTableOptionsByMagicNumber(magic_number, file_.get(), file_size); + } + + if (s.ok()) { + s = options_.table_factory->NewTableReader( + options_, soptions_, internal_comparator_, std::move(file_), file_size, + &table_reader_); + } return s; } diff --git a/util/xxhash.cc b/util/xxhash.cc new file mode 100644 index 000000000..6dfd4b283 --- /dev/null +++ b/util/xxhash.cc @@ -0,0 +1,475 @@ +/* +xxHash - Fast Hash algorithm +Copyright (C) 2012-2014, Yann Collet. +BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +* Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. +* Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +You can contact the author at : +- xxHash source repository : http://code.google.com/p/xxhash/ +*/ + + +//************************************** +// Tuning parameters +//************************************** +// Unaligned memory access is automatically enabled for "common" CPU, such as x86. +// For others CPU, the compiler will be more cautious, and insert extra code to ensure aligned access is respected. +// If you know your target CPU supports unaligned memory access, you want to force this option manually to improve performance. +// You can also enable this parameter if you know your input data will always be aligned (boundaries of 4, for U32). +#if defined(__ARM_FEATURE_UNALIGNED) || defined(__i386) || defined(_M_IX86) || defined(__x86_64__) || defined(_M_X64) +# define XXH_USE_UNALIGNED_ACCESS 1 +#endif + +// XXH_ACCEPT_NULL_INPUT_POINTER : +// If the input pointer is a null pointer, xxHash default behavior is to trigger a memory access error, since it is a bad pointer. +// When this option is enabled, xxHash output for null input pointers will be the same as a null-length input. +// This option has a very small performance cost (only measurable on small inputs). +// By default, this option is disabled. To enable it, uncomment below define : +//#define XXH_ACCEPT_NULL_INPUT_POINTER 1 + +// XXH_FORCE_NATIVE_FORMAT : +// By default, xxHash library provides endian-independent Hash values, based on little-endian convention. +// Results are therefore identical for little-endian and big-endian CPU. +// This comes at a performance cost for big-endian CPU, since some swapping is required to emulate little-endian format. +// Should endian-independance be of no importance for your application, you may set the #define below to 1. +// It will improve speed for Big-endian CPU. +// This option has no impact on Little_Endian CPU. +#define XXH_FORCE_NATIVE_FORMAT 0 + + +//************************************** +// Compiler Specific Options +//************************************** +// Disable some Visual warning messages +#ifdef _MSC_VER // Visual Studio +# pragma warning(disable : 4127) // disable: C4127: conditional expression is constant +#endif + +#ifdef _MSC_VER // Visual Studio +# define FORCE_INLINE static __forceinline +#else +# ifdef __GNUC__ +# define FORCE_INLINE static inline __attribute__((always_inline)) +# else +# define FORCE_INLINE static inline +# endif +#endif + + +//************************************** +// Includes & Memory related functions +//************************************** +#include "xxhash.h" +// Modify the local functions below should you wish to use some other memory related routines +// for malloc(), free() +#include +FORCE_INLINE void* XXH_malloc(size_t s) { return malloc(s); } +FORCE_INLINE void XXH_free (void* p) { free(p); } +// for memcpy() +#include +FORCE_INLINE void* XXH_memcpy(void* dest, const void* src, size_t size) { return memcpy(dest,src,size); } + + +//************************************** +// Basic Types +//************************************** +#if defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L // C99 +# include + typedef uint8_t BYTE; + typedef uint16_t U16; + typedef uint32_t U32; + typedef int32_t S32; + typedef uint64_t U64; +#else + typedef unsigned char BYTE; + typedef unsigned short U16; + typedef unsigned int U32; + typedef signed int S32; + typedef unsigned long long U64; +#endif + +#if defined(__GNUC__) && !defined(XXH_USE_UNALIGNED_ACCESS) +# define _PACKED __attribute__ ((packed)) +#else +# define _PACKED +#endif + +#if !defined(XXH_USE_UNALIGNED_ACCESS) && !defined(__GNUC__) +# ifdef __IBMC__ +# pragma pack(1) +# else +# pragma pack(push, 1) +# endif +#endif + +typedef struct _U32_S { U32 v; } _PACKED U32_S; + +#if !defined(XXH_USE_UNALIGNED_ACCESS) && !defined(__GNUC__) +# pragma pack(pop) +#endif + +#define A32(x) (((U32_S *)(x))->v) + + +//*************************************** +// Compiler-specific Functions and Macros +//*************************************** +#define GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__) + +// Note : although _rotl exists for minGW (GCC under windows), performance seems poor +#if defined(_MSC_VER) +# define XXH_rotl32(x,r) _rotl(x,r) +#else +# define XXH_rotl32(x,r) ((x << r) | (x >> (32 - r))) +#endif + +#if defined(_MSC_VER) // Visual Studio +# define XXH_swap32 _byteswap_ulong +#elif GCC_VERSION >= 403 +# define XXH_swap32 __builtin_bswap32 +#else +static inline U32 XXH_swap32 (U32 x) { + return ((x << 24) & 0xff000000 ) | + ((x << 8) & 0x00ff0000 ) | + ((x >> 8) & 0x0000ff00 ) | + ((x >> 24) & 0x000000ff );} +#endif + + +//************************************** +// Constants +//************************************** +#define PRIME32_1 2654435761U +#define PRIME32_2 2246822519U +#define PRIME32_3 3266489917U +#define PRIME32_4 668265263U +#define PRIME32_5 374761393U + + +//************************************** +// Architecture Macros +//************************************** +typedef enum { XXH_bigEndian=0, XXH_littleEndian=1 } XXH_endianess; +#ifndef XXH_CPU_LITTLE_ENDIAN // It is possible to define XXH_CPU_LITTLE_ENDIAN externally, for example using a compiler switch + static const int one = 1; +# define XXH_CPU_LITTLE_ENDIAN (*(char*)(&one)) +#endif + + +//************************************** +// Macros +//************************************** +#define XXH_STATIC_ASSERT(c) { enum { XXH_static_assert = 1/(!!(c)) }; } // use only *after* variable declarations + + +//**************************** +// Memory reads +//**************************** +typedef enum { XXH_aligned, XXH_unaligned } XXH_alignment; + +FORCE_INLINE U32 XXH_readLE32_align(const U32* ptr, XXH_endianess endian, XXH_alignment align) +{ + if (align==XXH_unaligned) + return endian==XXH_littleEndian ? A32(ptr) : XXH_swap32(A32(ptr)); + else + return endian==XXH_littleEndian ? *ptr : XXH_swap32(*ptr); +} + +FORCE_INLINE U32 XXH_readLE32(const U32* ptr, XXH_endianess endian) { return XXH_readLE32_align(ptr, endian, XXH_unaligned); } + + +//**************************** +// Simple Hash Functions +//**************************** +FORCE_INLINE U32 XXH32_endian_align(const void* input, int len, U32 seed, XXH_endianess endian, XXH_alignment align) +{ + const BYTE* p = (const BYTE*)input; + const BYTE* const bEnd = p + len; + U32 h32; + +#ifdef XXH_ACCEPT_NULL_INPUT_POINTER + if (p==NULL) { len=0; p=(const BYTE*)(size_t)16; } +#endif + + if (len>=16) + { + const BYTE* const limit = bEnd - 16; + U32 v1 = seed + PRIME32_1 + PRIME32_2; + U32 v2 = seed + PRIME32_2; + U32 v3 = seed + 0; + U32 v4 = seed - PRIME32_1; + + do + { + v1 += XXH_readLE32_align((const U32*)p, endian, align) * PRIME32_2; v1 = XXH_rotl32(v1, 13); v1 *= PRIME32_1; p+=4; + v2 += XXH_readLE32_align((const U32*)p, endian, align) * PRIME32_2; v2 = XXH_rotl32(v2, 13); v2 *= PRIME32_1; p+=4; + v3 += XXH_readLE32_align((const U32*)p, endian, align) * PRIME32_2; v3 = XXH_rotl32(v3, 13); v3 *= PRIME32_1; p+=4; + v4 += XXH_readLE32_align((const U32*)p, endian, align) * PRIME32_2; v4 = XXH_rotl32(v4, 13); v4 *= PRIME32_1; p+=4; + } while (p<=limit); + + h32 = XXH_rotl32(v1, 1) + XXH_rotl32(v2, 7) + XXH_rotl32(v3, 12) + XXH_rotl32(v4, 18); + } + else + { + h32 = seed + PRIME32_5; + } + + h32 += (U32) len; + + while (p<=bEnd-4) + { + h32 += XXH_readLE32_align((const U32*)p, endian, align) * PRIME32_3; + h32 = XXH_rotl32(h32, 17) * PRIME32_4 ; + p+=4; + } + + while (p> 15; + h32 *= PRIME32_2; + h32 ^= h32 >> 13; + h32 *= PRIME32_3; + h32 ^= h32 >> 16; + + return h32; +} + + +U32 XXH32(const void* input, int len, U32 seed) +{ +#if 0 + // Simple version, good for code maintenance, but unfortunately slow for small inputs + void* state = XXH32_init(seed); + XXH32_update(state, input, len); + return XXH32_digest(state); +#else + XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; + +# if !defined(XXH_USE_UNALIGNED_ACCESS) + if ((((size_t)input) & 3)) // Input is aligned, let's leverage the speed advantage + { + if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) + return XXH32_endian_align(input, len, seed, XXH_littleEndian, XXH_aligned); + else + return XXH32_endian_align(input, len, seed, XXH_bigEndian, XXH_aligned); + } +# endif + + if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) + return XXH32_endian_align(input, len, seed, XXH_littleEndian, XXH_unaligned); + else + return XXH32_endian_align(input, len, seed, XXH_bigEndian, XXH_unaligned); +#endif +} + + +//**************************** +// Advanced Hash Functions +//**************************** + +struct XXH_state32_t +{ + U64 total_len; + U32 seed; + U32 v1; + U32 v2; + U32 v3; + U32 v4; + int memsize; + char memory[16]; +}; + + +int XXH32_sizeofState() +{ + XXH_STATIC_ASSERT(XXH32_SIZEOFSTATE >= sizeof(struct XXH_state32_t)); // A compilation error here means XXH32_SIZEOFSTATE is not large enough + return sizeof(struct XXH_state32_t); +} + + +XXH_errorcode XXH32_resetState(void* state_in, U32 seed) +{ + struct XXH_state32_t * state = (struct XXH_state32_t *) state_in; + state->seed = seed; + state->v1 = seed + PRIME32_1 + PRIME32_2; + state->v2 = seed + PRIME32_2; + state->v3 = seed + 0; + state->v4 = seed - PRIME32_1; + state->total_len = 0; + state->memsize = 0; + return XXH_OK; +} + + +void* XXH32_init (U32 seed) +{ + void* state = XXH_malloc (sizeof(struct XXH_state32_t)); + XXH32_resetState(state, seed); + return state; +} + + +FORCE_INLINE XXH_errorcode XXH32_update_endian (void* state_in, const void* input, int len, XXH_endianess endian) +{ + struct XXH_state32_t * state = (struct XXH_state32_t *) state_in; + const BYTE* p = (const BYTE*)input; + const BYTE* const bEnd = p + len; + +#ifdef XXH_ACCEPT_NULL_INPUT_POINTER + if (input==NULL) return XXH_ERROR; +#endif + + state->total_len += len; + + if (state->memsize + len < 16) // fill in tmp buffer + { + XXH_memcpy(state->memory + state->memsize, input, len); + state->memsize += len; + return XXH_OK; + } + + if (state->memsize) // some data left from previous update + { + XXH_memcpy(state->memory + state->memsize, input, 16-state->memsize); + { + const U32* p32 = (const U32*)state->memory; + state->v1 += XXH_readLE32(p32, endian) * PRIME32_2; state->v1 = XXH_rotl32(state->v1, 13); state->v1 *= PRIME32_1; p32++; + state->v2 += XXH_readLE32(p32, endian) * PRIME32_2; state->v2 = XXH_rotl32(state->v2, 13); state->v2 *= PRIME32_1; p32++; + state->v3 += XXH_readLE32(p32, endian) * PRIME32_2; state->v3 = XXH_rotl32(state->v3, 13); state->v3 *= PRIME32_1; p32++; + state->v4 += XXH_readLE32(p32, endian) * PRIME32_2; state->v4 = XXH_rotl32(state->v4, 13); state->v4 *= PRIME32_1; p32++; + } + p += 16-state->memsize; + state->memsize = 0; + } + + if (p <= bEnd-16) + { + const BYTE* const limit = bEnd - 16; + U32 v1 = state->v1; + U32 v2 = state->v2; + U32 v3 = state->v3; + U32 v4 = state->v4; + + do + { + v1 += XXH_readLE32((const U32*)p, endian) * PRIME32_2; v1 = XXH_rotl32(v1, 13); v1 *= PRIME32_1; p+=4; + v2 += XXH_readLE32((const U32*)p, endian) * PRIME32_2; v2 = XXH_rotl32(v2, 13); v2 *= PRIME32_1; p+=4; + v3 += XXH_readLE32((const U32*)p, endian) * PRIME32_2; v3 = XXH_rotl32(v3, 13); v3 *= PRIME32_1; p+=4; + v4 += XXH_readLE32((const U32*)p, endian) * PRIME32_2; v4 = XXH_rotl32(v4, 13); v4 *= PRIME32_1; p+=4; + } while (p<=limit); + + state->v1 = v1; + state->v2 = v2; + state->v3 = v3; + state->v4 = v4; + } + + if (p < bEnd) + { + XXH_memcpy(state->memory, p, bEnd-p); + state->memsize = (int)(bEnd-p); + } + + return XXH_OK; +} + +XXH_errorcode XXH32_update (void* state_in, const void* input, int len) +{ + XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; + + if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) + return XXH32_update_endian(state_in, input, len, XXH_littleEndian); + else + return XXH32_update_endian(state_in, input, len, XXH_bigEndian); +} + + + +FORCE_INLINE U32 XXH32_intermediateDigest_endian (void* state_in, XXH_endianess endian) +{ + struct XXH_state32_t * state = (struct XXH_state32_t *) state_in; + const BYTE * p = (const BYTE*)state->memory; + BYTE* bEnd = (BYTE*)state->memory + state->memsize; + U32 h32; + + if (state->total_len >= 16) + { + h32 = XXH_rotl32(state->v1, 1) + XXH_rotl32(state->v2, 7) + XXH_rotl32(state->v3, 12) + XXH_rotl32(state->v4, 18); + } + else + { + h32 = state->seed + PRIME32_5; + } + + h32 += (U32) state->total_len; + + while (p<=bEnd-4) + { + h32 += XXH_readLE32((const U32*)p, endian) * PRIME32_3; + h32 = XXH_rotl32(h32, 17) * PRIME32_4; + p+=4; + } + + while (p> 15; + h32 *= PRIME32_2; + h32 ^= h32 >> 13; + h32 *= PRIME32_3; + h32 ^= h32 >> 16; + + return h32; +} + + +U32 XXH32_intermediateDigest (void* state_in) +{ + XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN; + + if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT) + return XXH32_intermediateDigest_endian(state_in, XXH_littleEndian); + else + return XXH32_intermediateDigest_endian(state_in, XXH_bigEndian); +} + + +U32 XXH32_digest (void* state_in) +{ + U32 h32 = XXH32_intermediateDigest(state_in); + + XXH_free(state_in); + + return h32; +} diff --git a/util/xxhash.h b/util/xxhash.h new file mode 100644 index 000000000..ceff06677 --- /dev/null +++ b/util/xxhash.h @@ -0,0 +1,164 @@ +/* + xxHash - Fast Hash algorithm + Header File + Copyright (C) 2012-2014, Yann Collet. + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - xxHash source repository : http://code.google.com/p/xxhash/ +*/ + +/* Notice extracted from xxHash homepage : + +xxHash is an extremely fast Hash algorithm, running at RAM speed limits. +It also successfully passes all tests from the SMHasher suite. + +Comparison (single thread, Windows Seven 32 bits, using SMHasher on a Core 2 Duo @3GHz) + +Name Speed Q.Score Author +xxHash 5.4 GB/s 10 +CrapWow 3.2 GB/s 2 Andrew +MumurHash 3a 2.7 GB/s 10 Austin Appleby +SpookyHash 2.0 GB/s 10 Bob Jenkins +SBox 1.4 GB/s 9 Bret Mulvey +Lookup3 1.2 GB/s 9 Bob Jenkins +SuperFastHash 1.2 GB/s 1 Paul Hsieh +CityHash64 1.05 GB/s 10 Pike & Alakuijala +FNV 0.55 GB/s 5 Fowler, Noll, Vo +CRC32 0.43 GB/s 9 +MD5-32 0.33 GB/s 10 Ronald L. Rivest +SHA1-32 0.28 GB/s 10 + +Q.Score is a measure of quality of the hash function. +It depends on successfully passing SMHasher test set. +10 is a perfect score. +*/ + +#pragma once + +#if defined (__cplusplus) +extern "C" { +#endif + + +//**************************** +// Type +//**************************** +typedef enum { XXH_OK=0, XXH_ERROR } XXH_errorcode; + + + +//**************************** +// Simple Hash Functions +//**************************** + +unsigned int XXH32 (const void* input, int len, unsigned int seed); + +/* +XXH32() : + Calculate the 32-bits hash of sequence of length "len" stored at memory address "input". + The memory between input & input+len must be valid (allocated and read-accessible). + "seed" can be used to alter the result predictably. + This function successfully passes all SMHasher tests. + Speed on Core 2 Duo @ 3 GHz (single thread, SMHasher benchmark) : 5.4 GB/s + Note that "len" is type "int", which means it is limited to 2^31-1. + If your data is larger, use the advanced functions below. +*/ + + + +//**************************** +// Advanced Hash Functions +//**************************** + +void* XXH32_init (unsigned int seed); +XXH_errorcode XXH32_update (void* state, const void* input, int len); +unsigned int XXH32_digest (void* state); + +/* +These functions calculate the xxhash of an input provided in several small packets, +as opposed to an input provided as a single block. + +It must be started with : +void* XXH32_init() +The function returns a pointer which holds the state of calculation. + +This pointer must be provided as "void* state" parameter for XXH32_update(). +XXH32_update() can be called as many times as necessary. +The user must provide a valid (allocated) input. +The function returns an error code, with 0 meaning OK, and any other value meaning there is an error. +Note that "len" is type "int", which means it is limited to 2^31-1. +If your data is larger, it is recommended to chunk your data into blocks +of size for example 2^30 (1GB) to avoid any "int" overflow issue. + +Finally, you can end the calculation anytime, by using XXH32_digest(). +This function returns the final 32-bits hash. +You must provide the same "void* state" parameter created by XXH32_init(). +Memory will be freed by XXH32_digest(). +*/ + + +int XXH32_sizeofState(); +XXH_errorcode XXH32_resetState(void* state, unsigned int seed); + +#define XXH32_SIZEOFSTATE 48 +typedef struct { long long ll[(XXH32_SIZEOFSTATE+(sizeof(long long)-1))/sizeof(long long)]; } XXH32_stateSpace_t; +/* +These functions allow user application to make its own allocation for state. + +XXH32_sizeofState() is used to know how much space must be allocated for the xxHash 32-bits state. +Note that the state must be aligned to access 'long long' fields. Memory must be allocated and referenced by a pointer. +This pointer must then be provided as 'state' into XXH32_resetState(), which initializes the state. + +For static allocation purposes (such as allocation on stack, or freestanding systems without malloc()), +use the structure XXH32_stateSpace_t, which will ensure that memory space is large enough and correctly aligned to access 'long long' fields. +*/ + + +unsigned int XXH32_intermediateDigest (void* state); +/* +This function does the same as XXH32_digest(), generating a 32-bit hash, +but preserve memory context. +This way, it becomes possible to generate intermediate hashes, and then continue feeding data with XXH32_update(). +To free memory context, use XXH32_digest(), or free(). +*/ + + + +//**************************** +// Deprecated function names +//**************************** +// The following translations are provided to ease code transition +// You are encouraged to no longer this function names +#define XXH32_feed XXH32_update +#define XXH32_result XXH32_digest +#define XXH32_getIntermediateResult XXH32_intermediateDigest + + + +#if defined (__cplusplus) +} +#endif