mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Use information about type in Delta codec to choose correct delta size if possible
This commit is contained in:
parent
37c78aa3d2
commit
361d26c643
@ -123,9 +123,16 @@ void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_
|
||||
void registerCodecDelta(CompressionCodecFactory & factory)
|
||||
{
|
||||
UInt8 method_code = UInt8(CompressionMethodByte::Delta);
|
||||
factory.registerCompressionCodec("Delta", method_code, [&](const ASTPtr & arguments) -> CompressionCodecPtr
|
||||
factory.registerCompressionCodecWithType("Delta", method_code, [&](const ASTPtr & arguments, DataTypePtr column_type) -> CompressionCodecPtr
|
||||
{
|
||||
UInt8 delta_bytes_size = CompressionCodecDelta::DEFAULT_BYTES_SIZE;
|
||||
UInt8 delta_bytes_size = 1;
|
||||
if (column_type && column_type->haveMaximumSizeOfValue())
|
||||
{
|
||||
size_t max_size = column_type->getSizeOfValueInMemory();
|
||||
if (max_size == 1 || max_size == 2 || max_size == 4 || max_size == 8)
|
||||
delta_bytes_size = static_cast<UInt8>(max_size);
|
||||
}
|
||||
|
||||
if (arguments && !arguments->children.empty())
|
||||
{
|
||||
if (arguments->children.size() > 1)
|
||||
@ -138,7 +145,6 @@ void registerCodecDelta(CompressionCodecFactory & factory)
|
||||
throw Exception("Delta value for delta codec can be 1, 2, 4 or 8, given " + toString(user_bytes_size), ErrorCodes::ILLEGAL_CODEC_PARAMETER);
|
||||
delta_bytes_size = static_cast<UInt8>(user_bytes_size);
|
||||
}
|
||||
|
||||
return std::make_shared<CompressionCodecDelta>(delta_bytes_size);
|
||||
});
|
||||
}
|
||||
|
@ -7,8 +7,6 @@ namespace DB
|
||||
class CompressionCodecDelta : public ICompressionCodec
|
||||
{
|
||||
public:
|
||||
static constexpr auto DEFAULT_BYTES_SIZE = 1;
|
||||
|
||||
CompressionCodecDelta(UInt8 delta_bytes_size_);
|
||||
|
||||
UInt8 getMethodByte() const override;
|
||||
@ -20,7 +18,7 @@ protected:
|
||||
|
||||
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
|
||||
|
||||
UInt32 getCompressedDataSize(UInt32 uncompressed_size) const override { return uncompressed_size + 2; }
|
||||
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override { return uncompressed_size + 2; }
|
||||
private:
|
||||
const UInt8 delta_bytes_size;
|
||||
};
|
||||
|
@ -36,7 +36,7 @@ String CompressionCodecLZ4::getCodecDesc() const
|
||||
return "LZ4";
|
||||
}
|
||||
|
||||
UInt32 CompressionCodecLZ4::getCompressedDataSize(UInt32 uncompressed_size) const
|
||||
UInt32 CompressionCodecLZ4::getMaxCompressedDataSize(UInt32 uncompressed_size) const
|
||||
{
|
||||
return LZ4_COMPRESSBOUND(uncompressed_size);
|
||||
}
|
||||
|
@ -24,7 +24,7 @@ protected:
|
||||
private:
|
||||
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
|
||||
|
||||
UInt32 getCompressedDataSize(UInt32 uncompressed_size) const override;
|
||||
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
|
||||
|
||||
mutable LZ4::PerformanceStatistics lz4_stat;
|
||||
};
|
||||
|
@ -43,7 +43,7 @@ String CompressionCodecMultiple::getCodecDesc() const
|
||||
return codec_desc;
|
||||
}
|
||||
|
||||
UInt32 CompressionCodecMultiple::getCompressedDataSize(UInt32 uncompressed_size) const
|
||||
UInt32 CompressionCodecMultiple::getMaxCompressedDataSize(UInt32 uncompressed_size) const
|
||||
{
|
||||
UInt32 compressed_size = uncompressed_size;
|
||||
for (auto & codec : codecs)
|
||||
|
@ -15,7 +15,7 @@ public:
|
||||
|
||||
String getCodecDesc() const override;
|
||||
|
||||
UInt32 getCompressedDataSize(UInt32 uncompressed_size) const override;
|
||||
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
|
||||
|
||||
protected:
|
||||
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
|
||||
|
@ -31,7 +31,7 @@ String CompressionCodecZSTD::getCodecDesc() const
|
||||
return "ZSTD(" + toString(level) + ")";
|
||||
}
|
||||
|
||||
UInt32 CompressionCodecZSTD::getCompressedDataSize(UInt32 uncompressed_size) const
|
||||
UInt32 CompressionCodecZSTD::getMaxCompressedDataSize(UInt32 uncompressed_size) const
|
||||
{
|
||||
return ZSTD_compressBound(uncompressed_size);
|
||||
}
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
|
||||
String getCodecDesc() const override;
|
||||
|
||||
UInt32 getCompressedDataSize(UInt32 uncompressed_size) const override;
|
||||
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
|
||||
|
||||
protected:
|
||||
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
|
||||
|
@ -55,7 +55,7 @@ CompressionCodecPtr CompressionCodecFactory::get(const std::vector<CodecNameWith
|
||||
return std::make_shared<CompressionCodecMultiple>(result);
|
||||
}
|
||||
|
||||
CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast) const
|
||||
CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, DataTypePtr column_type) const
|
||||
{
|
||||
if (const auto * func = typeid_cast<const ASTFunction *>(ast.get()))
|
||||
{
|
||||
@ -64,9 +64,9 @@ CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast) const
|
||||
for (const auto & inner_codec_ast : func->arguments->children)
|
||||
{
|
||||
if (const auto * family_name = typeid_cast<const ASTIdentifier *>(inner_codec_ast.get()))
|
||||
codecs.emplace_back(getImpl(family_name->name, {}));
|
||||
codecs.emplace_back(getImpl(family_name->name, {}, column_type));
|
||||
else if (const auto * ast_func = typeid_cast<const ASTFunction *>(inner_codec_ast.get()))
|
||||
codecs.emplace_back(getImpl(ast_func->name, ast_func->arguments));
|
||||
codecs.emplace_back(getImpl(ast_func->name, ast_func->arguments, column_type));
|
||||
else
|
||||
throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||
}
|
||||
@ -87,10 +87,11 @@ CompressionCodecPtr CompressionCodecFactory::get(const UInt8 byte_code) const
|
||||
if (family_code_and_creator == family_code_with_codec.end())
|
||||
throw Exception("Unknown codec family code : " + toString(byte_code), ErrorCodes::UNKNOWN_CODEC);
|
||||
|
||||
return family_code_and_creator->second({});
|
||||
return family_code_and_creator->second({}, nullptr);
|
||||
}
|
||||
|
||||
CompressionCodecPtr CompressionCodecFactory::getImpl(const String & family_name, const ASTPtr & arguments) const
|
||||
|
||||
CompressionCodecPtr CompressionCodecFactory::getImpl(const String & family_name, const ASTPtr & arguments, DataTypePtr column_type) const
|
||||
{
|
||||
if (family_name == "Multiple")
|
||||
throw Exception("Codec MULTIPLE cannot be specified directly", ErrorCodes::UNKNOWN_CODEC);
|
||||
@ -100,10 +101,13 @@ CompressionCodecPtr CompressionCodecFactory::getImpl(const String & family_name,
|
||||
if (family_and_creator == family_name_with_codec.end())
|
||||
throw Exception("Unknown codec family: " + family_name, ErrorCodes::UNKNOWN_CODEC);
|
||||
|
||||
return family_and_creator->second(arguments);
|
||||
return family_and_creator->second(arguments, column_type);
|
||||
}
|
||||
|
||||
void CompressionCodecFactory::registerCompressionCodec(const String & family_name, std::optional<UInt8> byte_code, Creator creator)
|
||||
void CompressionCodecFactory::registerCompressionCodecWithType(
|
||||
const String & family_name,
|
||||
std::optional<UInt8> byte_code,
|
||||
CreatorWithType creator)
|
||||
{
|
||||
if (creator == nullptr)
|
||||
throw Exception("CompressionCodecFactory: the codec family " + family_name + " has been provided a null constructor",
|
||||
@ -117,8 +121,18 @@ void CompressionCodecFactory::registerCompressionCodec(const String & family_nam
|
||||
throw Exception("CompressionCodecFactory: the codec family name '" + family_name + "' is not unique", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
void CompressionCodecFactory::registerSimpleCompressionCodec(const String & family_name, std::optional<UInt8> byte_code,
|
||||
std::function<CompressionCodecPtr()> creator)
|
||||
void CompressionCodecFactory::registerCompressionCodec(const String & family_name, std::optional<UInt8> byte_code, Creator creator)
|
||||
{
|
||||
registerCompressionCodecWithType(family_name, byte_code, [family_name, creator](const ASTPtr & ast, DataTypePtr /* data_type */)
|
||||
{
|
||||
return creator(ast);
|
||||
});
|
||||
}
|
||||
|
||||
void CompressionCodecFactory::registerSimpleCompressionCodec(
|
||||
const String & family_name,
|
||||
std::optional<UInt8> byte_code,
|
||||
SimpleCreator creator)
|
||||
{
|
||||
registerCompressionCodec(family_name, byte_code, [family_name, creator](const ASTPtr & ast)
|
||||
{
|
||||
|
@ -29,16 +29,19 @@ class CompressionCodecFactory final : public ext::singleton<CompressionCodecFact
|
||||
{
|
||||
protected:
|
||||
using Creator = std::function<CompressionCodecPtr(const ASTPtr & parameters)>;
|
||||
using CreatorWithType = std::function<CompressionCodecPtr(const ASTPtr & parameters, DataTypePtr column_type)>;
|
||||
using SimpleCreator = std::function<CompressionCodecPtr()>;
|
||||
using CompressionCodecsDictionary = std::unordered_map<String, Creator>;
|
||||
using CompressionCodecsCodeDictionary = std::unordered_map<UInt8, Creator>;
|
||||
using CompressionCodecsDictionary = std::unordered_map<String, CreatorWithType>;
|
||||
using CompressionCodecsCodeDictionary = std::unordered_map<UInt8, CreatorWithType>;
|
||||
public:
|
||||
|
||||
/// Return default codec (currently LZ4)
|
||||
CompressionCodecPtr getDefaultCodec() const;
|
||||
|
||||
/// Get codec by AST
|
||||
CompressionCodecPtr get(const ASTPtr & ast) const;
|
||||
/// Get codec by AST and possible column_type
|
||||
/// some codecs can use information about type to improve inner settings
|
||||
/// but every codec should be able to work without information about type
|
||||
CompressionCodecPtr get(const ASTPtr & ast, DataTypePtr column_type=nullptr) const;
|
||||
|
||||
/// Get codec by method byte (no params available)
|
||||
CompressionCodecPtr get(const UInt8 byte_code) const;
|
||||
@ -47,6 +50,9 @@ public:
|
||||
CompressionCodecPtr get(const String & family_name, std::optional<int> level) const;
|
||||
|
||||
CompressionCodecPtr get(const std::vector<CodecNameWithLevel> & codecs) const;
|
||||
|
||||
/// Register codec with parameters and column type
|
||||
void registerCompressionCodecWithType(const String & family_name, std::optional<UInt8> byte_code, CreatorWithType creator);
|
||||
/// Register codec with parameters
|
||||
void registerCompressionCodec(const String & family_name, std::optional<UInt8> byte_code, Creator creator);
|
||||
|
||||
@ -54,7 +60,7 @@ public:
|
||||
void registerSimpleCompressionCodec(const String & family_name, std::optional<UInt8> byte_code, SimpleCreator creator);
|
||||
|
||||
protected:
|
||||
CompressionCodecPtr getImpl(const String & family_name, const ASTPtr & arguments) const;
|
||||
CompressionCodecPtr getImpl(const String & family_name, const ASTPtr & arguments, DataTypePtr column_type) const;
|
||||
|
||||
private:
|
||||
CompressionCodecsDictionary family_name_with_codec;
|
||||
|
@ -41,7 +41,7 @@ public:
|
||||
virtual UInt32 decompress(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() + getCompressedDataSize(uncompressed_size); }
|
||||
virtual UInt32 getCompressedReserveSize(UInt32 uncompressed_size) const { return getHeaderSize() + getMaxCompressedDataSize(uncompressed_size); }
|
||||
|
||||
/// Some codecs (LZ4, for example) require additional bytes at end of buffer
|
||||
virtual UInt32 getAdditionalSizeAtTheEndOfBuffer() const { return 0; }
|
||||
@ -61,7 +61,7 @@ public:
|
||||
protected:
|
||||
|
||||
/// Return size of compressed data without header
|
||||
virtual UInt32 getCompressedDataSize(UInt32 uncompressed_size) const { return uncompressed_size; }
|
||||
virtual UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const { return uncompressed_size; }
|
||||
|
||||
/// Actually compress data, without header
|
||||
virtual UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const = 0;
|
||||
|
@ -195,9 +195,11 @@ static ColumnsDeclarationAndModifiers parseColumns(const ASTExpressionList & col
|
||||
{
|
||||
auto & col_decl = typeid_cast<ASTColumnDeclaration &>(*ast);
|
||||
|
||||
DataTypePtr column_type = nullptr;
|
||||
if (col_decl.type)
|
||||
{
|
||||
columns.emplace_back(col_decl.name, DataTypeFactory::instance().get(col_decl.type));
|
||||
column_type = DataTypeFactory::instance().get(col_decl.type);
|
||||
columns.emplace_back(col_decl.name, column_type);
|
||||
}
|
||||
else
|
||||
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions
|
||||
@ -228,7 +230,7 @@ static ColumnsDeclarationAndModifiers parseColumns(const ASTExpressionList & col
|
||||
|
||||
if (col_decl.codec)
|
||||
{
|
||||
auto codec = CompressionCodecFactory::instance().get(col_decl.codec);
|
||||
auto codec = CompressionCodecFactory::instance().get(col_decl.codec, column_type);
|
||||
codecs.emplace(col_decl.name, codec);
|
||||
}
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
2018-01-01 1
|
||||
2018-01-01 2
|
||||
CODEC(ZSTD)
|
||||
CODEC(ZSTD(1))
|
||||
2018-01-01 1 default_value
|
||||
2018-01-01 2 default_value
|
||||
2018-01-01 3 3
|
||||
@ -18,7 +18,7 @@ CODEC(NONE)
|
||||
2018-01-01 4 4
|
||||
2018-01-01 5 5
|
||||
2018-01-01 6 6
|
||||
CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE)
|
||||
CODEC(ZSTD(1), LZ4HC(0), LZ4, LZ4, NONE)
|
||||
2018-01-01 1 default_value
|
||||
2018-01-01 2 default_value
|
||||
2018-01-01 3 3
|
||||
@ -27,7 +27,7 @@ CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE)
|
||||
2018-01-01 6 6
|
||||
2018-01-01 7 7
|
||||
2018-01-01 8 8
|
||||
CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE)
|
||||
CODEC(NONE, LZ4, LZ4HC, ZSTD)
|
||||
CODEC(ZSTD(1), LZ4HC(0), LZ4, LZ4, NONE)
|
||||
CODEC(NONE, LZ4, LZ4HC(0), ZSTD(1))
|
||||
2
|
||||
1
|
||||
|
@ -14,3 +14,4 @@ CREATE TABLE test.compression_codec_multiple_more_types ( id Decimal(38, 13) COD
|
||||
7.1000000000000 xxxxxxxxxxxx [127] ['Henry']
|
||||
!
|
||||
222
|
||||
CREATE TABLE test.test_default_delta ( id UInt64 CODEC(Delta(8)), data String CODEC(Delta(1)), somedate Date CODEC(Delta(2)), somenum Float64 CODEC(Delta(8)), somestr FixedString(3) CODEC(Delta(1)), othernum Int64 CODEC(Delta(8)), yetothernum Float32 CODEC(Delta(4)), `ddd.age` Array(UInt8) CODEC(Delta(1)), `ddd.Name` Array(String) CODEC(Delta(1)), `ddd.OName` Array(String) CODEC(Delta(1)), `ddd.BName` Array(String) CODEC(Delta(1))) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
|
@ -117,3 +117,20 @@ INSERT INTO test.compression_codec_multiple_with_key SELECT toDate('2018-10-12')
|
||||
SELECT COUNT(DISTINCT data) FROM test.compression_codec_multiple_with_key WHERE id < 222;
|
||||
|
||||
DROP TABLE IF EXISTS test.compression_codec_multiple_with_key;
|
||||
|
||||
DROP TABLE IF EXISTS test.test_default_delta;
|
||||
|
||||
CREATE TABLE test.test_default_delta(
|
||||
id UInt64 CODEC(Delta),
|
||||
data String CODEC(Delta),
|
||||
somedate Date CODEC(Delta),
|
||||
somenum Float64 CODEC(Delta),
|
||||
somestr FixedString(3) CODEC(Delta),
|
||||
othernum Int64 CODEC(Delta),
|
||||
yetothernum Float32 CODEC(Delta),
|
||||
ddd Nested (age UInt8, Name String, OName String, BName String) CODEC(Delta)
|
||||
) ENGINE = MergeTree() ORDER BY tuple();
|
||||
|
||||
SHOW CREATE TABLE test.test_default_delta;
|
||||
|
||||
DROP TABLE IF EXISTS test.test_default_delta;
|
||||
|
Loading…
Reference in New Issue
Block a user