Post-PR fixes:

* BitHelpers.cpp was removed, corresponding code was moved to the header
* BitIO test as GTest-based test binary
* gtest-based unit test for DoubleDelta and Gorilla codecs
* getLeadingZeroBits from SFINAE to consexpr if
* removed couple of unneeded if's
* Fixed sql-test to use one table
This commit is contained in:
Vasily Nemkov 2019-06-13 17:04:38 +03:00
parent 2b5647ff02
commit 0ebb145d32
17 changed files with 910 additions and 725 deletions

View File

@ -34,29 +34,41 @@ inline size_t roundUpToPowerOfTwoOrZero(size_t n)
template <typename T>
inline std::enable_if_t<std::is_integral_v<T> && (sizeof(T) <= sizeof(unsigned int)), int>
getLeadingZeroBits(T x)
inline size_t getLeadingZeroBits(T x)
{
return x == 0 ? sizeof(x) * 8 : __builtin_clz(x);
if (!x)
return sizeof(x) * 8;
if constexpr (sizeof(T) <= sizeof(unsigned int))
{
return __builtin_clz(x);
}
else if constexpr (sizeof(T) <= sizeof(unsigned long int))
{
return __builtin_clzl(x);
}
else
{
return __builtin_clzll(x);
}
}
template <typename T>
inline std::enable_if_t<std::is_integral_v<T> && (sizeof(T) == sizeof(unsigned long long int)), int>
getLeadingZeroBits(T x)
inline size_t getTrailingZeroBits(T x)
{
return x == 0 ? sizeof(x) * 8 : __builtin_clzll(x);
}
if (!x)
return sizeof(x) * 8;
template <typename T>
inline std::enable_if_t<std::is_integral_v<T> && (sizeof(T) <= sizeof(unsigned int)), int>
getTrailingZeroBits(T x)
{
return x == 0 ? sizeof(x) * 8 : __builtin_ctz(x);
}
template <typename T>
inline std::enable_if_t<std::is_integral_v<T> && (sizeof(T) == sizeof(unsigned long long int)), int>
getTrailingZeroBits(T x)
{
return x == 0 ? sizeof(x) * 8 : __builtin_ctzll(x);
if constexpr (sizeof(T) <= sizeof(unsigned int))
{
return __builtin_ctz(x);
}
else if constexpr (sizeof(T) <= sizeof(unsigned long int))
{
return __builtin_ctzl(x);
}
else
{
return __builtin_ctzll(x);
}
}

View File

@ -48,9 +48,43 @@ UInt32 getCompressedDataSize(UInt8 data_bytes_size, UInt32 uncompressed_size)
return (items_count * max_item_size_bits + 8) / 8;
}
struct WriteSpec
{
const UInt8 prefix_bits;
const UInt8 prefix;
const UInt8 data_bits;
};
template <typename T>
WriteSpec getWriteSpec(const T & value)
{
if (value > -63 && value < 64)
{
return WriteSpec{2, 0b10, 7};
}
else if (value > -255 && value < 256)
{
return WriteSpec{3, 0b110, 9};
}
else if (value > -2047 && value < 2048)
{
return WriteSpec{4, 0b1110, 12};
}
else if (value > std::numeric_limits<Int32>::min() && value < std::numeric_limits<Int32>::max())
{
return WriteSpec{5, 0b11110, 32};
}
else
{
return WriteSpec{5, 0b11111, 64};
}
}
template <typename T, typename DeltaType>
UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest)
{
using UnsignedDeltaType = typename std::make_unsigned<DeltaType>::type;
if (source_size % sizeof(T) != 0)
throw Exception("Cannot compress, data size " + toString(source_size) + " is not aligned to " + toString(sizeof(T)), ErrorCodes::CANNOT_COMPRESS);
const char * source_end = source + source_size;
@ -85,10 +119,9 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest)
WriteBuffer buffer(dest, getCompressedDataSize(sizeof(T), source_size - sizeof(T)*2));
BitWriter writer(buffer);
while (source < source_end)
for (; source < source_end; source += sizeof(T))
{
const T curr_value = unalignedLoad<T>(source);
source += sizeof(curr_value);
const auto delta = curr_value - prev_value;
const DeltaType double_delta = static_cast<DeltaType>(delta - static_cast<T>(prev_delta));
@ -103,37 +136,12 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest)
else
{
const auto sign = std::signbit(double_delta);
const auto abs_value = static_cast<typename std::make_unsigned<DeltaType>::type>(std::abs(double_delta));
if (double_delta > -63 && double_delta < 64)
{
writer.writeBits(2, 0b10);
writer.writeBits(1, sign);
writer.writeBits(6, abs_value);
}
else if (double_delta > -255 && double_delta < 256)
{
writer.writeBits(3, 0b110);
writer.writeBits(1, sign);
writer.writeBits(8, abs_value);
}
else if (double_delta > -2047 && double_delta < 2048)
{
writer.writeBits(4, 0b1110);
writer.writeBits(1, sign);
writer.writeBits(11, abs_value);
}
else if (double_delta > std::numeric_limits<Int32>::min() && double_delta < std::numeric_limits<Int32>::max())
{
writer.writeBits(5, 0b11110);
writer.writeBits(1, sign);
writer.writeBits(31, abs_value);
}
else
{
writer.writeBits(5, 0b11111);
writer.writeBits(1, sign);
writer.writeBits(63, abs_value);
}
const auto abs_value = static_cast<UnsignedDeltaType>(std::abs(double_delta));
const auto write_spec = getWriteSpec(double_delta);
writer.writeBits(write_spec.prefix_bits, write_spec.prefix);
writer.writeBits(1, sign);
writer.writeBits(write_spec.data_bits - 1, abs_value);
}
}
@ -180,13 +188,8 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest)
for (UInt32 items_read = 2; items_read < items_count && !reader.eof(); ++items_read)
{
DeltaType double_delta = 0;
if (reader.readBit() == 0)
if (reader.readBit() == 1)
{
double_delta = 0;
}
else
{
// first bit is 1
const UInt8 data_sizes[] = {6, 8, 11, 31, 63};
UInt8 i = 0;
for (; i < sizeof(data_sizes) - 1; ++i)
@ -203,6 +206,8 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest)
double_delta *= -1;
}
}
// else if first bit is zero, no need to read more data.
const T curr_value = static_cast<T>(prev_value + prev_delta + double_delta);
unalignedStore(dest, curr_value);
dest += sizeof(curr_value);

View File

@ -140,7 +140,6 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest)
writer.writeBits(curr_xored_info.data_bits, xored_data >> curr_xored_info.trailing_zero_bits);
prev_xored_info = curr_xored_info;
}
std::cerr << std::endl;
prev_value = curr_value;
}
@ -182,29 +181,20 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest)
// we have to keep track of items to avoid reading more that there is.
for (UInt32 items_read = 1; items_read < items_count && !reader.eof(); ++items_read)
{
T curr_value{};
binary_value_info curr_xored_info;
T curr_value = prev_value;
binary_value_info curr_xored_info = prev_xored_info;
T xored_data{};
if (reader.readBit() == 0)
if (reader.readBit() == 1)
{
// 0b0 prefix
curr_value = prev_value;
}
else
{
if (reader.readBit() == 0)
{
// 0b10 prefix
curr_xored_info = prev_xored_info;
}
else
if (reader.readBit() == 1)
{
// 0b11 prefix
curr_xored_info.leading_zero_bits = reader.readBits(LEADING_ZEROES_BIT_LENGTH);
curr_xored_info.data_bits = reader.readBits(DATA_BIT_LENGTH);
curr_xored_info.trailing_zero_bits = sizeof(T) * 8 - curr_xored_info.leading_zero_bits - curr_xored_info.data_bits;
}
// else: 0b10 prefix - use prev_xored_info
if (curr_xored_info.leading_zero_bits == 0
&& curr_xored_info.data_bits == 0
@ -218,6 +208,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest)
xored_data <<= curr_xored_info.trailing_zero_bits;
curr_value = prev_value ^ xored_data;
}
// else: 0b0 prefix - use prev_value
unalignedStore(dest, curr_value);
dest += sizeof(curr_value);

View File

@ -29,7 +29,7 @@ namespace ErrorCodes
}
UInt32 ICompressionCodec::compress(char * source, UInt32 source_size, char * dest) const
UInt32 ICompressionCodec::compress(const char * source, UInt32 source_size, char * dest) const
{
dest[0] = getMethodByte();
UInt8 header_size = getHeaderSize();
@ -41,7 +41,7 @@ UInt32 ICompressionCodec::compress(char * source, UInt32 source_size, char * des
}
UInt32 ICompressionCodec::decompress(char * source, UInt32 source_size, char * dest) const
UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, char * dest) const
{
UInt8 method = source[0];
if (method != getMethodByte())

View File

@ -35,10 +35,10 @@ public:
virtual String getCodecDesc() const = 0;
/// Compressed bytes from uncompressed source to dest. Dest should preallocate memory
virtual UInt32 compress(char * source, UInt32 source_size, char * dest) const;
virtual UInt32 compress(const char * source, UInt32 source_size, char * dest) const;
/// Decompress bytes from compressed source to dest. Dest should preallocate memory
virtual UInt32 decompress(char * source, UInt32 source_size, char * dest) const;
virtual UInt32 decompress(const char * source, UInt32 source_size, char * dest) const;
/// Number of bytes, that will be used to compress uncompressed_size bytes with current codec
virtual UInt32 getCompressedReserveSize(UInt32 uncompressed_size) const { return getHeaderSize() + getMaxCompressedDataSize(uncompressed_size); }

View File

@ -0,0 +1,310 @@
#include <Compression/CompressionCodecDoubleDelta.h>
#include <Compression/CompressionCodecGorilla.h>
#include <Core/Types.h>
#include <IO/WriteHelpers.h>
#include <Common/PODArray.h>
#include <gtest/gtest.h>
#include <cmath>
#include <initializer_list>
#include <iomanip>
#include <memory>
#include <vector>
namespace
{
using namespace DB;
template <typename ContainerLeft, typename ContainerRight>
::testing::AssertionResult EqualContainers(const ContainerLeft & left, const ContainerRight & right)
{
const auto MAX_MISMATCHING_ITEMS = 5;
const auto l_size = std::size(left);
const auto r_size = std::size(right);
const auto size = std::min(l_size, r_size);
::testing::AssertionResult result = ::testing::AssertionSuccess();
size_t mismatching_items = 0;
if (l_size != r_size)
{
result = ::testing::AssertionFailure() << "size mismatch" << " expected: " << l_size << " got:" << r_size;
}
for (size_t i = 0; i < size; ++i)
{
if (left[i] != right[i])
{
if (result)
{
result = ::testing::AssertionFailure();
}
result << "pos " << i << ": "
<< " expected: " << std::hex << left[i]
<< " got:" << std::hex << right[i]
<< std::endl;
if (++mismatching_items >= MAX_MISMATCHING_ITEMS)
{
result << "..." << std::endl;
break;
}
}
}
return result;
}
template <typename T>
const char* type_name()
{
return typeid(T).name();
}
template <>
const char* type_name<UInt32>()
{
return "uint32";
}
template <>
const char* type_name<Int32>()
{
return "int32";
}
template <>
const char* type_name<UInt64>()
{
return "uint64";
}
template <>
const char* type_name<Int64>()
{
return "int64";
}
struct CodecTestParam
{
std::vector<char> source_data;
UInt8 data_byte_size;
std::string case_name;
};
std::ostream & operator<<(std::ostream & ostr, const CodecTestParam & param)
{
return ostr << "name: " << param.case_name
<< "\nbyte size: " << static_cast<UInt32>(param.data_byte_size)
<< "\ndata size: " << param.source_data.size();
}
template <typename... Args>
std::string to_string(Args && ... args)
{
std::ostringstream ostr;
(ostr << ... << std::forward<Args>(args));
return ostr.str();
}
template <typename T, typename... Args>
CodecTestParam makeParam(Args && ... args)
{
std::initializer_list<T> vals{static_cast<T>(args)...};
std::vector<char> data(sizeof(T) * std::size(vals));
char * write_pos = data.data();
for (const auto & v : vals)
{
unalignedStore<T>(write_pos, v);
write_pos += sizeof(v);
}
return CodecTestParam{std::move(data), sizeof(T), to_string(data.size(), " predefined values")};
}
template <typename T, size_t Begin = 1, size_t End = 10, typename Generator>
CodecTestParam generateParam(Generator gen, const char* gen_name)
{
static_assert (End >= Begin, "End must be not less than Begin");
std::vector<char> data(sizeof(T) * (End - Begin));
char * write_pos = data.data();
for (size_t i = Begin; i < End; ++i)
{
const T v = gen(static_cast<T>(i));
unalignedStore<T>(write_pos, v);
write_pos += sizeof(v);
}
return CodecTestParam{std::move(data), sizeof(T),
to_string(type_name<T>(), " from ", gen_name, "(", Begin, " => ", End, ")")};
}
void TestTranscoding(ICompressionCodec * codec, const CodecTestParam & param)
{
const auto & source_data = param.source_data;
const UInt32 encoded_max_size = codec->getCompressedReserveSize(source_data.size());
PODArray<char> encoded(encoded_max_size);
const UInt32 encoded_size = codec->compress(source_data.data(), source_data.size(), encoded.data());
encoded.resize(encoded_size);
PODArray<char> decoded(source_data.size());
const UInt32 decoded_size = codec->decompress(encoded.data(), encoded.size(), decoded.data());
decoded.resize(decoded_size);
ASSERT_TRUE(EqualContainers(source_data, decoded));
}
class CodecTest : public ::testing::TestWithParam<CodecTestParam>
{};
TEST_P(CodecTest, DoubleDelta)
{
const auto & param = GetParam();
auto codec = std::make_unique<CompressionCodecDoubleDelta>(param.data_byte_size);
TestTranscoding(codec.get(), param);
}
TEST_P(CodecTest, Gorilla)
{
const auto & param = GetParam();
auto codec = std::make_unique<CompressionCodecGorilla>(param.data_byte_size);
TestTranscoding(codec.get(), param);
}
auto SameValueGenerator = [](auto value)
{
return [=](auto i)
{
return static_cast<decltype(i)>(value);
};
};
auto SequentialGenerator = [](auto stride = 1)
{
return [=](auto i)
{
using ValueType = decltype(i);
return static_cast<ValueType>(stride * i);
};
};
template <typename T>
struct MonotonicGenerator
{
MonotonicGenerator(T stride = 1, size_t max_step = 10)
: prev_value{},
stride(stride),
max_step(max_step)
{}
template <typename U>
U operator()(U i)
{
if (!prev_value.has_value())
{
prev_value = i * stride;
}
const U result = *prev_value + static_cast<T>(stride * (rand() % max_step));
prev_value = result;
return result;
}
std::optional<T> prev_value;
const T stride;
const size_t max_step;
};
auto MinMaxGenerator = [](auto i)
{
if (i % 2 == 0)
{
return std::numeric_limits<decltype(i)>::min();
}
else
{
return std::numeric_limits<decltype(i)>::max();
}
};
auto RandomGenerator = [](auto i) {return static_cast<decltype(i)>(rand());};
INSTANTIATE_TEST_CASE_P(Basic,
CodecTest,
::testing::Values(
makeParam<UInt32>(1, 2, 3, 4),
makeParam<UInt64>(1, 2, 3, 4),
makeParam<Float32>(1.1, 2.2, 3.3, 4.4),
makeParam<Float64>(1.1, 2.2, 3.3, 4.4)
)
);
#define G(generator) generator, #generator
INSTANTIATE_TEST_CASE_P(Same,
CodecTest,
::testing::Values(
generateParam<UInt32>(G(SameValueGenerator(1000))),
generateParam<Int32>(G(SameValueGenerator(-1000))),
generateParam<UInt64>(G(SameValueGenerator(1000))),
generateParam<Int64>(G(SameValueGenerator(-1000))),
generateParam<Float32>(G(SameValueGenerator(M_E))),
generateParam<Float64>(G(SameValueGenerator(M_E)))
)
);
INSTANTIATE_TEST_CASE_P(Sequential,
CodecTest,
::testing::Values(
generateParam<UInt32>(G(SequentialGenerator(1))),
generateParam<Int32>(G(SequentialGenerator(-1))),
generateParam<UInt64>(G(SequentialGenerator(1))),
generateParam<Int64>(G(SequentialGenerator(-1))),
generateParam<Float32>(G(SequentialGenerator(M_E))),
generateParam<Float64>(G(SequentialGenerator(M_E)))
)
);
INSTANTIATE_TEST_CASE_P(Monotonic,
CodecTest,
::testing::Values(
generateParam<UInt32>(G(MonotonicGenerator<UInt32>(1, 5))),
generateParam<Int32>(G(MonotonicGenerator<Int32>(-1, 5))),
generateParam<UInt64>(G(MonotonicGenerator<UInt64>(1, 5))),
generateParam<Int64>(G(MonotonicGenerator<Int64>(-1, 5))),
generateParam<Float32>(G(MonotonicGenerator<Float32>(M_E, 5))),
generateParam<Float64>(G(MonotonicGenerator<Float64>(M_E, 5)))
)
);
INSTANTIATE_TEST_CASE_P(Random,
CodecTest,
::testing::Values(
generateParam<UInt32>(G(RandomGenerator)),
generateParam<UInt64>(G(RandomGenerator))
)
);
INSTANTIATE_TEST_CASE_P(Overflow,
CodecTest,
::testing::Values(
generateParam<UInt32>(G(MinMaxGenerator)),
generateParam<Int32>(G(MinMaxGenerator)),
generateParam<UInt64>(G(MinMaxGenerator)),
generateParam<Int64>(G(MinMaxGenerator))
)
);
}

View File

@ -1,135 +0,0 @@
#include "BitHelpers.h"
#include <cassert>
namespace
{
const DB::UInt8 MAX_BUFFER_SIZE_BITS = 8;
}
namespace DB
{
BitReader::BitReader(ReadBuffer & buf_)
: buf(buf_),
bits_buffer(0),
bits_count(0)
{}
BitReader::~BitReader()
{}
UInt64 BitReader::readBits(UInt8 bits)
{
UInt64 result = 0;
bits = std::min(static_cast<UInt8>(sizeof(result) * 8), bits);
while (bits != 0)
{
if (bits_count == 0)
{
fillBuffer();
if (bits_count == 0)
{
// EOF.
break;
}
}
const auto to_read = std::min(bits, bits_count);
// read MSB bits from bits_bufer
const UInt8 v = bits_buffer >> (bits_count - to_read);
const UInt8 mask = static_cast<UInt8>(~(~0U << to_read));
const UInt8 value = v & mask;
result |= value;
// unset MSB that were read
bits_buffer &= ~(mask << (bits_count - to_read));
bits_count -= to_read;
bits -= to_read;
result <<= std::min(bits, static_cast<UInt8>(sizeof(bits_buffer)*8));
}
return result;
}
UInt8 BitReader::readBit()
{
return static_cast<UInt8>(readBits(1));
}
bool BitReader::eof() const
{
return bits_count == 0 && buf.eof();
}
void BitReader::fillBuffer()
{
auto read = buf.read(reinterpret_cast<char *>(&bits_buffer), MAX_BUFFER_SIZE_BITS/8);
bits_count = static_cast<UInt8>(read) * 8;
}
BitWriter::BitWriter(WriteBuffer & buf_)
: buf(buf_),
bits_buffer(0),
bits_count(0)
{}
BitWriter::~BitWriter()
{
flush();
}
void BitWriter::writeBits(UInt8 bits, UInt64 value)
{
bits = std::min(static_cast<UInt8>(sizeof(value) * 8), bits);
while (bits > 0)
{
auto v = value;
auto to_write = bits;
const UInt8 capacity = MAX_BUFFER_SIZE_BITS - bits_count;
if (capacity < bits)
{
// write MSB:
v >>= bits - capacity;
to_write = capacity;
}
const UInt64 mask = (1 << to_write) - 1;
v &= mask;
assert(v <= 255);
bits_buffer <<= to_write;
bits_buffer |= v;
bits_count += to_write;
if (bits_count < MAX_BUFFER_SIZE_BITS)
break;
doFlush();
bits -= to_write;
}
}
void BitWriter::flush()
{
if (bits_count != 0)
{
bits_buffer <<= (MAX_BUFFER_SIZE_BITS - bits_count);
doFlush();
}
}
void BitWriter::doFlush()
{
buf.write(reinterpret_cast<const char *>(&bits_buffer), MAX_BUFFER_SIZE_BITS/8);
bits_count = 0;
bits_buffer = 0;
}
} // namespace DB

View File

@ -7,10 +7,10 @@
namespace DB
{
/** Reads data from underlying ReadBuffer in bit by bit, max 64 bits at once.
/** Reads data from underlying ReadBuffer bit by bit, max 64 bits at once.
*
* reads MSB bits first, imagine that you have a data:
* 11110000 10101010 00100100 11111111
* 11110000 10101010 00100100 11111110
*
* Given that r is BitReader created with a ReadBuffer that reads from data above:
* r.readBits(3) => 0b111
@ -19,31 +19,84 @@ namespace DB
* r.readBit() => 0b1
* r.readBit() => 0b0
* r.readBits(15) => 0b10001001001111111
* r.readBit() => 0b1
* r.readBit() => 0b0
**/
class BitReader
{
ReadBuffer & buf;
UInt8 bits_buffer;
UInt8 bits_count;
static constexpr UInt8 BIT_BUFFER_SIZE = sizeof(bits_buffer) * 8;
public:
BitReader(ReadBuffer & buf_);
~BitReader();
BitReader(ReadBuffer & buf_)
: buf(buf_),
bits_buffer(0),
bits_count(0)
{}
BitReader(BitReader &&) = default;
~BitReader()
{}
// bits is at most 64
UInt64 readBits(UInt8 bits);
UInt8 readBit();
inline UInt64 readBits(UInt8 bits)
{
UInt64 result = 0;
bits = std::min(static_cast<UInt8>(sizeof(result) * 8), bits);
// true when both bit-buffer and underlying byte-buffer are empty.
bool eof() const;
while (bits != 0)
{
if (bits_count == 0)
{
fillBuffer();
if (bits_count == 0)
{
// EOF.
break;
}
}
const auto to_read = std::min(bits, bits_count);
// read MSB bits from bits_bufer
const UInt8 v = bits_buffer >> (bits_count - to_read);
const UInt8 mask = static_cast<UInt8>(~(~0U << to_read));
const UInt8 value = v & mask;
result |= value;
// unset MSB that were read
bits_buffer &= ~(mask << (bits_count - to_read));
bits_count -= to_read;
bits -= to_read;
result <<= std::min(bits, BIT_BUFFER_SIZE);
}
return result;
}
inline UInt64 peekBits(UInt8 /*bits*/)
{
return 0;
}
inline UInt8 readBit()
{
return static_cast<UInt8>(readBits(1));
}
inline bool eof() const
{
return bits_count == 0 && buf.eof();
}
private:
void fillBuffer();
void fillBuffer()
{
auto read = buf.read(reinterpret_cast<char *>(&bits_buffer), BIT_BUFFER_SIZE / 8);
bits_count = static_cast<UInt8>(read) * 8;
}
};
class BitWriter
@ -53,19 +106,71 @@ class BitWriter
UInt8 bits_buffer;
UInt8 bits_count;
static constexpr UInt8 BIT_BUFFER_SIZE = sizeof(bits_buffer) * 8;
public:
BitWriter(WriteBuffer & buf_);
~BitWriter();
BitWriter(WriteBuffer & buf_)
: buf(buf_),
bits_buffer(0),
bits_count(0)
{}
BitWriter(BitWriter &&) = default;
~BitWriter()
{
flush();
}
// write `size` low bits of the `value`.
void writeBits(UInt8 size, UInt64 value);
inline void writeBits(UInt8 bits, UInt64 value)
{
bits = std::min(static_cast<UInt8>(sizeof(value) * 8), bits);
void flush();
while (bits > 0)
{
auto v = value;
auto to_write = bits;
const UInt8 capacity = BIT_BUFFER_SIZE - bits_count;
if (capacity < bits)
{
// write MSB:
v >>= bits - capacity;
to_write = capacity;
}
const UInt64 mask = (1 << to_write) - 1;
v &= mask;
// assert(v <= 255);
bits_buffer <<= to_write;
bits_buffer |= v;
bits_count += to_write;
if (bits_count < BIT_BUFFER_SIZE)
break;
doFlush();
bits -= to_write;
}
}
inline void flush()
{
if (bits_count != 0)
{
bits_buffer <<= (BIT_BUFFER_SIZE - bits_count);
doFlush();
}
}
private:
void doFlush();
void doFlush()
{
buf.write(reinterpret_cast<const char *>(&bits_buffer), BIT_BUFFER_SIZE / 8);
bits_count = 0;
bits_buffer = 0;
}
};
} // namespace DB
}

View File

@ -82,6 +82,3 @@ target_link_libraries (zlib_ng_bug PRIVATE ${Poco_Foundation_LIBRARY})
if(NOT USE_INTERNAL_POCO_LIBRARY)
target_include_directories(zlib_ng_bug SYSTEM BEFORE PRIVATE ${Poco_INCLUDE_DIRS})
endif()
add_executable(bit_io bit_io.cpp)
target_link_libraries (bit_io PRIVATE clickhouse_common_io)

View File

@ -1,188 +0,0 @@
#include <IO/BitHelpers.h>
#include <Core/Types.h>
#include <IO/MemoryReadWriteBuffer.h>
#include <IO/ReadBufferFromMemory.h>
#include <memory>
#include <iostream>
#include <iomanip>
#include <bitset>
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wunused-const-variable"
#pragma GCC diagnostic ignored "-Wunused-variable"
namespace
{
using namespace DB;
// Intentionally asymetric both byte and word-size to detect read and write inconsistencies
// each prime bit is set to 0.
// v-61 v-53 v-47 v-41 v-37 v-31 v-23 v-17 v-11 v-5
const UInt64 BIT_PATTERN = 0b11101011'11101111'10111010'11101111'10101111'10111010'11101011'10101001;
const UInt8 PRIMES[] = {2, 3, 5, 7, 11, 13, 17, 19, 23, 29, 31, 37, 41, 43, 47, 53, 59, 61};
const UInt8 REPEAT_TIMES = 11;
template <typename T>
std::string bin(const T & value, size_t bits = sizeof(T)*8)
{
static const UInt8 MAX_BITS = sizeof(T)*8;
assert(bits <= MAX_BITS);
return std::bitset<sizeof(T) * 8>(static_cast<unsigned long long>(value))
.to_string().substr(MAX_BITS - bits, bits);
}
template <typename T>
T getBits(UInt8 bits, const T & value)
{
const T mask = ((static_cast<T>(1) << static_cast<T>(bits)) - 1);
return value & mask;
}
std::ostream & dumpBuffer(const char * begin, const char * end, std::ostream * destination, const char* col_sep = " ", const char* row_sep = "\n", const size_t cols_in_row = 8, UInt32 max_bytes = 0xFFFFFFFF)
{
size_t col = 0;
for (auto p = begin; p < end && p - begin < max_bytes; ++p)
{
*destination << bin(*p);
if (++col % cols_in_row == 0)
{
if (row_sep)
*destination << row_sep;
}
else if (col_sep)
{
*destination << col_sep;
}
}
return *destination;
}
std::ostream & dumpBufferContents(BufferBase & buffer, std::ostream * destination, const char* col_sep = " ", const char* row_sep = "\n", const size_t cols_in_row = 8, UInt32 max_bytes = 0xFFFFFFFF)
{
const auto & data = buffer.buffer();
return dumpBuffer(data.begin(), data.end(), destination, col_sep, row_sep, cols_in_row, max_bytes);
}
std::string dumpBufferContents(BufferBase & buffer, const char* col_sep = " ", const char* row_sep = "\n", const size_t cols_in_row = 8)
{
std::stringstream sstr;
dumpBufferContents(buffer, &sstr, col_sep, row_sep, cols_in_row);
return sstr.str();
}
bool test(const std::vector<std::pair<UInt8, UInt64>> & bits_and_vals, const char * expected_buffer_binary = nullptr)
{
MemoryWriteBuffer memory_write_buffer(1024, 1024, 1.5, 20*1024);
{
BitWriter writer(memory_write_buffer);
for (const auto & bv : bits_and_vals)
{
writer.writeBits(bv.first, bv.second);
}
writer.flush();
}
{
auto memory_read_buffer = memory_write_buffer.tryGetReadBuffer();
if (expected_buffer_binary != nullptr)
{
const auto actual_buffer_binary = dumpBufferContents(*memory_read_buffer, " ", " ");
if (actual_buffer_binary != expected_buffer_binary)
{
std::cerr << "Invalid buffer memory after writing\n"
<< "expected: " << strlen(expected_buffer_binary) << "\n" << expected_buffer_binary
<< "\ngot: " << actual_buffer_binary.size() << "\n" << actual_buffer_binary
<< std::endl;
return false;
}
}
BitReader reader(*memory_read_buffer);
int item = 0;
for (const auto & bv : bits_and_vals)
{
const auto expected_value = getBits(bv.first, bv.second);
const auto actual_value = reader.readBits(bv.first);
if (expected_value != actual_value)
{
std::cerr << "Invalid value #" << item << " with " << static_cast<UInt32>(bv.first) << ", " << bin(bv.second) << "\n"
<< "\texpected: " << bin(expected_value) << "\n"
<< "\tgot : " << bin(actual_value) << ".\n\n\nBuffer memory:\n";
dumpBufferContents(*memory_read_buffer, &std::cerr) << std::endl << std::endl;
return false;
}
++item;
}
}
return true;
}
bool primes_test()
{
std::vector<std::pair<UInt8, UInt64>> test_data;
MemoryWriteBuffer memory_write_buffer;
{
for (UInt8 r = 0; r < REPEAT_TIMES; ++r)
{
for (const auto p : PRIMES)
{
test_data.emplace_back(p, BIT_PATTERN);
}
}
}
return test(test_data);
}
void simple_test(UInt8 bits, UInt64 value)
{
test({{bits, value}});
}
} // namespace
int main()
{
UInt32 test_case = 0;
for (const auto p : PRIMES)
{
simple_test(p, 0xFFFFFFFFFFFFFFFF);
std::cout << ++test_case << " with all-ones and " << static_cast<UInt32>(p) << std::endl;
}
for (const auto p : PRIMES)
{
simple_test(p, BIT_PATTERN);
std::cout << ++test_case << " with fancy bit pattern and " << static_cast<UInt32>(p) << std::endl;
}
test({{9, 0xFFFFFFFF}, {9, 0x00}, {9, 0xFFFFFFFF}, {9, 0x00}, {9, 0xFFFFFFFF}},
"11111111 10000000 00111111 11100000 00001111 11111000 ");
test({{7, 0x3f}, {7, 0x3f}, {7, 0x3f}, {7, 0x3f}, {7, 0x3f}, {7, 0x3f}, {7, 0x3f}, {7, 0x3f}, {7, 0x3f}, {3, 0xFFFF}},
"01111110 11111101 11111011 11110111 11101111 11011111 10111111 01111111 11000000 ");
test({{33, 0xFF110d0b07050300}, {33, 0xAAEE29251f1d1713}, });
test({{33, BIT_PATTERN}, {33, BIT_PATTERN}});
std::cout << ++test_case << " primes " << std::endl;
primes_test();
return 0;
}

View File

@ -0,0 +1,188 @@
#include <IO/BitHelpers.h>
#include <Core/Types.h>
#include <IO/MemoryReadWriteBuffer.h>
#include <IO/ReadBufferFromMemory.h>
#include <gtest/gtest.h>
#include <memory>
#include <iostream>
#include <iomanip>
#include <bitset>
//#pragma GCC diagnostic push
//#pragma GCC diagnostic ignored "-Wunused-const-variable"
//#pragma GCC diagnostic ignored "-Wunused-variable"
//#pragma GCC diagnostic ignored "-Wunused-function"
namespace
{
using namespace DB;
// Intentionally asymetric both byte and word-size to detect read and write inconsistencies
// each prime bit is set to 0.
// v-61 v-53 v-47 v-41 v-37 v-31 v-23 v-17 v-11 v-5
const UInt64 BIT_PATTERN = 0b11101011'11101111'10111010'11101111'10101111'10111010'11101011'10101001;
const UInt8 PRIMES[] = {2, 3, 5, 7, 11, 13, 17, 19, 23, 29, 31, 37, 41, 43, 47, 53, 59, 61};
const UInt8 REPEAT_TIMES = 11;
template <typename T>
std::string bin(const T & value, size_t bits = sizeof(T)*8)
{
static const UInt8 MAX_BITS = sizeof(T)*8;
assert(bits <= MAX_BITS);
return std::bitset<sizeof(T) * 8>(static_cast<unsigned long long>(value))
.to_string().substr(MAX_BITS - bits, bits);
}
template <typename T>
T getBits(UInt8 bits, const T & value)
{
const T mask = ((static_cast<T>(1) << static_cast<T>(bits)) - 1);
return value & mask;
}
std::ostream & dumpBuffer(const char * begin,
const char * end,
std::ostream * destination,
const char* col_sep = " ",
const char* row_sep = "\n",
const size_t cols_in_row = 8,
UInt32 max_bytes = 0xFFFFFFFF)
{
size_t col = 0;
for (auto p = begin; p < end && p - begin < max_bytes; ++p)
{
*destination << bin(*p);
if (++col % cols_in_row == 0)
{
if (row_sep)
*destination << row_sep;
}
else if (col_sep)
{
*destination << col_sep;
}
}
return *destination;
}
std::string dumpBufferContents(BufferBase & buf,
const char* col_sep = " ",
const char* row_sep = "\n",
const size_t cols_in_row = 8)
{
std::stringstream sstr;
dumpBuffer(buf.buffer().begin(), buf.buffer().end(), &sstr, col_sep, row_sep, cols_in_row);
return sstr.str();
}
struct TestCaseParameter
{
std::vector<std::pair<UInt8, UInt64>> bits_and_vals;
std::string expected_buffer_binary;
explicit TestCaseParameter(std::vector<std::pair<UInt8, UInt64>> vals, std::string binary = std::string{})
: bits_and_vals(std::move(vals)),
expected_buffer_binary(binary)
{}
};
class BitIO : public ::testing::TestWithParam<TestCaseParameter>
{};
TEST_P(BitIO, WriteAndRead)
{
const auto & param = GetParam();
const auto & bits_and_vals = param.bits_and_vals;
const auto & expected_buffer_binary = param.expected_buffer_binary;
UInt64 max_buffer_size = 0;
for (const auto & bv : bits_and_vals)
{
max_buffer_size += bv.first;
}
max_buffer_size = (max_buffer_size + 8) / 8;
SCOPED_TRACE(max_buffer_size);
MemoryWriteBuffer memory_write_buffer(max_buffer_size * 2, max_buffer_size, 1.5, max_buffer_size);
{
BitWriter writer(memory_write_buffer);
for (const auto & bv : bits_and_vals)
{
writer.writeBits(bv.first, bv.second);
}
writer.flush();
}
{
auto memory_read_buffer = memory_write_buffer.tryGetReadBuffer();
if (expected_buffer_binary != std::string{})
{
const auto actual_buffer_binary = dumpBufferContents(*memory_read_buffer, " ", " ");
ASSERT_EQ(expected_buffer_binary, actual_buffer_binary);
}
BitReader reader(*memory_read_buffer);
int item = 0;
for (const auto & bv : bits_and_vals)
{
const auto actual_value = reader.readBits(bv.first);
ASSERT_EQ(getBits(bv.first, bv.second), actual_value)
<< "item #" << item << ", width: " << static_cast<UInt32>(bv.first)
<< ", value: " << bin(bv.second)
<< ".\n\n\nBuffer memory:\n" << dumpBufferContents(*memory_read_buffer);
++item;
}
}
}
INSTANTIATE_TEST_CASE_P(Simple,
BitIO,
::testing::Values(
TestCaseParameter(
{{9, 0xFFFFFFFF}, {9, 0x00}, {9, 0xFFFFFFFF}, {9, 0x00}, {9, 0xFFFFFFFF}},
"11111111 10000000 00111111 11100000 00001111 11111000 "),
TestCaseParameter(
{{7, 0x3f}, {7, 0x3f}, {7, 0x3f}, {7, 0x3f}, {7, 0x3f}, {7, 0x3f}, {7, 0x3f}, {7, 0x3f}, {7, 0x3f}, {3, 0xFFFF}},
"01111110 11111101 11111011 11110111 11101111 11011111 10111111 01111111 11000000 "),
TestCaseParameter({{33, 0xFF110d0b07050300}, {33, 0xAAEE29251f1d1713}}),
TestCaseParameter({{33, BIT_PATTERN}, {33, BIT_PATTERN}})
));
TestCaseParameter primes_case(UInt8 repeat_times, UInt64 pattern)
{
std::vector<std::pair<UInt8, UInt64>> test_data;
{
for (UInt8 r = 0; r < repeat_times; ++r)
{
for (const auto p : PRIMES)
{
test_data.emplace_back(p, pattern);
}
}
}
return TestCaseParameter(test_data);
}
INSTANTIATE_TEST_CASE_P(Primes,
BitIO,
::testing::Values(
primes_case(11, 0xFFFFFFFFFFFFFFFFULL),
primes_case(11, BIT_PATTERN)
));
} // namespace

View File

@ -1,211 +0,0 @@
DROP TABLE IF EXISTS reference;
DROP TABLE IF EXISTS doubleDelta;
CREATE TABLE reference (
key UInt64,
valueU64 UInt64,
valueU32 UInt32,
valueU16 UInt16,
valueU8 UInt8,
valueI64 Int64,
valueI32 Int32,
valueI16 Int16,
valueI8 Int8,
valueDT DateTime,
valueD Date
) Engine = MergeTree ORDER BY key;
CREATE TABLE doubleDelta (
key UInt64 CODEC(DoubleDelta),
valueU64 UInt64 CODEC(DoubleDelta),
valueU32 UInt32 CODEC(DoubleDelta),
valueU16 UInt16 CODEC(DoubleDelta),
valueU8 UInt8 CODEC(DoubleDelta),
valueI64 Int64 CODEC(DoubleDelta),
valueI32 Int32 CODEC(DoubleDelta),
valueI16 Int16 CODEC(DoubleDelta),
valueI8 Int8 CODEC(DoubleDelta),
valueDT DateTime CODEC(DoubleDelta),
valueD Date CODEC(DoubleDelta)
) Engine = MergeTree ORDER BY key;
-- n^3 covers all double delta storage cases, from small difference between neighbour values (stride) to big.
INSERT INTO reference (key, valueU64, valueU32, valueU16, valueU8, valueI64, valueI32, valueI16, valueI8, valueDT, valueD)
SELECT number as n, n * n * n as v, v, v, v, v, v, v, v, toDateTime(v), toDate(v) FROM system.numbers LIMIT 1, 100;
-- best case - constant stride
INSERT INTO reference (key, valueU64, valueU32, valueU16, valueU8, valueI64, valueI32, valueI16, valueI8, valueDT, valueD)
SELECT number as n, n as v, v, v, v, v, v, v, v, toDateTime(v), toDate(v) FROM system.numbers LIMIT 101, 100;
-- checking for overflow
INSERT INTO reference (key, valueU64, valueI64)
VALUES (201, 18446744073709551616, 9223372036854775808), (202, 0, -9223372036854775808), (203, 18446744073709551616, 9223372036854775808);
-- worst case - random stride
INSERT INTO reference (key, valueU64, valueU32, valueU16, valueU8, valueI64, valueI32, valueI16, valueI8, valueDT, valueD)
SELECT number as n, n + (rand64() - 9223372036854775808)/1000 as v, v, v, v, v, v, v, v, toDateTime(v), toDate(v) FROM system.numbers LIMIT 301, 100;
INSERT INTO doubleDelta SELECT * FROM reference;
-- same number of rows
SELECT a[1] - a[2] FROM (
SELECT groupArray(1) AS a FROM (
SELECT count() FROM reference
UNION ALL
SELECT count() FROM doubleDelta
)
);
SELECT 'U64';
SELECT
key,
r.valueU64, d.valueU64, r.valueU64 - d.valueU64 as dU64
FROM reference as r, doubleDelta as d
WHERE
r.key == d.key
AND
dU64 != 0
ORDER BY r.key
LIMIT 10;
SELECT 'U32';
SELECT
key,
r.valueU32, d.valueU32, r.valueU32 - d.valueU32 as dU32
FROM reference as r, doubleDelta as d
WHERE
r.key == d.key
AND
dU32 != 0
ORDER BY r.key
LIMIT 10;
SELECT 'U16';
SELECT
key,
r.valueU16, d.valueU16, r.valueU16 - d.valueU16 as dU16
FROM reference as r, doubleDelta as d
WHERE
r.key == d.key
AND
dU16 != 0
ORDER BY r.key
LIMIT 10;
SELECT 'U8';
SELECT
key,
r.valueU8, d.valueU8, r.valueU8 - d.valueU8 as dU8
FROM reference as r, doubleDelta as d
WHERE
r.key == d.key
AND
dU8 != 0
ORDER BY r.key
LIMIT 10;
SELECT 'I64';
SELECT
key,
r.valueI64, d.valueI64, r.valueI64 - d.valueI64 as dI64
FROM reference as r, doubleDelta as d
WHERE
r.key == d.key
AND
dI64 != 0
ORDER BY r.key
LIMIT 10;
SELECT 'I32';
SELECT
key,
r.valueI32, d.valueI32, r.valueI32 - d.valueI32 as dI32
FROM reference as r, doubleDelta as d
WHERE
r.key == d.key
AND
dI32 != 0
ORDER BY r.key
LIMIT 10;
SELECT 'I16';
SELECT
key,
r.valueI16, d.valueI16, r.valueI16 - d.valueI16 as dI16
FROM reference as r, doubleDelta as d
WHERE
r.key == d.key
AND
dI16 != 0
ORDER BY r.key
LIMIT 10;
SELECT 'I8';
SELECT
key,
r.valueI8, d.valueI8, r.valueI8 - d.valueI8 as dI8
FROM reference as r, doubleDelta as d
WHERE
r.key == d.key
AND
dI8 != 0
ORDER BY r.key
LIMIT 10;
SELECT 'DT';
SELECT
key,
r.valueDT, d.valueDT, r.valueDT - d.valueDT as dDT
FROM reference as r, doubleDelta as d
WHERE
r.key == d.key
AND
dDT != 0
ORDER BY r.key
LIMIT 10;
SELECT 'D';
SELECT
key,
r.valueD, d.valueD, r.valueD - d.valueD as dD
FROM reference as r, doubleDelta as d
WHERE
r.key == d.key
AND
dD != 0
ORDER BY r.key
LIMIT 10;
-- Compatibity with other codecs
DROP TABLE IF EXISTS dd_lz4_codec;
CREATE TABLE dd_lz4_codec (
key UInt64 CODEC(DoubleDelta, LZ4),
valueU64 UInt64 CODEC(DoubleDelta, LZ4),
valueU32 UInt32 CODEC(DoubleDelta, LZ4),
valueU16 UInt16 CODEC(DoubleDelta, LZ4),
valueU8 UInt8 CODEC(DoubleDelta, LZ4),
valueI64 Int64 CODEC(DoubleDelta, LZ4),
valueI32 Int32 CODEC(DoubleDelta, LZ4),
valueI16 Int16 CODEC(DoubleDelta, LZ4),
valueI8 Int8 CODEC(DoubleDelta, LZ4),
valueDT DateTime CODEC(DoubleDelta, LZ4),
valueD Date CODEC(DoubleDelta, LZ4)
) Engine = MergeTree ORDER BY key;
INSERT INTO dd_lz4_codec SELECT * FROM reference;
DROP TABLE IF EXISTS reference;
DROP TABLE IF EXISTS doubleDelta;
DROP TABLE IF EXISTS dd_lz4_codec;

View File

@ -1,89 +0,0 @@
DROP DATABASE IF EXISTS codec_test;
CREATE DATABASE codec_test;
USE codec_test;
DROP TABLE IF EXISTS reference;
DROP TABLE IF EXISTS gorilla;
CREATE TABLE reference (
key UInt64,
valueF64 Float64,
valueF32 Float32
) Engine = MergeTree ORDER BY key;
CREATE TABLE gorilla (
key UInt64,
valueF64 Float64 CODEC(Gorilla),
valueF32 Float32 CODEC(Gorilla)
) Engine = MergeTree ORDER BY key;
-- best case - same value
INSERT INTO reference (key, valueF64, valueF32)
SELECT number AS n, e() AS v, v FROM system.numbers LIMIT 1, 100;
-- good case - values that grow insignificantly
INSERT INTO reference (key, valueF64, valueF32)
SELECT number AS n, log2(n) AS v, v FROM system.numbers LIMIT 1001, 100;
-- bad case - values differ significantly
INSERT INTO reference (key, valueF64, valueF32)
SELECT number AS n, n*sqrt(n) AS v, v FROM system.numbers LIMIT 2001, 100;
-- worst case - random values
INSERT INTO reference (key, valueF64, valueF32)
SELECT number AS n, (rand64() - 9223372036854775808)/10000000000000 AS v, v FROM system.numbers LIMIT 3001, 100;
INSERT INTO gorilla SELECT * FROM reference;
SELECT a[1] - a[2] FROM (
SELECT groupArray(1) AS a FROM (
SELECT count() FROM reference
UNION ALL
SELECT count() FROM gorilla
)
);
-- These floating-point values are expected to be BINARY equal, hence comparing the values are safe.
SELECT 'F64';
SELECT
key,
r.valueF64, g.valueF64, r.valueF64 - g.valueF64 AS dU64
FROM reference AS r, gorilla AS g
WHERE
r.key == g.key
AND
dU64 != 0
ORDER BY r.key
LIMIT 10;
SELECT 'F32';
SELECT
key,
r.valueF32, g.valueF32, r.valueF32 - g.valueF32 AS dU32
FROM reference AS r, gorilla AS g
WHERE
r.key == g.key
AND
dU32 != 0
ORDER BY r.key
LIMIT 10;
-- Compatibity with other codecs
DROP TABLE IF EXISTS g_lz4_codec;
CREATE TABLE g_lz4_codec (
key UInt64 CODEC(Gorilla, LZ4),
valueU64 Float64 CODEC(Gorilla, LZ4),
valueU32 Float32 CODEC(Gorilla, LZ4)
) Engine = MergeTree ORDER BY key;
INSERT INTO g_lz4_codec SELECT * FROM reference;
DROP TABLE IF EXISTS reference;
DROP TABLE IF EXISTS gorilla;
DROP TABLE IF EXISTS g_lz4_codec;

View File

@ -0,0 +1,151 @@
USE test;
DROP TABLE IF EXISTS codecTest;
CREATE TABLE codecTest (
key UInt64,
ref_valueU64 UInt64,
ref_valueU32 UInt32,
ref_valueU16 UInt16,
ref_valueU8 UInt8,
ref_valueI64 Int64,
ref_valueI32 Int32,
ref_valueI16 Int16,
ref_valueI8 Int8,
ref_valueDT DateTime,
ref_valueD Date,
valueU64 UInt64 CODEC(DoubleDelta),
valueU32 UInt32 CODEC(DoubleDelta),
valueU16 UInt16 CODEC(DoubleDelta),
valueU8 UInt8 CODEC(DoubleDelta),
valueI64 Int64 CODEC(DoubleDelta),
valueI32 Int32 CODEC(DoubleDelta),
valueI16 Int16 CODEC(DoubleDelta),
valueI8 Int8 CODEC(DoubleDelta),
valueDT DateTime CODEC(DoubleDelta),
valueD Date CODEC(DoubleDelta)
) Engine = MergeTree ORDER BY key;
-- checking for overflow
INSERT INTO codecTest (key, ref_valueU64, valueU64, ref_valueI64, valueI64)
VALUES (101, 18446744073709551615, 18446744073709551615, 9223372036854775807, 9223372036854775807), (202, 0, 0, -9223372036854775808, -9223372036854775808), (203, 18446744073709551615, 18446744073709551615, 9223372036854775807, 9223372036854775807);
-- n^3 covers all double delta storage cases, from small difference between neighbouref_values (stride) to big.
INSERT INTO codecTest (key, ref_valueU64, valueU64, ref_valueU32, valueU32, ref_valueU16, valueU16, ref_valueU8, valueU8, ref_valueI64, valueI64, ref_valueI32, valueI32, ref_valueI16, valueI16, ref_valueI8, valueI8, ref_valueDT, valueDT, ref_valueD, valueD)
SELECT number as n, n * n * n as v, v, v, v, v, v, v, v, v, v, v, v, v, v, v, v, toDateTime(v), toDateTime(v), toDate(v), toDate(v)
FROM system.numbers LIMIT 101, 100;
-- best case - constant stride
INSERT INTO codecTest (key, ref_valueU64, valueU64, ref_valueU32, valueU32, ref_valueU16, valueU16, ref_valueU8, valueU8, ref_valueI64, valueI64, ref_valueI32, valueI32, ref_valueI16, valueI16, ref_valueI8, valueI8, ref_valueDT, valueDT, ref_valueD, valueD)
SELECT number as n, n as v, v, v, v, v, v, v, v, v, v, v, v, v, v, v, v, toDateTime(v), toDateTime(v), toDate(v), toDate(v)
FROM system.numbers LIMIT 201, 100;
-- worst case - random stride
INSERT INTO codecTest (key, ref_valueU64, valueU64, ref_valueU32, valueU32, ref_valueU16, valueU16, ref_valueU8, valueU8, ref_valueI64, valueI64, ref_valueI32, valueI32, ref_valueI16, valueI16, ref_valueI8, valueI8, ref_valueDT, valueDT, ref_valueD, valueD)
SELECT number as n, n + (rand64() - 9223372036854775807)/1000 as v, v, v, v, v, v, v, v, v, v, v, v, v, v, v, v, toDateTime(v), toDateTime(v), toDate(v), toDate(v)
FROM system.numbers LIMIT 301, 100;
SELECT 'U64';
SELECT
key,
ref_valueU64, valueU64, ref_valueU64 - valueU64 as dU64
FROM codecTest
WHERE
dU64 != 0
LIMIT 10;
SELECT 'U32';
SELECT
key,
ref_valueU32, valueU32, ref_valueU32 - valueU32 as dU32
FROM codecTest
WHERE
dU32 != 0
LIMIT 10;
SELECT 'U16';
SELECT
key,
ref_valueU16, valueU16, ref_valueU16 - valueU16 as dU16
FROM codecTest
WHERE
dU16 != 0
LIMIT 10;
SELECT 'U8';
SELECT
key,
ref_valueU8, valueU8, ref_valueU8 - valueU8 as dU8
FROM codecTest
WHERE
dU8 != 0
LIMIT 10;
SELECT 'I64';
SELECT
key,
ref_valueI64, valueI64, ref_valueI64 - valueI64 as dI64
FROM codecTest
WHERE
dI64 != 0
LIMIT 10;
SELECT 'I32';
SELECT
key,
ref_valueI32, valueI32, ref_valueI32 - valueI32 as dI32
FROM codecTest
WHERE
dI32 != 0
LIMIT 10;
SELECT 'I16';
SELECT
key,
ref_valueI16, valueI16, ref_valueI16 - valueI16 as dI16
FROM codecTest
WHERE
dI16 != 0
LIMIT 10;
SELECT 'I8';
SELECT
key,
ref_valueI8, valueI8, ref_valueI8 - valueI8 as dI8
FROM codecTest
WHERE
dI8 != 0
LIMIT 10;
SELECT 'DT';
SELECT
key,
ref_valueDT, valueDT, ref_valueDT - valueDT as dDT
FROM codecTest
WHERE
dDT != 0
LIMIT 10;
SELECT 'D';
SELECT
key,
ref_valueD, valueD, ref_valueD - valueD as dD
FROM codecTest
WHERE
dD != 0
LIMIT 10;
DROP TABLE IF EXISTS codecTest;

View File

@ -0,0 +1,51 @@
USE test;
DROP TABLE IF EXISTS codecTest;
CREATE TABLE codecTest (
key UInt64,
ref_valueF64 Float64,
ref_valueF32 Float32,
valueF64 Float64 CODEC(Gorilla),
valueF32 Float32 CODEC(Gorilla)
) Engine = MergeTree ORDER BY key;
-- best case - same value
INSERT INTO codecTest (key, ref_valueF64, valueF64, ref_valueF32, valueF32)
SELECT number AS n, e() AS v, v, v, v FROM system.numbers LIMIT 1, 100;
-- good case - values that grow insignificantly
INSERT INTO codecTest (key, ref_valueF64, valueF64, ref_valueF32, valueF32)
SELECT number AS n, log2(n) AS v, v, v, v FROM system.numbers LIMIT 101, 100;
-- bad case - values differ significantly
INSERT INTO codecTest (key, ref_valueF64, valueF64, ref_valueF32, valueF32)
SELECT number AS n, n*sqrt(n) AS v, v, v, v FROM system.numbers LIMIT 201, 100;
-- worst case - random values
INSERT INTO codecTest (key, ref_valueF64, valueF64, ref_valueF32, valueF32)
SELECT number AS n, (rand64() - 9223372036854775808)/10000000000000 AS v, v, v, v FROM system.numbers LIMIT 3001, 100;
-- These floating-point values are expected to be BINARY equal, hence comparing the values are safe.
SELECT 'F64';
SELECT
key,
ref_valueF64, valueF64, ref_valueF64 - valueF64 AS dF64
FROM codecTest
WHERE
dF64 != 0
LIMIT 10;
SELECT 'F32';
SELECT
key,
ref_valueF32, valueF32, ref_valueF32 - valueF32 AS dF32
FROM codecTest
WHERE
dF32 != 0
LIMIT 10;
DROP TABLE IF EXISTS codecTest;