mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
refine patch according to comments
This commit is contained in:
parent
cbdb408ec8
commit
f1192d59af
@ -380,7 +380,7 @@ High compression levels are useful for asymmetric scenarios, like compress once,
|
||||
|
||||
`DEFLATE_QPL` — [Deflate compression algorithm](https://github.com/intel/qpl) implemented by Intel® Query Processing Library. Some limitations apply:
|
||||
|
||||
- DEFLATE_QPL is disabled by default and can only be used after setting configuration parameter `enable_qpl_deflate=1`.
|
||||
- DEFLATE_QPL is disabled by default and can only be used after setting configuration parameter `enable_qpl_deflate_codec=1`.
|
||||
- DEFLATE_QPL requires a ClickHouse build compiled with SSE 4.2 instructions (by default, this is the case). Refer to [Build Clickhouse with DEFLATE_QPL](/docs/en/development/building_and_benchmarking_deflate_qpl.md/#Build-Clickhouse-with-DEFLATE_QPL) for more details.
|
||||
- DEFLATE_QPL works best if the system has a Intel® IAA (In-Memory Analytics Accelerator) offloading device. Refer to [Accelerator Configuration](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) and [Benchmark with DEFLATE_QPL](/docs/en/development/building_and_benchmarking_deflate_qpl.md/#Run-Benchmark-with-DEFLATE_QPL) for more details.
|
||||
- DEFLATE_QPL-compressed data can only be transferred between ClickHouse nodes compiled with SSE 4.2 enabled.
|
||||
|
@ -588,7 +588,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_qpl_deflate);
|
||||
CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs, settings->allow_experimental_codecs, settings->enable_qpl_deflate_codec);
|
||||
compression_codec = CompressionCodecFactory::instance().get(method, level);
|
||||
}
|
||||
else
|
||||
|
@ -40,10 +40,10 @@ public:
|
||||
CompressionCodecPtr getDefaultCodec() const;
|
||||
|
||||
/// Validate codecs AST specified by user and parses codecs description (substitute default parameters)
|
||||
ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate) const;
|
||||
ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate_codec) 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_qpl_deflate) const;
|
||||
void validateCodec(const String & family_name, std::optional<int> level, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate_codec) const;
|
||||
|
||||
/// Get codec by AST and possible column_type. Some codecs can use
|
||||
/// information about type to improve inner settings, but every codec should
|
||||
|
@ -34,7 +34,7 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
void CompressionCodecFactory::validateCodec(
|
||||
const String & family_name, std::optional<int> level, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate) const
|
||||
const String & family_name, std::optional<int> level, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate_codec) const
|
||||
{
|
||||
if (family_name.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Compression codec name cannot be empty");
|
||||
@ -43,13 +43,13 @@ 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_qpl_deflate);
|
||||
{}, sanity_check, allow_experimental_codecs, enable_qpl_deflate_codec);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto identifier = std::make_shared<ASTIdentifier>(Poco::toUpper(family_name));
|
||||
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier),
|
||||
{}, sanity_check, allow_experimental_codecs, enable_qpl_deflate);
|
||||
{}, sanity_check, allow_experimental_codecs, enable_qpl_deflate_codec);
|
||||
}
|
||||
}
|
||||
|
||||
@ -77,7 +77,7 @@ bool innerDataTypeIsFloat(const DataTypePtr & type)
|
||||
}
|
||||
|
||||
ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
|
||||
const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate) const
|
||||
const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate_codec) const
|
||||
{
|
||||
if (const auto * func = ast->as<ASTFunction>())
|
||||
{
|
||||
@ -159,10 +159,10 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
|
||||
" You can enable it with the 'allow_experimental_codecs' setting.",
|
||||
codec_family_name);
|
||||
|
||||
if (!enable_qpl_deflate && result_codec->isDeflateQplCompression())
|
||||
if (!enable_qpl_deflate_codec && result_codec->isDeflateQplCompression())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Codec {} is disabled by default."
|
||||
" You can enable it with the 'enable_qpl_deflate' setting.",
|
||||
" You can enable it with the 'enable_qpl_deflate_codec' setting.",
|
||||
codec_family_name);
|
||||
|
||||
codecs_descriptions->children.emplace_back(result_codec->getCodecDesc());
|
||||
|
@ -109,12 +109,12 @@ public:
|
||||
/// It will not be allowed to use unless the user will turn off the safety switch.
|
||||
virtual bool isExperimental() const { return false; }
|
||||
|
||||
/// If it does nothing.
|
||||
virtual bool isNone() const { return false; }
|
||||
|
||||
/// This is a knob for Deflate QPL codec.
|
||||
virtual bool isDeflateQplCompression() const { return false; }
|
||||
|
||||
/// If it does nothing.
|
||||
virtual bool isNone() const { return false; }
|
||||
|
||||
protected:
|
||||
/// This is used for fuzz testing
|
||||
friend int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size);
|
||||
|
@ -319,7 +319,7 @@ class IColumn;
|
||||
M(Bool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \
|
||||
M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \
|
||||
M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \
|
||||
M(Bool, enable_qpl_deflate, false, "If it is set to true, allow to use deflate_qpl for compression.", 0) \
|
||||
M(Bool, enable_qpl_deflate_codec, false, "If it is set to true, allow usage of the DEFLATE_QPL codec.", 0) \
|
||||
M(UInt64, query_profiler_real_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
|
||||
M(UInt64, query_profiler_cpu_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
|
||||
M(Bool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \
|
||||
|
@ -571,7 +571,7 @@ 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_qpl_deflate = attach || context_->getSettingsRef().enable_qpl_deflate;
|
||||
bool enable_qpl_deflate_codec = attach || context_->getSettingsRef().enable_qpl_deflate_codec;
|
||||
|
||||
ColumnsDescription res;
|
||||
auto name_type_it = column_names_and_types.begin();
|
||||
@ -632,7 +632,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_qpl_deflate);
|
||||
col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_qpl_deflate_codec);
|
||||
}
|
||||
|
||||
if (col_decl.ttl)
|
||||
|
@ -1775,7 +1775,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_qpl_deflate);
|
||||
CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs, query_settings.allow_experimental_codecs, query_settings.enable_qpl_deflate_codec);
|
||||
|
||||
state.maybe_compressed_out = std::make_shared<CompressedWriteBuffer>(
|
||||
*out, CompressionCodecFactory::instance().get(method, level));
|
||||
|
@ -1067,7 +1067,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
|
||||
"this column name is reserved for lightweight delete feature", backQuote(column_name));
|
||||
|
||||
if (command.codec)
|
||||
CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate);
|
||||
CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate_codec);
|
||||
|
||||
all_columns.add(ColumnDescription(column_name, command.data_type));
|
||||
}
|
||||
@ -1093,7 +1093,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
|
||||
{
|
||||
if (all_columns.hasAlias(column_name))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot specify codec for column type ALIAS");
|
||||
CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate);
|
||||
CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate_codec);
|
||||
}
|
||||
auto column_default = all_columns.getDefault(column_name);
|
||||
if (column_default)
|
||||
|
@ -733,7 +733,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_qpl_deflate);
|
||||
CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs, settings.enable_qpl_deflate_codec);
|
||||
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_qpl_deflate);
|
||||
ttl_element->recompression_codec, {}, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate_codec);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -20,7 +20,7 @@ def get_options(i, upgrade_check):
|
||||
'''--db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i)
|
||||
)
|
||||
client_options.append("allow_experimental_database_replicated=1")
|
||||
client_options.append("enable_qpl_deflate=1")
|
||||
client_options.append("enable_qpl_deflate_codec=1")
|
||||
|
||||
# If database name is not specified, new database is created for each functional test.
|
||||
# Run some threads with one database for all tests.
|
||||
|
@ -1,7 +1,7 @@
|
||||
<clickhouse>
|
||||
<profiles>
|
||||
<default>
|
||||
<enable_qpl_deflate>1</enable_qpl_deflate>
|
||||
<enable_qpl_deflate_codec>1</enable_qpl_deflate_codec>
|
||||
</default>
|
||||
</profiles>
|
||||
</clickhouse>
|
||||
|
@ -38,16 +38,16 @@ node5 = cluster.add_instance(
|
||||
)
|
||||
node6 = cluster.add_instance(
|
||||
"node6",
|
||||
main_configs=["configs/allow_experimental_codecs.xml"],
|
||||
user_configs=["configs/allow_suspicious_codecs.xml"],
|
||||
main_configs=["configs/deflateqpl_compression_by_default.xml"],
|
||||
user_configs=[
|
||||
"configs/allow_suspicious_codecs.xml",
|
||||
"configs/enable_deflateqpl_codec.xml",
|
||||
],
|
||||
)
|
||||
node7 = cluster.add_instance(
|
||||
"node7",
|
||||
main_configs=["configs/deflateqpl_compression_by_default.xml"],
|
||||
user_configs=[
|
||||
"configs/enable_deflateqpl_codec.xml",
|
||||
"configs/allow_suspicious_codecs.xml",
|
||||
],
|
||||
main_configs=["configs/allow_experimental_codecs.xml"],
|
||||
user_configs=["configs/allow_suspicious_codecs.xml"],
|
||||
)
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
@ -253,7 +253,7 @@ def test_uncompressed_cache_plus_zstd_codec(start_cluster):
|
||||
)
|
||||
|
||||
def test_preconfigured_deflateqpl_codec(start_cluster):
|
||||
node7.query(
|
||||
node6.query(
|
||||
"""
|
||||
CREATE TABLE compression_codec_multiple_with_key (
|
||||
somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), DEFLATE_QPL),
|
||||
@ -263,46 +263,46 @@ def test_preconfigured_deflateqpl_codec(start_cluster):
|
||||
) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2;
|
||||
"""
|
||||
)
|
||||
node7.query(
|
||||
node6.query(
|
||||
"INSERT INTO compression_codec_multiple_with_key VALUES(toDate('2018-10-12'), 100000, 'hello', 88.88), (toDate('2018-10-12'), 100002, 'world', 99.99), (toDate('2018-10-12'), 1111, '!', 777.777)"
|
||||
)
|
||||
assert (
|
||||
node7.query(
|
||||
node6.query(
|
||||
"SELECT COUNT(*) FROM compression_codec_multiple_with_key WHERE id % 2 == 0"
|
||||
)
|
||||
== "2\n"
|
||||
)
|
||||
assert (
|
||||
node7.query(
|
||||
node6.query(
|
||||
"SELECT DISTINCT somecolumn FROM compression_codec_multiple_with_key ORDER BY id"
|
||||
)
|
||||
== "777.777\n88.88\n99.99\n"
|
||||
)
|
||||
assert (
|
||||
node7.query(
|
||||
node6.query(
|
||||
"SELECT data FROM compression_codec_multiple_with_key WHERE id >= 1112 AND somedate = toDate('2018-10-12') AND somecolumn <= 100"
|
||||
)
|
||||
== "hello\nworld\n"
|
||||
)
|
||||
|
||||
node7.query(
|
||||
node6.query(
|
||||
"INSERT INTO compression_codec_multiple_with_key SELECT toDate('2018-10-12'), number, toString(number), 1.0 FROM system.numbers LIMIT 10000"
|
||||
)
|
||||
|
||||
assert (
|
||||
node7.query(
|
||||
node6.query(
|
||||
"SELECT COUNT(id) FROM compression_codec_multiple_with_key WHERE id % 10 == 0"
|
||||
)
|
||||
== "1001\n"
|
||||
)
|
||||
assert (
|
||||
node7.query(
|
||||
node6.query(
|
||||
"SELECT SUM(somecolumn) FROM compression_codec_multiple_with_key"
|
||||
)
|
||||
== str(777.777 + 88.88 + 99.99 + 1.0 * 10000) + "\n"
|
||||
)
|
||||
assert (
|
||||
node7.query(
|
||||
node6.query(
|
||||
"SELECT count(*) FROM compression_codec_multiple_with_key GROUP BY somedate"
|
||||
)
|
||||
== "10003\n"
|
||||
|
@ -1,7 +1,7 @@
|
||||
<test>
|
||||
<settings>
|
||||
<allow_suspicious_codecs>1</allow_suspicious_codecs>
|
||||
<enable_deflate_qpl >1</enable_deflate_qpl >
|
||||
<enable_deflate_qpl>1</enable_deflate_qpl>
|
||||
</settings>
|
||||
|
||||
<substitutions>
|
||||
@ -11,7 +11,6 @@
|
||||
<value>NONE</value> <!-- as a baseline -->
|
||||
<value>LZ4</value>
|
||||
<value>ZSTD</value>
|
||||
<value>DEFLATE_QPL</value>
|
||||
<value>DoubleDelta</value>
|
||||
<value>Gorilla</value>
|
||||
<value>FPC</value>
|
||||
|
@ -1,7 +1,7 @@
|
||||
<test>
|
||||
<settings>
|
||||
<allow_suspicious_codecs>1</allow_suspicious_codecs>
|
||||
<enable_deflate_qpl >1</enable_deflate_qpl >
|
||||
<enable_deflate_qpl>1</enable_deflate_qpl>
|
||||
</settings>
|
||||
|
||||
<substitutions>
|
||||
@ -11,7 +11,6 @@
|
||||
<value>NONE</value> <!-- as a baseline -->
|
||||
<value>LZ4</value>
|
||||
<value>ZSTD</value>
|
||||
<value>DEFLATE_QPL</value>
|
||||
<value>DoubleDelta</value>
|
||||
<value>Gorilla</value>
|
||||
<value>FPC</value>
|
||||
|
@ -1,7 +1,7 @@
|
||||
<test>
|
||||
<settings>
|
||||
<allow_suspicious_codecs>1</allow_suspicious_codecs>
|
||||
<enable_deflate_qpl >1</enable_deflate_qpl >
|
||||
<enable_deflate_qpl>1</enable_deflate_qpl>
|
||||
</settings>
|
||||
|
||||
<substitutions>
|
||||
@ -11,7 +11,6 @@
|
||||
<value>NONE</value> <!-- as a baseline -->
|
||||
<value>LZ4</value>
|
||||
<value>ZSTD</value>
|
||||
<value>DEFLATE_QPL</value>
|
||||
<value>Delta</value>
|
||||
<value>T64</value>
|
||||
<value>DoubleDelta</value>
|
||||
|
@ -1,7 +1,7 @@
|
||||
<test>
|
||||
<settings>
|
||||
<allow_suspicious_codecs>1</allow_suspicious_codecs>
|
||||
<enable_deflate_qpl >1</enable_deflate_qpl >
|
||||
<enable_deflate_qpl>1</enable_deflate_qpl>
|
||||
</settings>
|
||||
|
||||
<substitutions>
|
||||
@ -11,7 +11,6 @@
|
||||
<value>NONE</value> <!-- as a baseline -->
|
||||
<value>LZ4</value>
|
||||
<value>ZSTD</value>
|
||||
<value>DEFLATE_QPL</value>
|
||||
<value>Delta</value>
|
||||
<value>T64</value>
|
||||
<value>DoubleDelta</value>
|
||||
|
@ -12,15 +12,13 @@ CODEC(NONE)
|
||||
2018-01-01 4 4
|
||||
2018-01-01 5 5
|
||||
2018-01-01 6 6
|
||||
CODEC(DEFLATE_QPL)
|
||||
2018-01-01 1 default_value
|
||||
2018-01-01 2 default_value
|
||||
2018-01-01 3 3
|
||||
2018-01-01 4 4
|
||||
2018-01-01 5 5
|
||||
2018-01-01 6 6
|
||||
2018-01-01 7 7
|
||||
2018-01-01 8 8
|
||||
CODEC(DEFLATE_QPL)
|
||||
2018-01-01 1 default_value
|
||||
2018-01-01 2 default_value
|
||||
2018-01-01 3 3
|
||||
|
@ -25,7 +25,10 @@ INSERT INTO alter_compression_codec VALUES('2018-01-01', 5, '5');
|
||||
INSERT INTO alter_compression_codec VALUES('2018-01-01', 6, '6');
|
||||
SELECT * FROM alter_compression_codec ORDER BY id;
|
||||
|
||||
SET enable_qpl_deflate = 1;
|
||||
OPTIMIZE TABLE alter_compression_codec FINAL;
|
||||
SELECT * FROM alter_compression_codec ORDER BY id;
|
||||
|
||||
SET enable_qpl_deflate_codec = 1;
|
||||
ALTER TABLE alter_compression_codec MODIFY COLUMN alter_column CODEC(DEFLATE_QPL);
|
||||
SELECT compression_codec FROM system.columns WHERE database = currentDatabase() AND table = 'alter_compression_codec' AND name = 'alter_column';
|
||||
|
||||
@ -33,9 +36,6 @@ INSERT INTO alter_compression_codec VALUES('2018-01-01', 7, '7');
|
||||
INSERT INTO alter_compression_codec VALUES('2018-01-01', 8, '8');
|
||||
SELECT * FROM alter_compression_codec ORDER BY id;
|
||||
|
||||
OPTIMIZE TABLE alter_compression_codec FINAL;
|
||||
SELECT * FROM alter_compression_codec ORDER BY id;
|
||||
|
||||
SET allow_suspicious_codecs = 1;
|
||||
ALTER TABLE alter_compression_codec MODIFY COLUMN alter_column CODEC(ZSTD, LZ4HC, LZ4, LZ4, DEFLATE_QPL, NONE);
|
||||
SELECT compression_codec FROM system.columns WHERE database = currentDatabase() AND table = 'alter_compression_codec' AND name = 'alter_column';
|
||||
@ -62,7 +62,7 @@ ALTER TABLE alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01
|
||||
|
||||
ALTER TABLE alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 00:00:00' CODEC(ZSTD(100)); -- { serverError 433 }
|
||||
|
||||
ALTER TABLE alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 00:00:00' CODEC(DEFLATE_QPL(100)); -- { serverError 378 }
|
||||
ALTER TABLE alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 00:00:00' CODEC(DEFLATE_QPL(100)); -- { serverError DATA_TYPE_CANNOT_HAVE_ARGUMENTS }
|
||||
|
||||
DROP TABLE IF EXISTS alter_bad_codec;
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
1 hello 2018-12-14 1.1 aaa 5 qpl11 11
|
||||
2 world 2018-12-15 2.2 bbb 6 qpl22 22
|
||||
3 ! 2018-12-16 3.3 ccc 7 qpl33 33
|
||||
1 hello 2018-12-14 2018-12-14 1.1 aaa 5
|
||||
2 world 2018-12-15 2018-12-15 2.2 bbb 6
|
||||
3 ! 2018-12-16 2018-12-16 3.3 ccc 7
|
||||
2
|
||||
1 world 2018-10-05 1.1
|
||||
2 hello 2018-10-01 2.2
|
||||
|
@ -1,6 +1,6 @@
|
||||
SET send_logs_level = 'fatal';
|
||||
SET allow_suspicious_codecs = 1;
|
||||
SET enable_qpl_deflate = 1;
|
||||
SET enable_qpl_deflate_codec = 1;
|
||||
|
||||
DROP TABLE IF EXISTS compression_codec;
|
||||
|
||||
@ -8,22 +8,21 @@ CREATE TABLE compression_codec(
|
||||
id UInt64 CODEC(LZ4),
|
||||
data String CODEC(ZSTD),
|
||||
ddd Date CODEC(NONE),
|
||||
ddd32 Date32 CODEC(DEFLATE_QPL),
|
||||
somenum Float64 CODEC(ZSTD(2)),
|
||||
somestr FixedString(3) CODEC(LZ4HC(7)),
|
||||
othernum Int64 CODEC(Delta),
|
||||
qplstr String CODEC(DEFLATE_QPL),
|
||||
qplnum UInt32 CODEC(DEFLATE_QPL),
|
||||
) ENGINE = MergeTree() ORDER BY tuple();
|
||||
|
||||
INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5, 'qpl11', 11);
|
||||
INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6,'qpl22', 22);
|
||||
INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7, 'qpl33', 33);
|
||||
INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), toDate32('2018-12-14'), 1.1, 'aaa', 5);
|
||||
INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), toDate32('2018-12-15'), 2.2, 'bbb', 6);
|
||||
INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), toDate32('2018-12-16'), 3.3, 'ccc', 7);
|
||||
|
||||
SELECT * FROM compression_codec ORDER BY id;
|
||||
|
||||
OPTIMIZE TABLE compression_codec FINAL;
|
||||
|
||||
INSERT INTO compression_codec VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8, 'qpl44', 44);
|
||||
INSERT INTO compression_codec VALUES(2, '', toDate('2018-12-13'), toDate32('2018-12-13'), 4.4, 'ddd', 8);
|
||||
|
||||
DETACH TABLE compression_codec;
|
||||
ATTACH TABLE compression_codec;
|
||||
|
@ -1,6 +1,6 @@
|
||||
SET send_logs_level = 'fatal';
|
||||
SET allow_suspicious_codecs = 1;
|
||||
SET enable_qpl_deflate = 1;
|
||||
SET enable_qpl_deflate_codec = 1;
|
||||
|
||||
-- copy-paste for storage log
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
1 hello 2018-12-14 1.1 aaa 5 qpl11 11
|
||||
2 world 2018-12-15 2.2 bbb 6 qpl22 22
|
||||
3 ! 2018-12-16 3.3 ccc 7 qpl33 33
|
||||
1 hello 2018-12-14 2018-12-14 1.1 aaa 5 [1,2,3] {'k1':1,'k2':2} (1,2)
|
||||
2 world 2018-12-15 2018-12-15 2.2 bbb 6 [4,5,6] {'k3':3,'k4':4} (3,4)
|
||||
3 ! 2018-12-16 2018-12-16 3.3 ccc 7 [7,8,9] {'k5':5,'k6':6} (5,6)
|
||||
2
|
||||
|
@ -1,5 +1,5 @@
|
||||
SET send_logs_level = 'fatal';
|
||||
SET enable_qpl_deflate = 1;
|
||||
SET enable_qpl_deflate_codec = 1;
|
||||
|
||||
DROP TABLE IF EXISTS compression_codec;
|
||||
|
||||
@ -7,22 +7,24 @@ CREATE TABLE compression_codec(
|
||||
id UInt64 CODEC(DEFLATE_QPL),
|
||||
data String CODEC(DEFLATE_QPL),
|
||||
ddd Date CODEC(DEFLATE_QPL),
|
||||
ddd32 Date32 CODEC(DEFLATE_QPL),
|
||||
somenum Float64 CODEC(DEFLATE_QPL),
|
||||
somestr FixedString(3) CODEC(DEFLATE_QPL),
|
||||
othernum Int64 CODEC(DEFLATE_QPL),
|
||||
qplstr String CODEC(DEFLATE_QPL),
|
||||
qplnum UInt32 CODEC(DEFLATE_QPL),
|
||||
somearray Array(UInt8) CODEC(DEFLATE_QPL),
|
||||
somemap Map(String, UInt32) CODEC(DEFLATE_QPL),
|
||||
sometuple Tuple(UInt16, UInt64) CODEC(DEFLATE_QPL),
|
||||
) ENGINE = MergeTree() ORDER BY tuple();
|
||||
|
||||
INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5, 'qpl11', 11);
|
||||
INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6,'qpl22', 22);
|
||||
INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7, 'qpl33', 33);
|
||||
INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), toDate32('2018-12-14'), 1.1, 'aaa', 5, [1,2,3], map('k1',1,'k2',2), tuple(1,2));
|
||||
INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), toDate32('2018-12-15'), 2.2, 'bbb', 6, [4,5,6], map('k3',3,'k4',4), tuple(3,4));
|
||||
INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), toDate32('2018-12-16'), 3.3, 'ccc', 7, [7,8,9], map('k5',5,'k6',6), tuple(5,6));
|
||||
|
||||
SELECT * FROM compression_codec ORDER BY id;
|
||||
|
||||
OPTIMIZE TABLE compression_codec FINAL;
|
||||
|
||||
INSERT INTO compression_codec VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8, 'qpl44', 44);
|
||||
INSERT INTO compression_codec VALUES(2, '', toDate('2018-12-13'), toDate32('2018-12-13'), 4.4, 'ddd', 8, [10,11,12], map('k7',7,'k8',8), tuple(7,8));
|
||||
|
||||
DETACH TABLE compression_codec;
|
||||
ATTACH TABLE compression_codec;
|
||||
|
@ -4,5 +4,3 @@
|
||||
1
|
||||
32
|
||||
1
|
||||
17
|
||||
1
|
||||
|
@ -115,41 +115,3 @@ USING(key);
|
||||
|
||||
DROP TABLE IF EXISTS delta_codec_string;
|
||||
DROP TABLE IF EXISTS default_codec_string;
|
||||
|
||||
SET enable_qpl_deflate = 1;
|
||||
DROP TABLE IF EXISTS delta_codec_string_qpl;
|
||||
DROP TABLE IF EXISTS default_codec_string_qpl;
|
||||
|
||||
CREATE TABLE delta_codec_string_qpl
|
||||
(
|
||||
id Float64 Codec(Delta, DEFLATE_QPL)
|
||||
) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false;
|
||||
|
||||
CREATE TABLE default_codec_string_qpl
|
||||
(
|
||||
id Float64 Codec(DEFLATE_QPL)
|
||||
) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false;
|
||||
|
||||
INSERT INTO delta_codec_string_qpl SELECT concat(toString(number), toString(number % 100)) FROM numbers(1547510400, 500000);
|
||||
INSERT INTO default_codec_string_qpl SELECT * from delta_codec_string_qpl;
|
||||
|
||||
OPTIMIZE TABLE delta_codec_string_qpl FINAL;
|
||||
OPTIMIZE TABLE default_codec_string_qpl FINAL;
|
||||
|
||||
SELECT
|
||||
floor(big_size / small_size) as ratio
|
||||
FROM
|
||||
(SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database = currentDatabase() and table = 'delta_codec_string_qpl' and active)
|
||||
INNER JOIN
|
||||
(SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database = currentDatabase() and table = 'default_codec_string_qpl' and active) USING(key);
|
||||
|
||||
SELECT
|
||||
small_hash == big_hash
|
||||
FROM
|
||||
(SELECT 1 AS key, sum(cityHash64(*)) AS small_hash FROM delta_codec_string_qpl)
|
||||
INNER JOIN
|
||||
(SELECT 1 AS key, sum(cityHash64(*)) AS big_hash FROM default_codec_string_qpl)
|
||||
USING(key);
|
||||
|
||||
DROP TABLE IF EXISTS delta_codec_string_qpl;
|
||||
DROP TABLE IF EXISTS default_codec_string_qpl;
|
||||
|
Loading…
Reference in New Issue
Block a user