mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #4052 from yandex/compression_codec_delta
Compression codec delta
This commit is contained in:
commit
8cd1e8c21d
151
dbms/src/Compression/CompressionCodecDelta.cpp
Normal file
151
dbms/src/Compression/CompressionCodecDelta.cpp
Normal file
@ -0,0 +1,151 @@
|
||||
#include <Compression/CompressionCodecDelta.h>
|
||||
#include <Compression/CompressionInfo.h>
|
||||
#include <Compression/CompressionFactory.h>
|
||||
#include <common/unaligned.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <cstdlib>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_COMPRESS;
|
||||
extern const int CANNOT_DECOMPRESS;
|
||||
extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE;
|
||||
extern const int ILLEGAL_CODEC_PARAMETER;
|
||||
}
|
||||
|
||||
CompressionCodecDelta::CompressionCodecDelta(UInt8 delta_bytes_size_)
|
||||
: delta_bytes_size(delta_bytes_size_)
|
||||
{
|
||||
}
|
||||
|
||||
UInt8 CompressionCodecDelta::getMethodByte() const
|
||||
{
|
||||
return static_cast<UInt8>(CompressionMethodByte::Delta);
|
||||
}
|
||||
|
||||
String CompressionCodecDelta::getCodecDesc() const
|
||||
{
|
||||
return "Delta(" + toString(delta_bytes_size) + ")";
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
void compressDataForType(const char * source, UInt32 source_size, char * dest)
|
||||
{
|
||||
if (source_size % sizeof(T) != 0)
|
||||
throw Exception("Cannot delta compress, data size " + toString(source_size) + " is not aligned to " + toString(sizeof(T)), ErrorCodes::CANNOT_COMPRESS);
|
||||
|
||||
const auto * source_with_type = reinterpret_cast<const T *>(source);
|
||||
auto * dest_with_type = reinterpret_cast<T *>(dest);
|
||||
|
||||
if (source_size > 0)
|
||||
dest_with_type[0] = source_with_type[0];
|
||||
|
||||
for (size_t dest_index = 1, dest_end = source_size / sizeof(T); dest_index < dest_end; ++dest_index)
|
||||
unalignedStore<T>(&dest_with_type[dest_index], source_with_type[dest_index] - source_with_type[dest_index - 1]);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void decompressDataForType(const char * source, UInt32 source_size, char * dest)
|
||||
{
|
||||
if (source_size % sizeof(T) != 0)
|
||||
throw Exception("Cannot delta decompress, data size " + toString(source_size) + " is not aligned to " + toString(sizeof(T)), ErrorCodes::CANNOT_DECOMPRESS);
|
||||
|
||||
const auto * source_with_type = reinterpret_cast<const T *>(source);
|
||||
auto * dest_with_type = reinterpret_cast<T *>(dest);
|
||||
|
||||
if (source_size > 0)
|
||||
dest_with_type[0] = source_with_type[0];
|
||||
|
||||
for (size_t dest_index = 1, dest_end = source_size / sizeof(T); dest_index < dest_end; ++dest_index)
|
||||
unalignedStore<T>(&dest_with_type[dest_index], source_with_type[dest_index] + dest_with_type[dest_index - 1]);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
UInt32 CompressionCodecDelta::doCompressData(const char * source, UInt32 source_size, char * dest) const
|
||||
{
|
||||
UInt8 bytes_to_skip = source_size % delta_bytes_size;
|
||||
dest[0] = delta_bytes_size;
|
||||
dest[1] = bytes_to_skip;
|
||||
memcpy(&dest[2], source, bytes_to_skip);
|
||||
size_t start_pos = 2 + bytes_to_skip;
|
||||
switch (delta_bytes_size)
|
||||
{
|
||||
case 1:
|
||||
compressDataForType<UInt8>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
|
||||
break;
|
||||
case 2:
|
||||
compressDataForType<UInt16>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
|
||||
break;
|
||||
case 4:
|
||||
compressDataForType<UInt32>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
|
||||
break;
|
||||
case 8:
|
||||
compressDataForType<UInt64>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
|
||||
break;
|
||||
}
|
||||
return 1 + 1 + source_size;
|
||||
}
|
||||
|
||||
void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 /* uncompressed_size */) const
|
||||
{
|
||||
UInt8 bytes_size = source[0];
|
||||
UInt8 bytes_to_skip = source[1];
|
||||
|
||||
memcpy(dest, &source[2], bytes_to_skip);
|
||||
UInt32 source_size_no_header = source_size - bytes_to_skip - 2;
|
||||
switch (bytes_size)
|
||||
{
|
||||
case 1:
|
||||
decompressDataForType<UInt8>(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]);
|
||||
break;
|
||||
case 2:
|
||||
decompressDataForType<UInt16>(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]);
|
||||
break;
|
||||
case 4:
|
||||
decompressDataForType<UInt32>(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]);
|
||||
break;
|
||||
case 8:
|
||||
decompressDataForType<UInt64>(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
void registerCodecDelta(CompressionCodecFactory & factory)
|
||||
{
|
||||
UInt8 method_code = UInt8(CompressionMethodByte::Delta);
|
||||
factory.registerCompressionCodecWithType("Delta", method_code, [&](const ASTPtr & arguments, DataTypePtr column_type) -> CompressionCodecPtr
|
||||
{
|
||||
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)
|
||||
throw Exception("Delta codec must have 1 parameter, given " + std::to_string(arguments->children.size()), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
|
||||
|
||||
const auto children = arguments->children;
|
||||
const ASTLiteral * literal = static_cast<const ASTLiteral *>(children[0].get());
|
||||
size_t user_bytes_size = literal->value.safeGet<UInt64>();
|
||||
if (user_bytes_size != 1 && user_bytes_size != 2 && user_bytes_size != 4 && user_bytes_size != 8)
|
||||
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);
|
||||
});
|
||||
}
|
||||
}
|
26
dbms/src/Compression/CompressionCodecDelta.h
Normal file
26
dbms/src/Compression/CompressionCodecDelta.h
Normal file
@ -0,0 +1,26 @@
|
||||
#pragma once
|
||||
|
||||
#include <Compression/ICompressionCodec.h>
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class CompressionCodecDelta : public ICompressionCodec
|
||||
{
|
||||
public:
|
||||
CompressionCodecDelta(UInt8 delta_bytes_size_);
|
||||
|
||||
UInt8 getMethodByte() const override;
|
||||
|
||||
String getCodecDesc() const override;
|
||||
|
||||
protected:
|
||||
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
|
||||
|
||||
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
|
||||
|
||||
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override { return uncompressed_size + 2; }
|
||||
private:
|
||||
const UInt8 delta_bytes_size;
|
||||
};
|
||||
}
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include "CompressionCodecLZ4.h"
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic ignored "-Wold-style-cast"
|
||||
@ -35,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);
|
||||
}
|
||||
@ -61,7 +62,7 @@ void registerCodecLZ4(CompressionCodecFactory & factory)
|
||||
|
||||
String CompressionCodecLZ4HC::getCodecDesc() const
|
||||
{
|
||||
return "LZ4HC";
|
||||
return "LZ4HC(" + toString(level) + ")";
|
||||
}
|
||||
|
||||
UInt32 CompressionCodecLZ4HC::doCompressData(const char * source, UInt32 source_size, char * dest) const
|
||||
|
@ -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;
|
||||
};
|
||||
@ -41,7 +41,7 @@ protected:
|
||||
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
|
||||
|
||||
private:
|
||||
int level;
|
||||
const int level;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -28,10 +28,10 @@ UInt8 CompressionCodecZSTD::getMethodByte() const
|
||||
|
||||
String CompressionCodecZSTD::getCodecDesc() const
|
||||
{
|
||||
return "ZSTD";
|
||||
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;
|
||||
@ -27,7 +27,7 @@ protected:
|
||||
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
|
||||
|
||||
private:
|
||||
int level;
|
||||
const int level;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -10,7 +10,6 @@
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Compression/CompressionCodecMultiple.h>
|
||||
#include <Compression/CompressionCodecLZ4.h>
|
||||
#include <Compression/CompressionCodecNone.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
@ -56,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()))
|
||||
{
|
||||
@ -65,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);
|
||||
}
|
||||
@ -88,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);
|
||||
@ -101,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",
|
||||
@ -118,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)
|
||||
{
|
||||
@ -135,7 +148,7 @@ void registerCodecNone(CompressionCodecFactory & factory);
|
||||
void registerCodecZSTD(CompressionCodecFactory & factory);
|
||||
void registerCodecMultiple(CompressionCodecFactory & factory);
|
||||
void registerCodecLZ4HC(CompressionCodecFactory & factory);
|
||||
//void registerCodecDelta(CompressionCodecFactory & factory);
|
||||
void registerCodecDelta(CompressionCodecFactory & factory);
|
||||
|
||||
CompressionCodecFactory::CompressionCodecFactory()
|
||||
{
|
||||
@ -145,7 +158,7 @@ CompressionCodecFactory::CompressionCodecFactory()
|
||||
registerCodecZSTD(*this);
|
||||
registerCodecMultiple(*this);
|
||||
registerCodecLZ4HC(*this);
|
||||
// registerCodecDelta(*this);
|
||||
registerCodecDelta(*this);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <optional>
|
||||
#include <unordered_map>
|
||||
#include <ext/singleton.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Common/IFactoryWithAliases.h>
|
||||
#include <Compression/ICompressionCodec.h>
|
||||
#include <Compression/CompressionInfo.h>
|
||||
@ -28,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;
|
||||
@ -46,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);
|
||||
|
||||
@ -53,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;
|
||||
|
@ -39,6 +39,7 @@ enum class CompressionMethodByte : uint8_t
|
||||
LZ4 = 0x82,
|
||||
ZSTD = 0x90,
|
||||
Multiple = 0x91,
|
||||
Delta = 0x92,
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -1,6 +1,6 @@
|
||||
1 hello 2018-12-14 1.1 aaa
|
||||
2 world 2018-12-15 2.2 bbb
|
||||
3 ! 2018-12-16 3.3 ccc
|
||||
1 hello 2018-12-14 1.1 aaa 5
|
||||
2 world 2018-12-15 2.2 bbb 6
|
||||
3 ! 2018-12-16 3.3 ccc 7
|
||||
2
|
||||
1 world 2018-10-05 1.1
|
||||
2 hello 2018-10-01 2.2
|
||||
@ -9,8 +9,9 @@
|
||||
10003
|
||||
274972506.6
|
||||
9175437371954010821
|
||||
CREATE TABLE test.compression_codec_multiple_more_types ( id Decimal(38, 13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, LZ4HC), data FixedString(12) CODEC(ZSTD, ZSTD, NONE, NONE, NONE, LZ4HC), `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD), `ddd.Name` Array(String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD)) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.compression_codec_multiple_more_types ( id Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)), data FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)), `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)), `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
1.5555555555555 hello world! [77] ['John']
|
||||
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
|
||||
|
@ -2,17 +2,27 @@ SET send_logs_level = 'none';
|
||||
|
||||
DROP TABLE IF EXISTS test.compression_codec;
|
||||
|
||||
CREATE TABLE test.compression_codec(id UInt64 CODEC(LZ4), data String CODEC(ZSTD), ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7))) ENGINE = MergeTree() ORDER BY tuple();
|
||||
CREATE TABLE test.compression_codec(
|
||||
id UInt64 CODEC(LZ4),
|
||||
data String CODEC(ZSTD),
|
||||
ddd Date CODEC(NONE),
|
||||
somenum Float64 CODEC(ZSTD(2)),
|
||||
somestr FixedString(3) CODEC(LZ4HC(7)),
|
||||
othernum Int64 CODEC(Delta)
|
||||
) ENGINE = MergeTree() ORDER BY tuple();
|
||||
|
||||
INSERT INTO test.compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa');
|
||||
INSERT INTO test.compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb');
|
||||
INSERT INTO test.compression_codec VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc');
|
||||
INSERT INTO test.compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5);
|
||||
INSERT INTO test.compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6);
|
||||
INSERT INTO test.compression_codec VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7);
|
||||
|
||||
SELECT * FROM test.compression_codec ORDER BY id;
|
||||
|
||||
OPTIMIZE TABLE test.compression_codec FINAL;
|
||||
|
||||
INSERT INTO test.compression_codec VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd');
|
||||
INSERT INTO test.compression_codec VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8);
|
||||
|
||||
DETACH TABLE test.compression_codec;
|
||||
ATTACH TABLE test.compression_codec;
|
||||
|
||||
SELECT count(*) FROM test.compression_codec WHERE id = 2 GROUP BY id;
|
||||
|
||||
@ -23,28 +33,34 @@ DROP TABLE IF EXISTS test.params_when_no_params;
|
||||
DROP TABLE IF EXISTS test.too_many_params;
|
||||
DROP TABLE IF EXISTS test.codec_multiple_direct_specification_1;
|
||||
DROP TABLE IF EXISTS test.codec_multiple_direct_specification_2;
|
||||
DROP TABLE IF EXISTS test.delta_bad_params1;
|
||||
DROP TABLE IF EXISTS test.delta_bad_params2;
|
||||
|
||||
CREATE TABLE test.bad_codec(id UInt64 CODEC(adssadads)) ENGINE = MergeTree() order by tuple(); -- { serverError 432 }
|
||||
CREATE TABLE test.too_many_params(id UInt64 CODEC(ZSTD(2,3,4,5))) ENGINE = MergeTree() order by tuple(); -- { serverError 431 }
|
||||
CREATE TABLE test.params_when_no_params(id UInt64 CODEC(LZ4(1))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 378 }
|
||||
CREATE TABLE test.codec_multiple_direct_specification_1(id UInt64 CODEC(MULTIPLE(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 432 }
|
||||
CREATE TABLE test.codec_multiple_direct_specification_2(id UInt64 CODEC(multiple(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 432 }
|
||||
CREATE TABLE test.delta_bad_params1(id UInt64 CODEC(Delta(3))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 433 }
|
||||
CREATE TABLE test.delta_bad_params2(id UInt64 CODEC(Delta(16))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 433 }
|
||||
|
||||
DROP TABLE IF EXISTS test.bad_codec;
|
||||
DROP TABLE IF EXISTS test.params_when_no_params;
|
||||
DROP TABLE IF EXISTS test.too_many_params;
|
||||
DROP TABLE IF EXISTS test.codec_multiple_direct_specification_1;
|
||||
DROP TABLE IF EXISTS test.codec_multiple_direct_specification_2;
|
||||
DROP TABLE IF EXISTS test.delta_bad_params1;
|
||||
DROP TABLE IF EXISTS test.delta_bad_params2;
|
||||
|
||||
DROP TABLE IF EXISTS test.compression_codec_multiple;
|
||||
|
||||
SET network_compression_method = 'lz4hc';
|
||||
|
||||
CREATE TABLE test.compression_codec_multiple (
|
||||
id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC),
|
||||
data String CODEC(ZSTD(2), NONE, LZ4HC, LZ4, LZ4),
|
||||
ddd Date CODEC(NONE, NONE, NONE, LZ4, ZSTD, LZ4HC, LZ4HC),
|
||||
somenum Float64 CODEC(LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD)
|
||||
id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4)),
|
||||
data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)),
|
||||
ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC),
|
||||
somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD)
|
||||
) ENGINE = MergeTree() ORDER BY tuple();
|
||||
|
||||
INSERT INTO test.compression_codec_multiple VALUES (1, 'world', toDate('2018-10-05'), 1.1), (2, 'hello', toDate('2018-10-01'), 2.2), (3, 'buy', toDate('2018-10-11'), 3.3);
|
||||
@ -68,9 +84,9 @@ SELECT sum(cityHash64(*)) FROM test.compression_codec_multiple;
|
||||
DROP TABLE IF EXISTS test.compression_codec_multiple_more_types;
|
||||
|
||||
CREATE TABLE test.compression_codec_multiple_more_types (
|
||||
id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, LZ4HC),
|
||||
data FixedString(12) CODEC(ZSTD, ZSTD, NONE, NONE, NONE, LZ4HC),
|
||||
ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD)
|
||||
id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, Delta(2), Delta(4), Delta(1), LZ4HC),
|
||||
data FixedString(12) CODEC(ZSTD, ZSTD, Delta, Delta, Delta, NONE, NONE, NONE, LZ4HC),
|
||||
ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD, Delta(8))
|
||||
) ENGINE = MergeTree() ORDER BY tuple();
|
||||
|
||||
SHOW CREATE TABLE test.compression_codec_multiple_more_types;
|
||||
@ -86,9 +102,9 @@ SET network_compression_method = 'zstd';
|
||||
SET network_zstd_compression_level = 5;
|
||||
|
||||
CREATE TABLE test.compression_codec_multiple_with_key (
|
||||
somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12)),
|
||||
id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC),
|
||||
data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4)
|
||||
somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), Delta, Delta),
|
||||
id UInt64 CODEC(LZ4, ZSTD, Delta, NONE, LZ4HC, Delta),
|
||||
data String CODEC(ZSTD(2), Delta, LZ4HC, NONE, LZ4, LZ4)
|
||||
) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2;
|
||||
|
||||
|
||||
@ -101,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