Skip to content

Commit 773fc0e

Browse files
Merge branch 'fix-test-01287_max_execution_speed' into stripe-log-use-new-serialization-formats
2 parents a7fe389 + f1c6275 commit 773fc0e

17 files changed

+177
-13
lines changed

docs/en/interfaces/schema-inference.md

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1449,6 +1449,10 @@ DESC format(JSONEachRow, $$
14491449

14501450
#### input_format_try_infer_integers {#input-format-try-infer-integers}
14511451

1452+
:::note
1453+
This setting does not apply to the `JSON` data type.
1454+
:::
1455+
14521456
If enabled, ClickHouse will try to infer integers instead of floats in schema inference for text formats.
14531457
If all numbers in the column from sample data are integers, the result type will be `Int64`, if at least one number is float, the result type will be `Float64`.
14541458
If the sample data contains only integers and at least one integer is positive and overflows `Int64`, ClickHouse will infer `UInt64`.

src/Analyzer/ConstantNode.cpp

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -199,12 +199,12 @@ QueryTreeNodePtr ConstantNode::cloneImpl() const
199199

200200
ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const
201201
{
202+
if (!options.add_cast_for_constants)
203+
return std::make_shared<ASTLiteral>(getFieldFromColumnForASTLiteral(constant_value.getColumn(), 0, constant_value.getType()));
204+
202205
const auto & constant_value_type = constant_value.getType();
203206
auto constant_value_ast = std::make_shared<ASTLiteral>(getValue());
204207

205-
if (!options.add_cast_for_constants)
206-
return constant_value_ast;
207-
208208
// Add cast if constant was created as a result of constant folding.
209209
// Constant folding may lead to type transformation and literal on shard
210210
// may have a different type.

src/Processors/QueryPlan/Optimizations/filterPushDown.cpp

Lines changed: 35 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -387,8 +387,33 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan::
387387

388388
size_t updated_steps = 0;
389389

390+
/// For the logical join step, we need to merge pre-join actions to filter dag.
391+
/// TODO: this should be refactored and replaced with optimizations which
392+
/// 1. push filter/expression into JOIN (as post-filter)
393+
/// 2. move filter within JOIN step, potentially changing JoinKind
394+
/// 3. push filter/expression out of JOIN (from pre-filter)
395+
auto fix_predicate_for_join_logical_step = [&](ActionsDAG filter_dag, const ActionsDAG & side_dag)
396+
{
397+
filter_dag = ActionsDAG::merge(side_dag.clone(), std::move(filter_dag));
398+
auto & outputs = filter_dag.getOutputs();
399+
outputs.resize(1);
400+
outputs.insert(outputs.end(), filter_dag.getInputs().begin(), filter_dag.getInputs().end());
401+
filter_dag.removeUnusedActions();
402+
return filter_dag;
403+
};
404+
390405
if (join_filter_push_down_actions.left_stream_filter_to_push_down)
391406
{
407+
if (logical_join)
408+
{
409+
410+
join_filter_push_down_actions.left_stream_filter_to_push_down = fix_predicate_for_join_logical_step(
411+
std::move(*join_filter_push_down_actions.left_stream_filter_to_push_down),
412+
*logical_join->getExpressionActions().left_pre_join_actions
413+
);
414+
join_filter_push_down_actions.left_stream_filter_removes_filter = true;
415+
}
416+
392417
const auto & result_name = join_filter_push_down_actions.left_stream_filter_to_push_down->getOutputs()[0]->result_name;
393418
updated_steps += addNewFilterStepOrThrow(parent_node,
394419
nodes,
@@ -404,6 +429,16 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan::
404429

405430
if (join_filter_push_down_actions.right_stream_filter_to_push_down && allow_push_down_to_right)
406431
{
432+
if (logical_join)
433+
{
434+
435+
join_filter_push_down_actions.right_stream_filter_to_push_down = fix_predicate_for_join_logical_step(
436+
std::move(*join_filter_push_down_actions.right_stream_filter_to_push_down),
437+
*logical_join->getExpressionActions().right_pre_join_actions
438+
);
439+
join_filter_push_down_actions.right_stream_filter_removes_filter = true;
440+
}
441+
407442
const auto & result_name = join_filter_push_down_actions.right_stream_filter_to_push_down->getOutputs()[0]->result_name;
408443
updated_steps += addNewFilterStepOrThrow(parent_node,
409444
nodes,

src/Storages/tests/gtest_transform_query_for_external_database.cpp

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -301,8 +301,7 @@ TEST(TransformQueryForExternalDatabase, Issue7245)
301301

302302
check(state, 1, {"apply_id", "apply_type", "apply_status", "create_time"},
303303
"SELECT apply_id FROM test.table WHERE apply_type = 2 AND create_time > addDays(toDateTime('2019-01-01 01:02:03', 'UTC'),-7) AND apply_status IN (3,4)",
304-
R"(SELECT "apply_id", "apply_type", "apply_status", "create_time" FROM "test"."table" WHERE ("apply_type" = 2) AND ("create_time" > '2018-12-25 01:02:03') AND ("apply_status" IN (3, 4)))",
305-
R"(SELECT "apply_id", "apply_type", "apply_status", "create_time" FROM "test"."table" WHERE ("apply_type" = 2) AND ("create_time" > 1545699723) AND ("apply_status" IN (3, 4)))");
304+
R"(SELECT "apply_id", "apply_type", "apply_status", "create_time" FROM "test"."table" WHERE ("apply_type" = 2) AND ("create_time" > '2018-12-25 01:02:03') AND ("apply_status" IN (3, 4)))");
306305
}
307306

308307
TEST(TransformQueryForExternalDatabase, Aliases)
@@ -396,8 +395,7 @@ TEST(TransformQueryForExternalDatabase, ToDate)
396395

397396
check(state, 1, {"a", "b", "foo"},
398397
"SELECT foo FROM table WHERE a=10 AND b=toDate('2019-10-05', 'UTC')",
399-
R"(SELECT "a", "b", "foo" FROM "test"."table" WHERE ("a" = 10) AND ("b" = '2019-10-05'))",
400-
R"(SELECT "a", "b", "foo" FROM "test"."table" WHERE ("a" = 10) AND ("b" = 18174))");
398+
R"(SELECT "a", "b", "foo" FROM "test"."table" WHERE ("a" = 10) AND ("b" = '2019-10-05'))");
401399
}
402400

403401
TEST(TransformQueryForExternalDatabase, Analyzer)
@@ -422,8 +420,7 @@ TEST(TransformQueryForExternalDatabase, Analyzer)
422420

423421
check(state, 1, {"is_value"},
424422
"SELECT is_value FROM table WHERE is_value = true",
425-
R"(SELECT "is_value" FROM "test"."table" WHERE "is_value" = true)",
426-
R"(SELECT "is_value" FROM "test"."table" WHERE "is_value" = 1)");
423+
R"(SELECT "is_value" FROM "test"."table" WHERE "is_value" = true)");
427424

428425
check(state, 1, {"is_value"},
429426
"SELECT is_value FROM table WHERE is_value = 1",
Lines changed: 26 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,26 @@
1+
<clickhouse>
2+
<storage_configuration>
3+
<disks>
4+
<disk_s3_plain_rewritable_03517>
5+
<type>object_storage</type>
6+
<object_storage_type>s3</object_storage_type>
7+
<metadata_type>plain_rewritable</metadata_type>
8+
<endpoint>http://localhost:11111/test/common/</endpoint>
9+
<access_key_id>clickhouse</access_key_id>
10+
<secret_access_key>clickhouse</secret_access_key>
11+
</disk_s3_plain_rewritable_03517>
12+
<disk_cache_03517>
13+
<type>cache</type>
14+
<disk>disk_s3_plain_rewritable_03517</disk>
15+
<path>disks/cache_03517/</path>
16+
<max_size>1Mi</max_size>
17+
<cache_on_write_operations>1</cache_on_write_operations>
18+
</disk_cache_03517>
19+
<disk_encrypted_03517>
20+
<type>encrypted</type>
21+
<disk>disk_cache_03517</disk>
22+
<key>1234567812345678</key>
23+
</disk_encrypted_03517>
24+
</disks>
25+
</storage_configuration>
26+
</clickhouse>

tests/config/install.sh

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -255,6 +255,7 @@ if [[ "$EXPORT_S3_STORAGE_POLICIES" == "1" ]]; then
255255
fi
256256
ln -sf $SRC_PATH/config.d/storage_conf_02963.xml $DEST_SERVER_PATH/config.d/
257257
ln -sf $SRC_PATH/config.d/storage_conf_02961.xml $DEST_SERVER_PATH/config.d/
258+
ln -sf $SRC_PATH/config.d/storage_conf_03517.xml $DEST_SERVER_PATH/config.d/
258259
ln -sf $SRC_PATH/users.d/s3_cache.xml $DEST_SERVER_PATH/users.d/
259260
ln -sf $SRC_PATH/users.d/s3_cache_new.xml $DEST_SERVER_PATH/users.d/
260261
fi

tests/integration/test_storage_postgresql/test.py

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -869,6 +869,29 @@ def _create_and_fill_table(table):
869869
cursor.execute(f'DROP TABLE "{table}\'"')
870870

871871

872+
def test_postgres_datetime(started_cluster):
873+
cursor = started_cluster.postgres_conn.cursor()
874+
cursor.execute(f"DROP TABLE IF EXISTS test_datetime")
875+
cursor.execute("CREATE TABLE test_datetime AS (SELECT '2025-01-02 03:04:05.678900'::timestamptz AS ts, '2025-01-02'::date as d)")
876+
877+
node1.query("DROP TABLE IF EXISTS test_datetime")
878+
node1.query(
879+
f"CREATE TABLE test_datetime (ts DateTime64(6, 'UTC'), d Date) ENGINE = PostgreSQL('postgres1:5432', 'postgres', 'test_datetime', 'postgres', '{pg_pass}')"
880+
)
881+
882+
result = node1.query("SELECT ts FROM test_datetime WHERE ts > '2025-01-01'::DateTime")
883+
assert result == "2025-01-02 03:04:05.678900\n"
884+
885+
result = node1.query("SELECT ts FROM test_datetime WHERE ts > '2025-01-01'::DateTime64")
886+
assert result == "2025-01-02 03:04:05.678900\n"
887+
888+
result = node1.query("SELECT ts FROM test_datetime WHERE ts > '2025-01-01'::Nullable(DateTime)")
889+
assert result == "2025-01-02 03:04:05.678900\n"
890+
891+
result = node1.query("SELECT ts FROM test_datetime WHERE ts > '2025-01-01'::Nullable(DateTime64)")
892+
assert result == "2025-01-02 03:04:05.678900\n"
893+
894+
872895
if __name__ == "__main__":
873896
cluster.start()
874897
input("Cluster created, press any key to destroy...")

tests/queries/0_stateless/01287_max_execution_speed.sql

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@ SELECT count() FROM system.numbers; -- { serverError TOO_SLOW }
1313
SET min_execution_speed_bytes = 0;
1414
SELECT 'Ok (2)';
1515

16+
SET timeout_before_checking_execution_speed = 60;
1617
SET max_execution_speed = 1000000;
1718
SET max_block_size = 100;
1819

tests/queries/0_stateless/03322_initial_query_start_time_check.sql

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -2,8 +2,8 @@
22

33
DROP TABLE IF EXISTS tmp;
44

5-
CREATE OR REPLACE VIEW tmp AS SELECT initialQueryStartTime() as it, now() AS t FROM system.one WHERE NOT ignore(sleep(0.1));
6-
SELECT now()==max(t), initialQueryStartTime()==max(it), initialQueryStartTime()==min(it), initialQueryStartTime()>=now()-1 FROM (SELECT it, t FROM remote('127.0.0.{1..20}', currentDatabase(), tmp)) SETTINGS max_distributed_connections=1, async_socket_for_remote=0;
5+
CREATE OR REPLACE VIEW tmp AS SELECT initialQueryStartTime() as it, now() AS t FROM system.one WHERE NOT ignore(sleep(0.5));
6+
SELECT now()==max(t), initialQueryStartTime()==max(it), initialQueryStartTime()==min(it), initialQueryStartTime()>=now()-1 FROM (SELECT it, t FROM remote('127.0.0.{1..10}', currentDatabase(), tmp)) SETTINGS max_distributed_connections=1, async_socket_for_remote=0;
77

88
DROP TABLE tmp;
99

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,2 @@
1+
1
2+
1

0 commit comments

Comments
 (0)