Merge pull request #45376 from ClickHouse/block-non-float-gorilla-v2

Mark Gorilla codec on non-float columns as suspicous
This commit is contained in:
robot-ch-test-poll2 2023-01-28 01:50:37 +01:00 committed by GitHub
commit 520e057fd5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 149 additions and 89 deletions

View File

@ -293,7 +293,7 @@ These codecs are designed to make compression more effective by using specific f
#### Gorilla
`Gorilla` — Calculates XOR between current and previous value and writes it in compact binary form. Efficient when storing a series of floating point values that change slowly, because the best compression rate is achieved when neighboring values are binary equal. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. For additional information, see Compressing Values in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf).
`Gorilla` — Calculates XOR between current and previous floating point value and writes it in compact binary form. The smaller the difference between consecutive values is, i.e. the slower the values of the series changes, the better the compression rate. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. For additional information, see section 4.1 in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](https://doi.org/10.14778/2824032.2824078).
#### FPC

View File

@ -39,6 +39,7 @@ protected:
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return false; }
bool isFloatingPointTimeSeriesCodec() const override { return true; }
private:
static constexpr UInt32 HEADER_SIZE = 2;

View File

@ -122,6 +122,7 @@ protected:
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return false; }
bool isFloatingPointTimeSeriesCodec() const override { return true; }
private:
const UInt8 data_bytes_size;
@ -442,14 +443,14 @@ void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 sourc
void registerCodecGorilla(CompressionCodecFactory & factory)
{
UInt8 method_code = static_cast<UInt8>(CompressionMethodByte::Gorilla);
factory.registerCompressionCodecWithType("Gorilla", method_code,
[&](const ASTPtr & arguments, const IDataType * column_type) -> CompressionCodecPtr
auto codec_builder = [&](const ASTPtr & arguments, const IDataType * column_type) -> CompressionCodecPtr
{
if (arguments)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Gorilla does not accept any arguments");
UInt8 data_bytes_size = column_type ? getDataBytesSize(column_type) : 0;
return std::make_shared<CompressionCodecGorilla>(data_bytes_size);
});
};
factory.registerCompressionCodecWithType("Gorilla", method_code, codec_builder);
}
}

View File

@ -15,7 +15,9 @@
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/DataTypeNullable.h>
#include <Common/Exception.h>
@ -51,6 +53,28 @@ void CompressionCodecFactory::validateCodec(
}
}
namespace
{
bool innerDataTypeIsFloat(const DataTypePtr & type)
{
if (isFloat(type))
return true;
if (const DataTypeNullable * type_nullable = typeid_cast<const DataTypeNullable *>(type.get()))
return innerDataTypeIsFloat(type_nullable->getNestedType());
if (const DataTypeArray * type_array = typeid_cast<const DataTypeArray *>(type.get()))
return innerDataTypeIsFloat(type_array->getNestedType());
if (const DataTypeTuple * type_tuple = typeid_cast<const DataTypeTuple *>(type.get()))
{
for (const auto & subtype : type_tuple->getElements())
if (innerDataTypeIsFloat(subtype))
return true;
return false;
}
return false;
}
}
ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs) const
@ -59,15 +83,16 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
{
ASTPtr codecs_descriptions = std::make_shared<ASTExpressionList>();
bool is_compression = false;
bool has_none = false;
bool with_compression_codec = false;
bool with_none_codec = false;
bool with_floating_point_timeseries_codec = false;
std::optional<size_t> generic_compression_codec_pos;
std::set<size_t> encryption_codecs;
std::set<size_t> encryption_codecs_pos;
bool can_substitute_codec_arguments = true;
for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i)
{
const auto & inner_codec_ast = func->arguments->children[i];
const ASTPtr & inner_codec_ast = func->arguments->children[i];
String codec_family_name;
ASTPtr codec_arguments;
if (const auto * family_name = inner_codec_ast->as<ASTIdentifier>())
@ -136,21 +161,22 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
codecs_descriptions->children.emplace_back(result_codec->getCodecDesc());
}
is_compression |= result_codec->isCompression();
has_none |= result_codec->isNone();
with_compression_codec |= result_codec->isCompression();
with_none_codec |= result_codec->isNone();
with_floating_point_timeseries_codec |= result_codec->isFloatingPointTimeSeriesCodec();
if (!generic_compression_codec_pos && result_codec->isGenericCompression())
generic_compression_codec_pos = i;
if (result_codec->isEncryption())
encryption_codecs.insert(i);
encryption_codecs_pos.insert(i);
}
String codec_description = queryToString(codecs_descriptions);
if (sanity_check)
{
if (codecs_descriptions->children.size() > 1 && has_none)
if (codecs_descriptions->children.size() > 1 && with_none_codec)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"It does not make sense to have codec NONE along with other compression codecs: {}. "
"(Note: you can enable setting 'allow_suspicious_codecs' to skip this check).",
@ -159,7 +185,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
/// Allow to explicitly specify single NONE codec if user don't want any compression.
/// But applying other transformations solely without compression (e.g. Delta) does not make sense.
/// It's okay to apply encryption codecs solely without anything else.
if (!is_compression && !has_none && encryption_codecs.size() != codecs_descriptions->children.size())
if (!with_compression_codec && !with_none_codec && encryption_codecs_pos.size() != codecs_descriptions->children.size())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Compression codec {} does not compress anything. "
"You may want to add generic compression algorithm after other transformations, like: {}, LZ4. "
"(Note: you can enable setting 'allow_suspicious_codecs' to skip this check).",
@ -167,17 +193,25 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
/// It does not make sense to apply any non-encryption codecs
/// after encryption one.
if (!encryption_codecs.empty() &&
*encryption_codecs.begin() != codecs_descriptions->children.size() - encryption_codecs.size())
if (!encryption_codecs_pos.empty() &&
*encryption_codecs_pos.begin() != codecs_descriptions->children.size() - encryption_codecs_pos.size())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The combination of compression codecs {} is meaningless, "
"because it does not make sense to apply any non-post-processing codecs after "
"post-processing ones. (Note: you can enable setting 'allow_suspicious_codecs' "
"to skip this check).", codec_description);
/// Floating-point time series codecs are not supposed to compress non-floating-point data
if (with_floating_point_timeseries_codec &&
column_type && !innerDataTypeIsFloat(column_type))
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"The combination of compression codecs {} is meaningless,"
" because it does not make sense to apply a floating-point time series codec to non-floating-point columns"
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", codec_description);
/// It does not make sense to apply any transformations after generic compression algorithm
/// So, generic compression can be only one and only at the end.
if (generic_compression_codec_pos &&
*generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - encryption_codecs.size())
*generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - encryption_codecs_pos.size())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The combination of compression codecs {} is meaningless, "
"because it does not make sense to apply any transformations after generic "
"compression algorithm. (Note: you can enable setting 'allow_suspicious_codecs' "

View File

@ -99,6 +99,9 @@ public:
/// If it is a post-processing codec such as encryption. Usually it does not make sense to apply non-post-processing codecs after this.
virtual bool isEncryption() const { return false; }
/// If it is a specialized codec for floating-point time series. Applying it to non-floating point data is suspicious.
virtual bool isFloatingPointTimeSeriesCodec() const { return false; }
/// It is a codec available only for evaluation purposes and not meant to be used in production.
/// It will not be allowed to use unless the user will turn off the safety switch.
virtual bool isExperimental() const { return false; }

View File

@ -529,6 +529,13 @@ public:
TEST_P(CodecTest, TranscodingWithDataType)
{
/// Gorilla can only be applied to floating point columns
bool codec_is_gorilla = std::get<0>(GetParam()).codec_statement.find("Gorilla") != std::string::npos;
WhichDataType which(std::get<1>(GetParam()).data_type.get());
bool data_is_float = which.isFloat();
if (codec_is_gorilla && !data_is_float)
GTEST_SKIP() << "Skipping Gorilla-compressed non-float column";
const auto codec = makeCodec(CODEC_WITH_DATA_TYPE);
testTranscoding(*codec);
}
@ -1204,68 +1211,20 @@ auto DDperformanceTestSequence()
}
// prime numbers in ascending order with some random repetitions hit all the cases of Gorilla.
auto PrimesWithMultiplierGenerator = [](int multiplier = 1)
{
return [multiplier](auto i)
{
static const int vals[] = {
2, 3, 5, 7, 11, 11, 13, 17, 19, 23, 29, 29, 31, 37, 41, 43,
47, 47, 53, 59, 61, 61, 67, 71, 73, 79, 83, 89, 89, 97, 101, 103,
107, 107, 109, 113, 113, 127, 127, 127
};
static const size_t count = sizeof(vals)/sizeof(vals[0]);
return static_cast<UInt64>(vals[i % count]) * multiplier;
};
};
template <typename ValueType>
auto GCompatibilityTestSequence()
{
// Also multiply result by some factor to test large values on types that can hold those.
return generateSeq<ValueType>(G(PrimesWithMultiplierGenerator(intExp10(sizeof(ValueType)))), 0, 42);
}
INSTANTIATE_TEST_SUITE_P(Gorilla,
CodecTestCompatibility,
::testing::Combine(
::testing::Values(Codec("Gorilla")),
::testing::ValuesIn(std::initializer_list<std::tuple<CodecTestSequence, std::string>>{
{
GCompatibilityTestSequence<Int8>(),
BIN_STR("\x95\x35\x00\x00\x00\x2a\x00\x00\x00\x01\x00\x2a\x00\x00\x00\x14\xe1\xdd\x25\xe5\x7b\x29\x86\xee\x2a\x16\x5a\xc5\x0b\x23\x75\x1b\x3c\xb1\x97\x8b\x5f\xcb\x43\xd9\xc5\x48\xab\x23\xaf\x62\x93\x71\x4a\x73\x0f\xc6\x0a")
},
{
GCompatibilityTestSequence<UInt8>(),
BIN_STR("\x95\x35\x00\x00\x00\x2a\x00\x00\x00\x01\x00\x2a\x00\x00\x00\x14\xe1\xdd\x25\xe5\x7b\x29\x86\xee\x2a\x16\x5a\xc5\x0b\x23\x75\x1b\x3c\xb1\x97\x8b\x5f\xcb\x43\xd9\xc5\x48\xab\x23\xaf\x62\x93\x71\x4a\x73\x0f\xc6\x0a")
},
{
GCompatibilityTestSequence<Int16>(),
BIN_STR("\x95\x52\x00\x00\x00\x54\x00\x00\x00\x02\x00\x2a\x00\x00\x00\xc8\x00\xdc\xfe\x66\xdb\x1f\x4e\xa7\xde\xdc\xd5\xec\x6e\xf7\x37\x3a\x23\xe7\x63\xf5\x6a\x8e\x99\x37\x34\xf9\xf8\x2e\x76\x35\x2d\x51\xbb\x3b\xc3\x6d\x13\xbf\x86\x53\x9e\x25\xe4\xaf\xaf\x63\xd5\x6a\x6e\x76\x35\x3a\x27\xd3\x0f\x91\xae\x6b\x33\x57\x6e\x64\xcc\x55\x81\xe4")
},
{
GCompatibilityTestSequence<UInt16>(),
BIN_STR("\x95\x52\x00\x00\x00\x54\x00\x00\x00\x02\x00\x2a\x00\x00\x00\xc8\x00\xdc\xfe\x66\xdb\x1f\x4e\xa7\xde\xdc\xd5\xec\x6e\xf7\x37\x3a\x23\xe7\x63\xf5\x6a\x8e\x99\x37\x34\xf9\xf8\x2e\x76\x35\x2d\x51\xbb\x3b\xc3\x6d\x13\xbf\x86\x53\x9e\x25\xe4\xaf\xaf\x63\xd5\x6a\x6e\x76\x35\x3a\x27\xd3\x0f\x91\xae\x6b\x33\x57\x6e\x64\xcc\x55\x81\xe4")
},
{
GCompatibilityTestSequence<Int32>(),
BIN_STR("\x95\x65\x00\x00\x00\xa8\x00\x00\x00\x04\x00\x2a\x00\x00\x00\x20\x4e\x00\x00\xe4\x57\x63\xc0\xbb\x67\xbc\xce\x91\x97\x99\x15\x9e\xe3\x36\x3f\x89\x5f\x8e\xf2\xec\x8e\xd3\xbf\x75\x43\x58\xc4\x7e\xcf\x93\x43\x38\xc6\x91\x36\x1f\xe7\xb6\x11\x6f\x02\x73\x46\xef\xe0\xec\x50\xfb\x79\xcb\x9c\x14\xfa\x13\xea\x8d\x66\x43\x48\xa0\xde\x3a\xcf\xff\x26\xe0\x5f\x93\xde\x5e\x7f\x6e\x36\x5e\xe6\xb4\x66\x5d\xb0\x0e\xc4")
},
{
GCompatibilityTestSequence<UInt32>(),
BIN_STR("\x95\x65\x00\x00\x00\xa8\x00\x00\x00\x04\x00\x2a\x00\x00\x00\x20\x4e\x00\x00\xe4\x57\x63\xc0\xbb\x67\xbc\xce\x91\x97\x99\x15\x9e\xe3\x36\x3f\x89\x5f\x8e\xf2\xec\x8e\xd3\xbf\x75\x43\x58\xc4\x7e\xcf\x93\x43\x38\xc6\x91\x36\x1f\xe7\xb6\x11\x6f\x02\x73\x46\xef\xe0\xec\x50\xfb\x79\xcb\x9c\x14\xfa\x13\xea\x8d\x66\x43\x48\xa0\xde\x3a\xcf\xff\x26\xe0\x5f\x93\xde\x5e\x7f\x6e\x36\x5e\xe6\xb4\x66\x5d\xb0\x0e\xc4")
},
{
GCompatibilityTestSequence<Int64>(),
BIN_STR("\x95\x91\x00\x00\x00\x50\x01\x00\x00\x08\x00\x2a\x00\x00\x00\x00\xc2\xeb\x0b\x00\x00\x00\x00\xe3\x2b\xa0\xa6\x19\x85\x98\xdc\x45\x74\x74\x43\xc2\x57\x41\x4c\x6e\x42\x79\xd9\x8f\x88\xa5\x05\xf3\xf1\x94\xa3\x62\x1e\x02\xdf\x05\x10\xf1\x15\x97\x35\x2a\x50\x71\x0f\x09\x6c\x89\xf7\x65\x1d\x11\xb7\xcc\x7d\x0b\x70\xc1\x86\x88\x48\x47\x87\xb6\x32\x26\xa7\x86\x87\x88\xd3\x93\x3d\xfc\x28\x68\x85\x05\x0b\x13\xc6\x5f\xd4\x70\xe1\x5e\x76\xf1\x9f\xf3\x33\x2a\x14\x14\x5e\x40\xc1\x5c\x28\x3f\xec\x43\x03\x05\x11\x91\xe8\xeb\x8e\x0a\x0e\x27\x21\x55\xcb\x39\xbc\x6a\xff\x11\x5d\x81\xa0\xa6\x10")
},
{
GCompatibilityTestSequence<UInt64>(),
BIN_STR("\x95\x91\x00\x00\x00\x50\x01\x00\x00\x08\x00\x2a\x00\x00\x00\x00\xc2\xeb\x0b\x00\x00\x00\x00\xe3\x2b\xa0\xa6\x19\x85\x98\xdc\x45\x74\x74\x43\xc2\x57\x41\x4c\x6e\x42\x79\xd9\x8f\x88\xa5\x05\xf3\xf1\x94\xa3\x62\x1e\x02\xdf\x05\x10\xf1\x15\x97\x35\x2a\x50\x71\x0f\x09\x6c\x89\xf7\x65\x1d\x11\xb7\xcc\x7d\x0b\x70\xc1\x86\x88\x48\x47\x87\xb6\x32\x26\xa7\x86\x87\x88\xd3\x93\x3d\xfc\x28\x68\x85\x05\x0b\x13\xc6\x5f\xd4\x70\xe1\x5e\x76\xf1\x9f\xf3\x33\x2a\x14\x14\x5e\x40\xc1\x5c\x28\x3f\xec\x43\x03\x05\x11\x91\xe8\xeb\x8e\x0a\x0e\x27\x21\x55\xcb\x39\xbc\x6a\xff\x11\x5d\x81\xa0\xa6\x10")
},
})
)
);
// auto PrimesWithMultiplierGenerator = [](int multiplier = 1)
// {
// return [multiplier](auto i)
// {
// static const int vals[] = {
// 2, 3, 5, 7, 11, 11, 13, 17, 19, 23, 29, 29, 31, 37, 41, 43,
// 47, 47, 53, 59, 61, 61, 67, 71, 73, 79, 83, 89, 89, 97, 101, 103,
// 107, 107, 109, 113, 113, 127, 127, 127
// };
// static const size_t count = sizeof(vals)/sizeof(vals[0]);
//
// return static_cast<UInt64>(vals[i % count]) * multiplier;
// };
// };
// These 'tests' try to measure performance of encoding and decoding and hence only make sense to be run locally,
// also they require pretty big data to run against and generating this data slows down startup of unit test process.

View File

@ -13,7 +13,6 @@
<value>Delta</value>
<value>T64</value>
<value>DoubleDelta</value>
<value>Gorilla</value>
</values>
</substitution>
<substitution>

View File

@ -13,7 +13,6 @@
<value>Delta</value>
<value>T64</value>
<value>DoubleDelta</value>
<value>Gorilla</value>
</values>
</substitution>
<substitution>

View File

@ -6,6 +6,9 @@ CREATE TABLE default.codecs5\n(\n `a` UInt8 CODEC(LZ4, ZSTD(1))\n)\nENGINE =
CREATE TABLE default.codecs6\n(\n `a` UInt8 CODEC(Delta(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs7\n(\n `a` UInt8 CODEC(Delta(1), Delta(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs8\n(\n `a` UInt8 CODEC(LZ4, Delta(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs9\n(\n `a` UInt8 CODEC(Gorilla)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs10\n(\n `a` FixedString(2) CODEC(Gorilla)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs11\n(\n `a` Decimal(15, 5) CODEC(Gorilla)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs1\n(\n `a` UInt8 CODEC(NONE, NONE)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs2\n(\n `a` UInt8 CODEC(NONE, LZ4)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs3\n(\n `a` UInt8 CODEC(LZ4, NONE)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
@ -14,3 +17,6 @@ CREATE TABLE default.codecs5\n(\n `a` UInt8 CODEC(LZ4, ZSTD(1))\n)\nENGINE =
CREATE TABLE default.codecs6\n(\n `a` UInt8 CODEC(Delta(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs7\n(\n `a` UInt8 CODEC(Delta(1), Delta(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs8\n(\n `a` UInt8 CODEC(LZ4, Delta(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs9\n(\n `a` UInt8 CODEC(Gorilla)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs10\n(\n `a` FixedString(2) CODEC(Gorilla)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs11\n(\n `a` Decimal(15, 5) CODEC(Gorilla)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192

View File

@ -11,7 +11,7 @@ CREATE TABLE codecs
c Float32 CODEC(Gorilla),
d UInt8 CODEC(Delta, LZ4),
e Float64 CODEC(Gorilla, ZSTD),
f UInt32 CODEC(Delta, Delta, Gorilla),
f UInt32 CODEC(Delta, Delta, T64),
g DateTime CODEC(DoubleDelta),
h DateTime64 CODEC(DoubleDelta, LZ4),
i String CODEC(NONE)
@ -21,14 +21,17 @@ DROP TABLE codecs;
-- test what should not work
CREATE TABLE codecs (a UInt8 CODEC(NONE, NONE)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(NONE, LZ4)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, NONE)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, LZ4)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, ZSTD)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(Delta)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(Delta, Delta)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, Delta)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(NONE, NONE)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(NONE, LZ4)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, NONE)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, LZ4)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, ZSTD)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(Delta)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(Delta, Delta)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, Delta)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(Gorilla)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a FixedString(2) CODEC(Gorilla)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a Decimal(15,5) CODEC(Gorilla)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
-- test that sanity check is not performed in ATTACH query
@ -40,6 +43,9 @@ DROP TABLE IF EXISTS codecs5;
DROP TABLE IF EXISTS codecs6;
DROP TABLE IF EXISTS codecs7;
DROP TABLE IF EXISTS codecs8;
DROP TABLE IF EXISTS codecs9;
DROP TABLE IF EXISTS codecs10;
DROP TABLE IF EXISTS codecs11;
SET allow_suspicious_codecs = 1;
@ -51,6 +57,9 @@ CREATE TABLE codecs5 (a UInt8 CODEC(LZ4, ZSTD)) ENGINE = MergeTree ORDER BY tupl
CREATE TABLE codecs6 (a UInt8 CODEC(Delta)) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE codecs7 (a UInt8 CODEC(Delta, Delta)) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE codecs8 (a UInt8 CODEC(LZ4, Delta)) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE codecs9 (a UInt8 CODEC(Gorilla)) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE codecs10 (a FixedString(2) CODEC(Gorilla)) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE codecs11 (a Decimal(15,5) CODEC(Gorilla)) ENGINE = MergeTree ORDER BY tuple();
SET allow_suspicious_codecs = 0;
@ -62,6 +71,9 @@ SHOW CREATE TABLE codecs5;
SHOW CREATE TABLE codecs6;
SHOW CREATE TABLE codecs7;
SHOW CREATE TABLE codecs8;
SHOW CREATE TABLE codecs9;
SHOW CREATE TABLE codecs10;
SHOW CREATE TABLE codecs11;
DETACH TABLE codecs1;
DETACH TABLE codecs2;
@ -71,6 +83,9 @@ DETACH TABLE codecs5;
DETACH TABLE codecs6;
DETACH TABLE codecs7;
DETACH TABLE codecs8;
DETACH TABLE codecs9;
DETACH TABLE codecs10;
DETACH TABLE codecs11;
ATTACH TABLE codecs1;
ATTACH TABLE codecs2;
@ -80,6 +95,9 @@ ATTACH TABLE codecs5;
ATTACH TABLE codecs6;
ATTACH TABLE codecs7;
ATTACH TABLE codecs8;
ATTACH TABLE codecs9;
ATTACH TABLE codecs10;
ATTACH TABLE codecs11;
SHOW CREATE TABLE codecs1;
SHOW CREATE TABLE codecs2;
@ -89,6 +107,9 @@ SHOW CREATE TABLE codecs5;
SHOW CREATE TABLE codecs6;
SHOW CREATE TABLE codecs7;
SHOW CREATE TABLE codecs8;
SHOW CREATE TABLE codecs9;
SHOW CREATE TABLE codecs10;
SHOW CREATE TABLE codecs11;
SELECT * FROM codecs1;
SELECT * FROM codecs2;
@ -98,6 +119,9 @@ SELECT * FROM codecs5;
SELECT * FROM codecs6;
SELECT * FROM codecs7;
SELECT * FROM codecs8;
SELECT * FROM codecs9;
SELECT * FROM codecs10;
SELECT * FROM codecs11;
DROP TABLE codecs1;
DROP TABLE codecs2;
@ -107,3 +131,6 @@ DROP TABLE codecs5;
DROP TABLE codecs6;
DROP TABLE codecs7;
DROP TABLE codecs8;
DROP TABLE codecs9;
DROP TABLE codecs10;
DROP TABLE codecs11;

View File

@ -0,0 +1,31 @@
DROP TABLE IF EXISTS tbl;
-- Checks that (floating-point) time series codecs can be combined
-- with Nullable and
-- with composite types Array and Tuple
CREATE TABLE tbl (
-- Nullable
v1_gor Nullable(Float64) CODEC(Gorilla),
v1_fpc Nullable(Float64) CODEC(FPC),
-- Array
v2_gor Array(Float64) CODEC(Gorilla),
v2_fpc Array(Float64) CODEC(FPC),
v3_gor Array(Array(Float64)) CODEC(Gorilla),
v3_fpc Array(Array(Float64)) CODEC(FPC),
v4_gor Array(Nullable(Float64)) CODEC(Gorilla),
v4_fpc Array(Nullable(Float64)) CODEC(FPC),
v5_gor Array(Tuple(Float64)) CODEC(Gorilla),
v5_fpc Array(Tuple(Float64)) CODEC(FPC),
-- Tuple
v6_gor Tuple(Float64) CODEC(Gorilla),
v6_fpc Tuple(Float64) CODEC(FPC),
v7_gor Tuple(Tuple(Float64)) CODEC(Gorilla),
v7_fpc Tuple(Tuple(Float64)) CODEC(FPC),
v8_gor Tuple(Nullable(Float64)) CODEC(Gorilla),
v8_fpc Tuple(Nullable(Float64)) CODEC(FPC),
v9_gor Tuple(Array(Float64)) CODEC(Gorilla),
v9_fpc Tuple(Array(Float64)) CODEC(FPC),
) Engine = MergeTree ORDER BY tuple();
DROP TABLE IF EXISTS tbl;