mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Remove transitory parameter
This commit is contained in:
parent
aaf7653108
commit
574cab5d7e
@ -293,9 +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 section 4.1 in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](https://doi.org/10.14778/2824032.2824078).
|
||||
|
||||
It is currently possible to apply Gorilla compression to columns of non-floating-point type but this practice is strongly discouraged and will be removed in future.
|
||||
`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
|
||||
|
||||
|
@ -541,7 +541,7 @@ void Connection::sendQuery(
|
||||
if (method == "ZSTD")
|
||||
level = settings->network_zstd_compression_level;
|
||||
|
||||
CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs, settings->allow_experimental_codecs, settings->enable_gorilla_codec_for_non_float_data);
|
||||
CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs, settings->allow_experimental_codecs);
|
||||
compression_codec = CompressionCodecFactory::instance().get(method, level);
|
||||
}
|
||||
else
|
||||
|
@ -40,12 +40,10 @@ public:
|
||||
CompressionCodecPtr getDefaultCodec() const;
|
||||
|
||||
/// Validate codecs AST specified by user and parses codecs description (substitute default parameters)
|
||||
///
|
||||
/// Note: enable_gorilla_codec_for_non_float_data is a transitory parameter and can be removed after v23.6 (being implicitly false then), see comments in Core/Settings.h.
|
||||
ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_gorilla_codec_for_non_float_data) const;
|
||||
ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs) const;
|
||||
|
||||
/// Validate codecs AST specified by user
|
||||
void validateCodec(const String & family_name, std::optional<int> level, bool sanity_check, bool allow_experimental_codecs, bool enable_gorilla_codec_for_non_float_data) const;
|
||||
void validateCodec(const String & family_name, std::optional<int> level, bool sanity_check, bool allow_experimental_codecs) const;
|
||||
|
||||
/// Get codec by AST and possible column_type. Some codecs can use
|
||||
/// information about type to improve inner settings, but every codec should
|
||||
|
@ -32,7 +32,7 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
void CompressionCodecFactory::validateCodec(
|
||||
const String & family_name, std::optional<int> level, bool sanity_check, bool allow_experimental_codecs, bool enable_gorilla_codec_for_non_float_data) const
|
||||
const String & family_name, std::optional<int> level, bool sanity_check, bool allow_experimental_codecs) const
|
||||
{
|
||||
if (family_name.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Compression codec name cannot be empty");
|
||||
@ -41,19 +41,19 @@ void CompressionCodecFactory::validateCodec(
|
||||
{
|
||||
auto literal = std::make_shared<ASTLiteral>(static_cast<UInt64>(*level));
|
||||
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)),
|
||||
{}, sanity_check, allow_experimental_codecs, enable_gorilla_codec_for_non_float_data);
|
||||
{}, sanity_check, allow_experimental_codecs);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto identifier = std::make_shared<ASTIdentifier>(Poco::toUpper(family_name));
|
||||
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier),
|
||||
{}, sanity_check, allow_experimental_codecs, enable_gorilla_codec_for_non_float_data);
|
||||
{}, sanity_check, allow_experimental_codecs);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
|
||||
const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_gorilla_codec_for_non_float_data) const
|
||||
const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs) const
|
||||
{
|
||||
if (const auto * func = ast->as<ASTFunction>())
|
||||
{
|
||||
@ -177,9 +177,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
|
||||
"to skip this check).", codec_description);
|
||||
|
||||
if (column_type)
|
||||
if (with_floating_point_timeseries_codec &&
|
||||
!WhichDataType(*column_type).isFloat() &&
|
||||
!enable_gorilla_codec_for_non_float_data)
|
||||
if (with_floating_point_timeseries_codec && !WhichDataType(*column_type).isFloat())
|
||||
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"
|
||||
|
@ -106,6 +106,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 isFloatingPointTimeSeries() 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; }
|
||||
@ -113,8 +116,6 @@ public:
|
||||
/// If it does nothing.
|
||||
virtual bool isNone() const { return false; }
|
||||
|
||||
virtual bool isFloatingPointTimeSeries() const { return false; }
|
||||
|
||||
protected:
|
||||
/// This is used for fuzz testing
|
||||
friend int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size);
|
||||
|
@ -624,10 +624,6 @@ class IColumn;
|
||||
\
|
||||
M(Bool, load_marks_asynchronously, false, "Load MergeTree marks asynchronously", 0) \
|
||||
\
|
||||
/* Transitory setting needed for ca. six months. After v23.6, the setting can be moved into the obsolete section below and assumed as false. */ \
|
||||
/* Gorilla is a float-codec which happened to be enabled for non-float datatypes. Users (self-hosted or on-premise) might have non-float Gorilla-compressed data so allow some time for migration. */ \
|
||||
M(Bool, enable_gorilla_codec_for_non_float_data, true, "Enable Gorilla compression of columns with non-float data type", 0) \
|
||||
\
|
||||
M(UInt64, use_structure_from_insertion_table_in_table_functions, 2, "Use structure from insertion table instead of schema inference from data. Possible values: 0 - disabled, 1 - enabled, 2 - auto", 0) \
|
||||
\
|
||||
M(UInt64, http_max_tries, 10, "Max attempts to read via http.", 0) \
|
||||
|
@ -560,7 +560,6 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
|
||||
|
||||
bool sanity_check_compression_codecs = !attach && !context_->getSettingsRef().allow_suspicious_codecs;
|
||||
bool allow_experimental_codecs = attach || context_->getSettingsRef().allow_experimental_codecs;
|
||||
bool enable_gorilla_codec_for_non_float_data = context_->getSettingsRef().enable_gorilla_codec_for_non_float_data;
|
||||
|
||||
ColumnsDescription res;
|
||||
auto name_type_it = column_names_and_types.begin();
|
||||
@ -621,7 +620,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
|
||||
if (col_decl.default_specifier == "ALIAS")
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot specify codec for column type ALIAS");
|
||||
column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(
|
||||
col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_gorilla_codec_for_non_float_data);
|
||||
col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs);
|
||||
}
|
||||
|
||||
if (col_decl.ttl)
|
||||
|
@ -1692,7 +1692,7 @@ void TCPHandler::initBlockOutput(const Block & block)
|
||||
|
||||
if (state.compression == Protocol::Compression::Enable)
|
||||
{
|
||||
CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs, query_settings.allow_experimental_codecs, query_settings.enable_gorilla_codec_for_non_float_data);
|
||||
CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs, query_settings.allow_experimental_codecs);
|
||||
|
||||
state.maybe_compressed_out = std::make_shared<CompressedWriteBuffer>(
|
||||
*out, CompressionCodecFactory::instance().get(method, level));
|
||||
|
@ -388,7 +388,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
column.comment = *comment;
|
||||
|
||||
if (codec)
|
||||
column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type, false, true, true);
|
||||
column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type, false, true);
|
||||
|
||||
column.ttl = ttl;
|
||||
|
||||
@ -429,7 +429,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
else
|
||||
{
|
||||
if (codec)
|
||||
column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type ? data_type : column.type, false, true, true);
|
||||
column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type ? data_type : column.type, false, true);
|
||||
|
||||
if (comment)
|
||||
column.comment = *comment;
|
||||
@ -1079,7 +1079,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
|
||||
|
||||
if (command.codec)
|
||||
CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type,
|
||||
!context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_gorilla_codec_for_non_float_data);
|
||||
!context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs);
|
||||
|
||||
all_columns.add(ColumnDescription(column_name, command.data_type));
|
||||
}
|
||||
@ -1104,7 +1104,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
|
||||
|
||||
if (command.codec)
|
||||
CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type,
|
||||
!context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_gorilla_codec_for_non_float_data);
|
||||
!context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs);
|
||||
auto column_default = all_columns.getDefault(column_name);
|
||||
if (column_default)
|
||||
{
|
||||
|
@ -130,7 +130,7 @@ void ColumnDescription::readText(ReadBuffer & buf)
|
||||
comment = col_ast->comment->as<ASTLiteral &>().value.get<String>();
|
||||
|
||||
if (col_ast->codec)
|
||||
codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(col_ast->codec, type, false, true, true);
|
||||
codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(col_ast->codec, type, false, true);
|
||||
|
||||
if (col_ast->ttl)
|
||||
ttl = col_ast->ttl;
|
||||
|
@ -701,7 +701,7 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const
|
||||
if (compression_method == "ZSTD")
|
||||
compression_level = settings.network_zstd_compression_level;
|
||||
|
||||
CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs, settings.enable_gorilla_codec_for_non_float_data);
|
||||
CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs);
|
||||
CompressionCodecPtr compression_codec = CompressionCodecFactory::instance().get(compression_method, compression_level);
|
||||
|
||||
/// tmp directory is used to ensure atomicity of transactions
|
||||
|
@ -285,7 +285,7 @@ TTLDescription TTLDescription::getTTLFromAST(
|
||||
{
|
||||
result.recompression_codec =
|
||||
CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(
|
||||
ttl_element->recompression_codec, {}, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_gorilla_codec_for_non_float_data);
|
||||
ttl_element->recompression_codec, {}, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
|
@ -29,6 +29,9 @@ CREATE TABLE codecs (a UInt8 CODEC(LZ4, ZSTD)) ENGINE = MergeTree ORDER BY tuple
|
||||
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;
|
||||
|
@ -1,34 +0,0 @@
|
||||
-- Welcome visitor from the future! If it is >= July 2023 and your intention is to adjust the test because "enable_gorilla_codec_for_non_float_data"
|
||||
-- is now obsolete, then please also extend 01272_suspicious_codecs.sql with new tests cases for Gorilla on non-float data.
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
|
||||
-- current default behavior is to enable non-float Gorilla compressed data
|
||||
|
||||
CREATE TABLE test (id UInt64, val Decimal(15,5) CODEC (Gorilla)) ENGINE = MergeTree() ORDER BY id;
|
||||
DROP TABLE IF EXISTS test;
|
||||
|
||||
CREATE TABLE test (id UInt64, val FixedString(2) CODEC (Gorilla)) ENGINE = MergeTree() ORDER BY id;
|
||||
DROP TABLE IF EXISTS test;
|
||||
|
||||
CREATE TABLE test (id UInt64, val UInt64 CODEC (Gorilla)) ENGINE = MergeTree() ORDER BY id;
|
||||
DROP TABLE IF EXISTS test;
|
||||
|
||||
-- this can be changed (and it is planned to be changed by default in future) with a setting
|
||||
SET enable_gorilla_codec_for_non_float_data = false;
|
||||
|
||||
CREATE TABLE test (id UInt64, val Decimal(15,5) CODEC (Gorilla)) ENGINE = MergeTree() ORDER BY id; -- { serverError BAD_ARGUMENTS }
|
||||
CREATE TABLE test (id UInt64, val FixedString(2) CODEC (Gorilla)) ENGINE = MergeTree() ORDER BY id; -- { serverError BAD_ARGUMENTS }
|
||||
CREATE TABLE test (id UInt64, val UInt64 CODEC (Gorilla)) ENGINE = MergeTree() ORDER BY id; -- { serverError BAD_ARGUMENTS }
|
||||
|
||||
-- even with above setting, it will still be possible to create non-float Gorilla-compressed data using allow_suspicious_codecs
|
||||
SET allow_suspicious_codecs = true;
|
||||
|
||||
CREATE TABLE test (id UInt64, val Decimal(15,5) CODEC (Gorilla)) ENGINE = MergeTree() ORDER BY id;
|
||||
DROP TABLE IF EXISTS test;
|
||||
|
||||
CREATE TABLE test (id UInt64, val FixedString(2) CODEC (Gorilla)) ENGINE = MergeTree() ORDER BY id;
|
||||
DROP TABLE IF EXISTS test;
|
||||
|
||||
CREATE TABLE test (id UInt64, val UInt64 CODEC (Gorilla)) ENGINE = MergeTree() ORDER BY id;
|
||||
DROP TABLE IF EXISTS test;
|
Loading…
Reference in New Issue
Block a user