mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge branch 'master' into lock-free-drop-partition
This commit is contained in:
commit
6fd7dcf26f
@ -388,6 +388,8 @@ else
|
||||
rm -f /etc/clickhouse-server/config.d/storage_conf.xml ||:
|
||||
rm -f /etc/clickhouse-server/config.d/azure_storage_conf.xml ||:
|
||||
|
||||
# Turn on after 22.12
|
||||
rm -f /etc/clickhouse-server/config.d/compressed_marks_and_index.xml ||:
|
||||
# it uses recently introduced settings which previous versions may not have
|
||||
rm -f /etc/clickhouse-server/users.d/insert_keeper_retries.xml ||:
|
||||
|
||||
|
@ -11,6 +11,14 @@ Projections store data in a format that optimizes query execution, this feature
|
||||
|
||||
You can define one or more projections for a table, and during the query analysis the projection with the least data to scan will be selected by ClickHouse without modifying the query provided by the user.
|
||||
|
||||
:::note Disk usage
|
||||
|
||||
Projections will create internally a new hidden table, this means that more IO and space on disk will be required.
|
||||
Example, If the projection has defined a different primary key, all the data from the original table will be duplicated.
|
||||
:::
|
||||
|
||||
You can see more technical details about how projections work internally on this [page](/docs/en/guides/improving-query-performance/sparse-primary-indexes/sparse-primary-indexes-multiple.md/#option-3-projections).
|
||||
|
||||
## Example filtering without using primary keys
|
||||
|
||||
Creating the table:
|
||||
|
@ -60,7 +60,7 @@ If you specify `POPULATE`, the existing table data is inserted into the view whe
|
||||
|
||||
A `SELECT` query can contain `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`. Note that the corresponding conversions are performed independently on each block of inserted data. For example, if `GROUP BY` is set, data is aggregated during insertion, but only within a single packet of inserted data. The data won’t be further aggregated. The exception is when using an `ENGINE` that independently performs data aggregation, such as `SummingMergeTree`.
|
||||
|
||||
The execution of [ALTER](../../../sql-reference/statements/alter/view.md) queries on materialized views has limitations, so they might be inconvenient. If the materialized view uses the construction `TO [db.]name`, you can `DETACH` the view, run `ALTER` for the target table, and then `ATTACH` the previously detached (`DETACH`) view.
|
||||
The execution of [ALTER](/docs/en/sql-reference/statements/alter/view.md) queries on materialized views has limitations, for example, you can not update the `SELECT` query, so this might be inconvenient. If the materialized view uses the construction `TO [db.]name`, you can `DETACH` the view, run `ALTER` for the target table, and then `ATTACH` the previously detached (`DETACH`) view.
|
||||
|
||||
Note that materialized view is influenced by [optimize_on_insert](../../../operations/settings/settings.md#optimize-on-insert) setting. The data is merged before the insertion into a view.
|
||||
|
||||
|
@ -164,7 +164,7 @@ SETTINGS index_granularity = 8192, index_granularity_bytes = 0;
|
||||
<li><font face = "monospace">index_granularity</font>: 显式设置为其默认值8192。这意味着对于每一组8192行,主索引将有一个索引条目,例如,如果表包含16384行,那么索引将有两个索引条目。
|
||||
</li>
|
||||
<br/>
|
||||
<li><font face = "monospace">index_granularity_bytes</font>: 设置为0表示禁止<a href="https://clickhouse.com/docs/en/whats-new/changelog/2019/#experimental-features-1" target="_blank"><font color="blue">字适应索引粒度</font></a>。自适应索引粒度意味着ClickHouse自动为一组n行创建一个索引条目
|
||||
<li><font face = "monospace">index_granularity_bytes</font>: 设置为0表示禁止<a href="https://clickhouse.com/docs/en/whats-new/changelog/2019/#experimental-features-1" target="_blank"><font color="blue">自适应索引粒度</font></a>。自适应索引粒度意味着ClickHouse自动为一组n行创建一个索引条目
|
||||
<ul>
|
||||
<li>如果n小于8192,但n行的合并行数据大小大于或等于10MB (index_granularity_bytes的默认值)或</li>
|
||||
<li>n达到8192</li>
|
||||
|
@ -155,7 +155,7 @@ public:
|
||||
"Values for {} are expected to be Numeric, Float or Decimal, passed type {}",
|
||||
getName(), value_type->getName()};
|
||||
|
||||
WhichDataType value_type_to_check(value_type);
|
||||
WhichDataType value_type_to_check(value_type_without_nullable);
|
||||
|
||||
/// Do not promote decimal because of implementation issues of this function design
|
||||
/// Currently we cannot get result column type in case of decimal we cannot get decimal scale
|
||||
|
@ -5455,25 +5455,7 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node,
|
||||
}
|
||||
}
|
||||
|
||||
/// TODO: Special functions that can take query
|
||||
/// TODO: Support qualified matchers for table function
|
||||
|
||||
for (auto & argument_node : table_function_node.getArguments().getNodes())
|
||||
{
|
||||
if (argument_node->getNodeType() == QueryTreeNodeType::MATCHER)
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Matcher as table function argument is not supported {}. In scope {}",
|
||||
join_tree_node->formatASTForErrorMessage(),
|
||||
scope.scope_node->formatASTForErrorMessage());
|
||||
}
|
||||
|
||||
auto * function_node = argument_node->as<FunctionNode>();
|
||||
if (function_node && table_function_factory.hasNameOrAlias(function_node->getFunctionName()))
|
||||
continue;
|
||||
|
||||
resolveExpressionNode(argument_node, scope, false /*allow_lambda_expression*/, true /*allow_table_expression*/);
|
||||
}
|
||||
resolveExpressionNodeList(table_function_node.getArgumentsNode(), scope, false /*allow_lambda_expression*/, true /*allow_table_expression*/);
|
||||
|
||||
auto table_function_ast = table_function_node.toAST();
|
||||
table_function_ptr->parseArguments(table_function_ast, scope_context);
|
||||
|
@ -524,11 +524,13 @@ void ColumnArray::insertRangeFrom(const IColumn & src, size_t start, size_t leng
|
||||
size_t nested_offset = src_concrete.offsetAt(start);
|
||||
size_t nested_length = src_concrete.getOffsets()[start + length - 1] - nested_offset;
|
||||
|
||||
Offsets & cur_offsets = getOffsets();
|
||||
/// Reserve offsets before to make it more exception safe (in case of MEMORY_LIMIT_EXCEEDED)
|
||||
cur_offsets.reserve(cur_offsets.size() + length);
|
||||
|
||||
getData().insertRangeFrom(src_concrete.getData(), nested_offset, nested_length);
|
||||
|
||||
Offsets & cur_offsets = getOffsets();
|
||||
const Offsets & src_offsets = src_concrete.getOffsets();
|
||||
|
||||
if (start == 0 && cur_offsets.empty())
|
||||
{
|
||||
cur_offsets.assign(src_offsets.begin(), src_offsets.begin() + length);
|
||||
|
@ -124,6 +124,9 @@ void ColumnString::insertRangeFrom(const IColumn & src, size_t start, size_t len
|
||||
size_t nested_offset = src_concrete.offsetAt(start);
|
||||
size_t nested_length = src_concrete.offsets[start + length - 1] - nested_offset;
|
||||
|
||||
/// Reserve offsets before to make it more exception safe (in case of MEMORY_LIMIT_EXCEEDED)
|
||||
offsets.reserve(offsets.size() + length);
|
||||
|
||||
size_t old_chars_size = chars.size();
|
||||
chars.resize(old_chars_size + nested_length);
|
||||
memcpy(&chars[old_chars_size], &src_concrete.chars[nested_offset], nested_length);
|
||||
|
@ -1015,7 +1015,7 @@ void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table)
|
||||
for (const auto & [disk_name, disk] : getContext()->getDisksMap())
|
||||
{
|
||||
String data_path = "store/" + getPathForUUID(table.table_id.uuid);
|
||||
if (!disk->exists(data_path) || disk->isReadOnly())
|
||||
if (disk->isReadOnly() || !disk->exists(data_path))
|
||||
continue;
|
||||
|
||||
LOG_INFO(log, "Removing data directory {} of dropped table {} from disk {}", data_path, table.table_id.getNameForLogs(), disk_name);
|
||||
|
@ -2029,8 +2029,7 @@ std::optional<std::string> getIndexExtensionFromFilesystem(const IDataPartStorag
|
||||
for (auto it = data_part_storage.iterate(); it->isValid(); it->next())
|
||||
{
|
||||
const auto & extension = fs::path(it->name()).extension();
|
||||
if (extension == getIndexExtension(false)
|
||||
|| extension == getIndexExtension(true))
|
||||
if (extension == getIndexExtension(true))
|
||||
return extension;
|
||||
}
|
||||
}
|
||||
|
6
tests/config/config.d/compressed_marks_and_index.xml
Normal file
6
tests/config/config.d/compressed_marks_and_index.xml
Normal file
@ -0,0 +1,6 @@
|
||||
<clickhouse>
|
||||
<merge_tree>
|
||||
<compress_marks>true</compress_marks>
|
||||
<compress_primary_key>true</compress_primary_key>
|
||||
</merge_tree>
|
||||
</clickhouse>
|
@ -52,6 +52,7 @@ ln -sf $SRC_PATH/config.d/enable_zero_copy_replication.xml $DEST_SERVER_PATH/con
|
||||
ln -sf $SRC_PATH/config.d/nlp.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/enable_keeper_map.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/display_name.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/compressed_marks_and_index.xml $DEST_SERVER_PATH/config.d/
|
||||
|
||||
# Not supported with fasttest.
|
||||
if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ]
|
||||
|
@ -1,5 +1,7 @@
|
||||
<clickhouse>
|
||||
<merge_tree>
|
||||
<min_bytes_for_wide_part>0</min_bytes_for_wide_part>
|
||||
<compress_marks>0</compress_marks>
|
||||
<compress_primary_key>0</compress_primary_key>
|
||||
</merge_tree>
|
||||
</clickhouse>
|
||||
|
@ -783,7 +783,7 @@ def test_cache_setting_compatibility(cluster, node_name):
|
||||
node.query("DROP TABLE IF EXISTS s3_test NO DELAY")
|
||||
|
||||
node.query(
|
||||
"CREATE TABLE s3_test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_r';"
|
||||
"CREATE TABLE s3_test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_r', compress_marks=false, compress_primary_key=false;"
|
||||
)
|
||||
node.query(
|
||||
"INSERT INTO s3_test SELECT * FROM generateRandom('key UInt32, value String') LIMIT 500"
|
||||
|
@ -1661,7 +1661,7 @@ def test_freeze(start_cluster):
|
||||
) ENGINE = MergeTree
|
||||
ORDER BY tuple()
|
||||
PARTITION BY toYYYYMM(d)
|
||||
SETTINGS storage_policy='small_jbod_with_external'
|
||||
SETTINGS storage_policy='small_jbod_with_external', compress_marks=false, compress_primary_key=false
|
||||
"""
|
||||
)
|
||||
|
||||
|
@ -38,7 +38,7 @@ def partition_table_simple(started_cluster):
|
||||
q(
|
||||
"CREATE TABLE test.partition_simple (date MATERIALIZED toDate(0), x UInt64, sample_key MATERIALIZED intHash64(x)) "
|
||||
"ENGINE=MergeTree PARTITION BY date SAMPLE BY sample_key ORDER BY (date,x,sample_key) "
|
||||
"SETTINGS index_granularity=8192, index_granularity_bytes=0"
|
||||
"SETTINGS index_granularity=8192, index_granularity_bytes=0, compress_marks=false, compress_primary_key=false"
|
||||
)
|
||||
q("INSERT INTO test.partition_simple ( x ) VALUES ( now() )")
|
||||
q("INSERT INTO test.partition_simple ( x ) VALUES ( now()+1 )")
|
||||
@ -117,7 +117,7 @@ def partition_table_complex(started_cluster):
|
||||
q("DROP TABLE IF EXISTS test.partition_complex")
|
||||
q(
|
||||
"CREATE TABLE test.partition_complex (p Date, k Int8, v1 Int8 MATERIALIZED k + 1) "
|
||||
"ENGINE = MergeTree PARTITION BY p ORDER BY k SETTINGS index_granularity=1, index_granularity_bytes=0"
|
||||
"ENGINE = MergeTree PARTITION BY p ORDER BY k SETTINGS index_granularity=1, index_granularity_bytes=0, compress_marks=false, compress_primary_key=false"
|
||||
)
|
||||
q("INSERT INTO test.partition_complex (p, k) VALUES(toDate(31), 1)")
|
||||
q("INSERT INTO test.partition_complex (p, k) VALUES(toDate(1), 2)")
|
||||
@ -155,7 +155,7 @@ def test_partition_complex(partition_table_complex):
|
||||
def cannot_attach_active_part_table(started_cluster):
|
||||
q("DROP TABLE IF EXISTS test.attach_active")
|
||||
q(
|
||||
"CREATE TABLE test.attach_active (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n"
|
||||
"CREATE TABLE test.attach_active (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n SETTINGS compress_marks=false, compress_primary_key=false"
|
||||
)
|
||||
q("INSERT INTO test.attach_active SELECT number FROM system.numbers LIMIT 16")
|
||||
|
||||
@ -183,9 +183,7 @@ def attach_check_all_parts_table(started_cluster):
|
||||
q("SYSTEM STOP MERGES")
|
||||
q("DROP TABLE IF EXISTS test.attach_partition")
|
||||
q(
|
||||
"CREATE TABLE test.attach_partition (n UInt64) "
|
||||
"ENGINE = MergeTree() "
|
||||
"PARTITION BY intDiv(n, 8) ORDER BY n"
|
||||
"CREATE TABLE test.attach_partition (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n SETTINGS compress_marks=false, compress_primary_key=false"
|
||||
)
|
||||
q(
|
||||
"INSERT INTO test.attach_partition SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8"
|
||||
@ -267,7 +265,7 @@ def drop_detached_parts_table(started_cluster):
|
||||
q("SYSTEM STOP MERGES")
|
||||
q("DROP TABLE IF EXISTS test.drop_detached")
|
||||
q(
|
||||
"CREATE TABLE test.drop_detached (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n"
|
||||
"CREATE TABLE test.drop_detached (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n SETTINGS compress_marks=false, compress_primary_key=false"
|
||||
)
|
||||
q(
|
||||
"INSERT INTO test.drop_detached SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8"
|
||||
@ -337,9 +335,15 @@ def test_drop_detached_parts(drop_detached_parts_table):
|
||||
|
||||
|
||||
def test_system_detached_parts(drop_detached_parts_table):
|
||||
q("create table sdp_0 (n int, x int) engine=MergeTree order by n")
|
||||
q("create table sdp_1 (n int, x int) engine=MergeTree order by n partition by x")
|
||||
q("create table sdp_2 (n int, x String) engine=MergeTree order by n partition by x")
|
||||
q(
|
||||
"create table sdp_0 (n int, x int) engine=MergeTree order by n SETTINGS compress_marks=false, compress_primary_key=false"
|
||||
)
|
||||
q(
|
||||
"create table sdp_1 (n int, x int) engine=MergeTree order by n partition by x SETTINGS compress_marks=false, compress_primary_key=false"
|
||||
)
|
||||
q(
|
||||
"create table sdp_2 (n int, x String) engine=MergeTree order by n partition by x SETTINGS compress_marks=false, compress_primary_key=false"
|
||||
)
|
||||
q(
|
||||
"create table sdp_3 (n int, x Enum('broken' = 0, 'all' = 1)) engine=MergeTree order by n partition by x"
|
||||
)
|
||||
@ -457,7 +461,9 @@ def test_system_detached_parts(drop_detached_parts_table):
|
||||
|
||||
|
||||
def test_detached_part_dir_exists(started_cluster):
|
||||
q("create table detached_part_dir_exists (n int) engine=MergeTree order by n")
|
||||
q(
|
||||
"create table detached_part_dir_exists (n int) engine=MergeTree order by n SETTINGS compress_marks=false, compress_primary_key=false"
|
||||
)
|
||||
q("insert into detached_part_dir_exists select 1") # will create all_1_1_0
|
||||
q(
|
||||
"alter table detached_part_dir_exists detach partition id 'all'"
|
||||
@ -499,7 +505,7 @@ def test_detached_part_dir_exists(started_cluster):
|
||||
|
||||
def test_make_clone_in_detached(started_cluster):
|
||||
q(
|
||||
"create table clone_in_detached (n int, m String) engine=ReplicatedMergeTree('/clone_in_detached', '1') order by n"
|
||||
"create table clone_in_detached (n int, m String) engine=ReplicatedMergeTree('/clone_in_detached', '1') order by n SETTINGS compress_marks=false, compress_primary_key=false"
|
||||
)
|
||||
|
||||
path = path_to_data + "data/default/clone_in_detached/"
|
||||
|
@ -728,7 +728,7 @@ def test_polymorphic_parts_index(start_cluster):
|
||||
"""
|
||||
CREATE TABLE test_index.index_compact(a UInt32, s String)
|
||||
ENGINE = MergeTree ORDER BY a
|
||||
SETTINGS min_rows_for_wide_part = 1000, index_granularity = 128, merge_max_block_size = 100"""
|
||||
SETTINGS min_rows_for_wide_part = 1000, index_granularity = 128, merge_max_block_size = 100, compress_marks=false, compress_primary_key=false"""
|
||||
)
|
||||
|
||||
node1.query(
|
||||
|
@ -1,26 +1,70 @@
|
||||
-- { echoOn }
|
||||
DROP TABLE IF EXISTS sum_map;
|
||||
CREATE TABLE sum_map(date Date, timeslot DateTime, statusMap Nested(status UInt16, requests UInt64)) ENGINE = Log;
|
||||
INSERT INTO sum_map VALUES ('2000-01-01', '2000-01-01 00:00:00', [1, 2, 3], [10, 10, 10]), ('2000-01-01', '2000-01-01 00:00:00', [3, 4, 5], [10, 10, 10]), ('2000-01-01', '2000-01-01 00:01:00', [4, 5, 6], [10, 10, 10]), ('2000-01-01', '2000-01-01 00:01:00', [6, 7, 8], [10, 10, 10]);
|
||||
SELECT * FROM sum_map ORDER BY timeslot, statusMap.status, statusMap.requests;
|
||||
2000-01-01 2000-01-01 00:00:00 [1,2,3] [10,10,10]
|
||||
2000-01-01 2000-01-01 00:00:00 [3,4,5] [10,10,10]
|
||||
2000-01-01 2000-01-01 00:01:00 [4,5,6] [10,10,10]
|
||||
2000-01-01 2000-01-01 00:01:00 [6,7,8] [10,10,10]
|
||||
SELECT sumMap(statusMap.status, statusMap.requests) FROM sum_map;
|
||||
([1,2,3,4,5,6,7,8],[10,10,20,20,20,20,10,10])
|
||||
SELECT sumMap((statusMap.status, statusMap.requests)) FROM sum_map;
|
||||
([1,2,3,4,5,6,7,8],[10,10,20,20,20,20,10,10])
|
||||
SELECT sumMapMerge(s) FROM (SELECT sumMapState(statusMap.status, statusMap.requests) AS s FROM sum_map);
|
||||
([1,2,3,4,5,6,7,8],[10,10,20,20,20,20,10,10])
|
||||
SELECT timeslot, sumMap(statusMap.status, statusMap.requests) FROM sum_map GROUP BY timeslot ORDER BY timeslot;
|
||||
2000-01-01 00:00:00 ([1,2,3,4,5],[10,10,20,10,10])
|
||||
2000-01-01 00:01:00 ([4,5,6,7,8],[10,10,20,10,10])
|
||||
SELECT timeslot, sumMap(statusMap.status, statusMap.requests).1, sumMap(statusMap.status, statusMap.requests).2 FROM sum_map GROUP BY timeslot ORDER BY timeslot;
|
||||
2000-01-01 00:00:00 [1,2,3,4,5] [10,10,20,10,10]
|
||||
2000-01-01 00:01:00 [4,5,6,7,8] [10,10,20,10,10]
|
||||
SELECT sumMapFiltered([1])(statusMap.status, statusMap.requests) FROM sum_map;
|
||||
([1],[10])
|
||||
SELECT sumMapFiltered([1, 4, 8])(statusMap.status, statusMap.requests) FROM sum_map;
|
||||
([1,4,8],[10,20,10])
|
||||
DROP TABLE sum_map;
|
||||
DROP TABLE IF EXISTS sum_map_overflow;
|
||||
CREATE TABLE sum_map_overflow(events Array(UInt8), counts Array(UInt8)) ENGINE = Log;
|
||||
INSERT INTO sum_map_overflow VALUES ([1], [255]), ([1], [2]);
|
||||
SELECT sumMap(events, counts) FROM sum_map_overflow;
|
||||
([1],[257])
|
||||
SELECT sumMapWithOverflow(events, counts) FROM sum_map_overflow;
|
||||
([1],[1])
|
||||
DROP TABLE sum_map_overflow;
|
||||
select sumMap(val, cnt) from ( SELECT [ CAST(1, 'UInt64') ] as val, [1] as cnt );
|
||||
([1],[1])
|
||||
select sumMap(val, cnt) from ( SELECT [ CAST(1, 'Float64') ] as val, [1] as cnt );
|
||||
([1],[1])
|
||||
select sumMap(val, cnt) from ( SELECT [ CAST('a', 'Enum16(\'a\'=1)') ] as val, [1] as cnt );
|
||||
(['a'],[1])
|
||||
select sumMap(val, cnt) from ( SELECT [ CAST(1, 'DateTime(\'Asia/Istanbul\')') ] as val, [1] as cnt );
|
||||
(['1970-01-01 02:00:01'],[1])
|
||||
select sumMap(val, cnt) from ( SELECT [ CAST(1, 'Date') ] as val, [1] as cnt );
|
||||
(['1970-01-02'],[1])
|
||||
select sumMap(val, cnt) from ( SELECT [ CAST('01234567-89ab-cdef-0123-456789abcdef', 'UUID') ] as val, [1] as cnt );
|
||||
(['01234567-89ab-cdef-0123-456789abcdef'],[1])
|
||||
select sumMap(val, cnt) from ( SELECT [ CAST(1.01, 'Decimal(10,2)') ] as val, [1] as cnt );
|
||||
([1.01],[1])
|
||||
select sumMap(val, cnt) from ( SELECT [ CAST('a', 'FixedString(1)'), CAST('b', 'FixedString(1)' ) ] as val, [1, 2] as cnt );
|
||||
(['a','b'],[1,2])
|
||||
select sumMap(val, cnt) from ( SELECT [ CAST('abc', 'String'), CAST('ab', 'String'), CAST('a', 'String') ] as val, [1, 2, 3] as cnt );
|
||||
(['a','ab','abc'],[3,2,1])
|
||||
DROP TABLE IF EXISTS sum_map_decimal;
|
||||
CREATE TABLE sum_map_decimal(
|
||||
statusMap Nested(
|
||||
goal_id UInt16,
|
||||
revenue Decimal32(5)
|
||||
)
|
||||
) ENGINE = Log;
|
||||
INSERT INTO sum_map_decimal VALUES ([1, 2, 3], [1.0, 2.0, 3.0]), ([3, 4, 5], [3.0, 4.0, 5.0]), ([4, 5, 6], [4.0, 5.0, 6.0]), ([6, 7, 8], [6.0, 7.0, 8.0]);
|
||||
SELECT sumMap(statusMap.goal_id, statusMap.revenue) FROM sum_map_decimal;
|
||||
([1,2,3,4,5,6,7,8],[1,2,6,8,10,12,7,8])
|
||||
SELECT sumMapWithOverflow(statusMap.goal_id, statusMap.revenue) FROM sum_map_decimal;
|
||||
([1,2,3,4,5,6,7,8],[1,2,6,8,10,12,7,8])
|
||||
DROP TABLE sum_map_decimal;
|
||||
CREATE TABLE sum_map_decimal_nullable (`statusMap` Array(Tuple(goal_id UInt16, revenue Nullable(Decimal(9, 5))))) engine=Log;
|
||||
INSERT INTO sum_map_decimal_nullable VALUES ([1, 2, 3], [1.0, 2.0, 3.0]), ([3, 4, 5], [3.0, 4.0, 5.0]), ([4, 5, 6], [4.0, 5.0, 6.0]), ([6, 7, 8], [6.0, 7.0, 8.0]);
|
||||
SELECT sumMap(statusMap.goal_id, statusMap.revenue) FROM sum_map_decimal_nullable;
|
||||
([1,2,3,4,5,6,7,8],[1,2,6,8,10,12,7,8])
|
||||
DROP TABLE sum_map_decimal_nullable;
|
||||
|
@ -1,5 +1,6 @@
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
-- { echoOn }
|
||||
DROP TABLE IF EXISTS sum_map;
|
||||
CREATE TABLE sum_map(date Date, timeslot DateTime, statusMap Nested(status UInt16, requests UInt64)) ENGINE = Log;
|
||||
|
||||
@ -54,3 +55,8 @@ SELECT sumMap(statusMap.goal_id, statusMap.revenue) FROM sum_map_decimal;
|
||||
SELECT sumMapWithOverflow(statusMap.goal_id, statusMap.revenue) FROM sum_map_decimal;
|
||||
|
||||
DROP TABLE sum_map_decimal;
|
||||
|
||||
CREATE TABLE sum_map_decimal_nullable (`statusMap` Array(Tuple(goal_id UInt16, revenue Nullable(Decimal(9, 5))))) engine=Log;
|
||||
INSERT INTO sum_map_decimal_nullable VALUES ([1, 2, 3], [1.0, 2.0, 3.0]), ([3, 4, 5], [3.0, 4.0, 5.0]), ([4, 5, 6], [4.0, 5.0, 6.0]), ([6, 7, 8], [6.0, 7.0, 8.0]);
|
||||
SELECT sumMap(statusMap.goal_id, statusMap.revenue) FROM sum_map_decimal_nullable;
|
||||
DROP TABLE sum_map_decimal_nullable;
|
||||
|
@ -12,7 +12,7 @@ CREATE TABLE check_system_tables
|
||||
ORDER BY name1
|
||||
PARTITION BY name2
|
||||
SAMPLE BY name1
|
||||
SETTINGS min_bytes_for_wide_part = 0;
|
||||
SETTINGS min_bytes_for_wide_part = 0, compress_marks=false, compress_primary_key=false;
|
||||
|
||||
SELECT name, partition_key, sorting_key, primary_key, sampling_key, storage_policy, total_rows
|
||||
FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase()
|
||||
@ -36,7 +36,8 @@ CREATE TABLE check_system_tables
|
||||
sign Int8
|
||||
) ENGINE = VersionedCollapsingMergeTree(sign, version)
|
||||
PARTITION BY date
|
||||
ORDER BY date;
|
||||
ORDER BY date
|
||||
SETTINGS compress_marks=false, compress_primary_key=false;
|
||||
|
||||
SELECT name, partition_key, sorting_key, primary_key, sampling_key
|
||||
FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase()
|
||||
|
@ -9,12 +9,12 @@ DROP TABLE IF EXISTS default_codec_synthetic;
|
||||
CREATE TABLE delta_codec_synthetic
|
||||
(
|
||||
id UInt64 Codec(Delta, ZSTD(3))
|
||||
) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0;
|
||||
) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false;
|
||||
|
||||
CREATE TABLE default_codec_synthetic
|
||||
(
|
||||
id UInt64 Codec(ZSTD(3))
|
||||
) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0;
|
||||
) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false;
|
||||
|
||||
INSERT INTO delta_codec_synthetic SELECT number FROM system.numbers LIMIT 5000000;
|
||||
INSERT INTO default_codec_synthetic SELECT number FROM system.numbers LIMIT 5000000;
|
||||
@ -47,12 +47,12 @@ DROP TABLE IF EXISTS default_codec_float;
|
||||
CREATE TABLE delta_codec_float
|
||||
(
|
||||
id Float64 Codec(Delta, LZ4HC)
|
||||
) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0;
|
||||
) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false;
|
||||
|
||||
CREATE TABLE default_codec_float
|
||||
(
|
||||
id Float64 Codec(LZ4HC)
|
||||
) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0;
|
||||
) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false;
|
||||
|
||||
INSERT INTO delta_codec_float SELECT number FROM numbers(1547510400, 500000) WHERE number % 3 == 0 OR number % 5 == 0 OR number % 7 == 0 OR number % 11 == 0;
|
||||
INSERT INTO default_codec_float SELECT * from delta_codec_float;
|
||||
@ -85,12 +85,12 @@ DROP TABLE IF EXISTS default_codec_string;
|
||||
CREATE TABLE delta_codec_string
|
||||
(
|
||||
id Float64 Codec(Delta, LZ4)
|
||||
) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0;
|
||||
) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false;
|
||||
|
||||
CREATE TABLE default_codec_string
|
||||
(
|
||||
id Float64 Codec(LZ4)
|
||||
) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0;
|
||||
) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false;
|
||||
|
||||
INSERT INTO delta_codec_string SELECT concat(toString(number), toString(number % 100)) FROM numbers(1547510400, 500000);
|
||||
INSERT INTO default_codec_string SELECT * from delta_codec_string;
|
||||
|
@ -1,7 +1,7 @@
|
||||
DROP TABLE IF EXISTS test_00961;
|
||||
|
||||
CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32)
|
||||
ENGINE = MergeTree PARTITION BY d ORDER BY (a, b) SETTINGS index_granularity = 111, min_bytes_for_wide_part = 0;
|
||||
ENGINE = MergeTree PARTITION BY d ORDER BY (a, b) SETTINGS index_granularity = 111, min_bytes_for_wide_part = 0, compress_marks=false, compress_primary_key=false;
|
||||
|
||||
INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
|
||||
|
||||
|
@ -7,7 +7,7 @@ CREATE TABLE test_table
|
||||
INDEX value_index value TYPE minmax GRANULARITY 1
|
||||
)
|
||||
Engine=MergeTree()
|
||||
ORDER BY key;
|
||||
ORDER BY key SETTINGS compress_marks=false;
|
||||
|
||||
INSERT INTO test_table VALUES (0, 'Value');
|
||||
SELECT * FROM system.data_skipping_indices WHERE database = currentDatabase();
|
||||
|
@ -3,7 +3,7 @@
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_6', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_6', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test FORMAT Null;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size;
|
||||
|
@ -6,7 +6,7 @@ SYSTEM DROP FILESYSTEM CACHE;
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_6', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_6', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
|
||||
SELECT * FROM test FORMAT Null;
|
||||
|
@ -5,7 +5,7 @@ SET enable_filesystem_cache_on_write_operations=0;
|
||||
SET skip_download_if_exceeds_query_cache=1;
|
||||
SET max_query_cache_size=128;
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_4', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_4', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test FORMAT Null;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size;
|
||||
|
@ -8,7 +8,7 @@ SET skip_download_if_exceeds_query_cache=1;
|
||||
SET max_query_cache_size=128;
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_4', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_4', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
|
||||
SELECT * FROM test FORMAT Null;
|
||||
|
@ -4,7 +4,7 @@ Using storage policy: s3_cache
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test FORMAT Null;
|
||||
@ -19,7 +19,7 @@ SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesy
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache;
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_3', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_3', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test FORMAT Null;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size;
|
||||
@ -39,7 +39,7 @@ Using storage policy: local_cache
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='local_cache', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='local_cache', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test FORMAT Null;
|
||||
@ -54,7 +54,7 @@ SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesy
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache;
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='local_cache_3', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='local_cache_3', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test FORMAT Null;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size;
|
||||
|
@ -3,7 +3,7 @@ Using storage policy: s3_cache
|
||||
|
||||
SET enable_filesystem_cache_on_write_operations=1;
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test;
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state
|
||||
@ -129,7 +129,7 @@ Using storage policy: local_cache
|
||||
|
||||
SET enable_filesystem_cache_on_write_operations=1;
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='local_cache', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='local_cache', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test;
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state
|
||||
|
@ -6,7 +6,7 @@ SET enable_filesystem_cache_log=1;
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
DROP TABLE IF EXISTS test;
|
||||
DROP TABLE IF EXISTS system.filesystem_cache_log;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100000);
|
||||
SELECT 2240, 's3_cache', * FROM test FORMAT Null;
|
||||
@ -27,7 +27,7 @@ SET enable_filesystem_cache_log=1;
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
DROP TABLE IF EXISTS test;
|
||||
DROP TABLE IF EXISTS system.filesystem_cache_log;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='local_cache', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='local_cache', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100000);
|
||||
SELECT 2240, 'local_cache', * FROM test FORMAT Null;
|
||||
|
@ -1,7 +1,7 @@
|
||||
-- Tags: no-backward-compatibility-check
|
||||
|
||||
drop table if exists test_02381;
|
||||
create table test_02381(a UInt64, b UInt64) ENGINE = MergeTree order by (a, b);
|
||||
create table test_02381(a UInt64, b UInt64) ENGINE = MergeTree order by (a, b) SETTINGS compress_marks=false, compress_primary_key=false;
|
||||
insert into test_02381 select number, number * 10 from system.numbers limit 1000000;
|
||||
|
||||
drop table if exists test_02381_compress;
|
||||
|
@ -3,7 +3,7 @@
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
DROP TABLE IF EXISTS nopers;
|
||||
CREATE TABLE nopers (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE nopers (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES nopers;
|
||||
INSERT INTO nopers SELECT number, toString(number) FROM numbers(10);
|
||||
SELECT * FROM nopers FORMAT Null;
|
||||
@ -22,7 +22,7 @@ ORDER BY file, cache, size;
|
||||
data.bin 0 114
|
||||
data.mrk3 0 80
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test FORMAT Null;
|
||||
@ -47,7 +47,7 @@ data.bin 0 746
|
||||
data.mrk3 0 80
|
||||
data.mrk3 0_persistent 80
|
||||
DROP TABLE IF EXISTS test2;
|
||||
CREATE TABLE test2 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test2 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test2;
|
||||
INSERT INTO test2 SELECT number, toString(number) FROM numbers(100000);
|
||||
SELECT * FROM test2 FORMAT Null;
|
||||
|
@ -7,7 +7,7 @@ SET enable_filesystem_cache_on_write_operations=0;
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
|
||||
DROP TABLE IF EXISTS nopers;
|
||||
CREATE TABLE nopers (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE nopers (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES nopers;
|
||||
|
||||
INSERT INTO nopers SELECT number, toString(number) FROM numbers(10);
|
||||
@ -26,7 +26,7 @@ ON data_paths.cache_path = caches.cache_path
|
||||
ORDER BY file, cache, size;
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test;
|
||||
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
@ -49,7 +49,7 @@ ON data_paths.cache_path = caches.cache_path
|
||||
ORDER BY file, cache, size;
|
||||
|
||||
DROP TABLE IF EXISTS test2;
|
||||
CREATE TABLE test2 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test2 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test2;
|
||||
|
||||
INSERT INTO test2 SELECT number, toString(number) FROM numbers(100000);
|
||||
|
@ -0,0 +1,28 @@
|
||||
0
|
||||
--
|
||||
0
|
||||
1
|
||||
--
|
||||
1
|
||||
2
|
||||
--
|
||||
(1) 0
|
||||
--
|
||||
(0,1) 0
|
||||
--
|
||||
(1,2) 1
|
||||
(1,2) 2
|
||||
--
|
||||
(1) 0
|
||||
--
|
||||
(0,1) 0
|
||||
--
|
||||
(1,2) 1
|
||||
(1,2) 2
|
||||
--
|
||||
('1') 0
|
||||
--
|
||||
('0','1') 0
|
||||
--
|
||||
('1','2') 1
|
||||
('1','2') 2
|
@ -0,0 +1,47 @@
|
||||
SET allow_experimental_analyzer = 1;
|
||||
|
||||
SELECT number FROM numbers(untuple(tuple(1)));
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT number FROM numbers(untuple(tuple(0, 2)));
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT number FROM numbers(untuple(tuple(1, 2)));
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT cast(tuple(1), 'Tuple(value UInt64)') AS value, number FROM numbers(untuple(value));
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT cast(tuple(0, 1), 'Tuple(value_1 UInt64, value_2 UInt64)') AS value, number FROM numbers(untuple(value));
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT cast(tuple(1, 2), 'Tuple(value_1 UInt64, value_2 UInt64)') AS value, number FROM numbers(untuple(value));
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT cast(tuple(1), 'Tuple(value UInt64)') AS value, number FROM numbers(value.*);
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT cast(tuple(0, 1), 'Tuple(value_1 UInt64, value_2 UInt64)') AS value, number FROM numbers(value.*);
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT cast(tuple(1, 2), 'Tuple(value_1 UInt64, value_2 UInt64)') AS value, number FROM numbers(value.*);
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT cast(tuple('1'), 'Tuple(value String)') AS value, number FROM numbers(value.* APPLY x -> toUInt64(x));
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT cast(tuple('0', '1'), 'Tuple(value_1 String, value_2 String)') AS value, number FROM numbers(value.* APPLY x -> toUInt64(x));
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT cast(tuple('1', '2'), 'Tuple(value_1 String, value_2 String)') AS value, number FROM numbers(value.* APPLY x -> toUInt64(x));
|
@ -3,7 +3,7 @@
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='_storagePolicy', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='_storagePolicy', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
|
||||
@ -18,7 +18,7 @@ SYSTEM DROP FILESYSTEM CACHE;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache;
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='_storagePolicy_3', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='_storagePolicy_3', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test FORMAT Null;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size;
|
||||
|
@ -3,7 +3,7 @@
|
||||
SET enable_filesystem_cache_on_write_operations=1;
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='_storagePolicy', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='_storagePolicy', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test;
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state
|
||||
|
@ -6,7 +6,7 @@ SET enable_filesystem_cache_on_write_operations=0;
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
DROP TABLE IF EXISTS system.filesystem_cache_log;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='_storagePolicy', min_bytes_for_wide_part = 10485760;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='_storagePolicy', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100000);
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user