Merge branch 'master' into is-secure-client-info

This commit is contained in:
mergify[bot] 2022-03-30 15:12:56 +00:00 committed by GitHub
commit 4813f0b305
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 36 additions and 5 deletions

View File

@ -6,6 +6,7 @@
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/ApplyWithSubqueryVisitor.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ParserCreateQuery.h>
@ -55,6 +56,9 @@ std::pair<String, StoragePtr> createTableFromAST(
ast_create_query.attach = true;
ast_create_query.setDatabase(database_name);
if (ast_create_query.select && ast_create_query.isView())
ApplyWithSubqueryVisitor().visit(*ast_create_query.select);
if (ast_create_query.as_table_function)
{
const auto & factory = TableFunctionFactory::instance();

View File

@ -116,7 +116,7 @@ def clickhouse_execute_http(base_args, query, timeout=30, settings=None, default
def clickhouse_execute(base_args, query, timeout=30, settings=None):
return clickhouse_execute_http(base_args, query, timeout, settings).strip()
def clickhouse_execute_json(base_args, query, timeout=30, settings=None):
def clickhouse_execute_json(base_args, query, timeout=60, settings=None):
data = clickhouse_execute_http(base_args, query, timeout, settings, 'JSONEachRow')
if not data:
return None

View File

@ -0,0 +1,25 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance("node", stay_alive=True)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_materialized_view_with_subquery(start_cluster):
node.query("create table test (x UInt32) engine=TinyLog()")
node.query(
"create materialized view mv engine = TinyLog() as with subquery as (select * from test) select * from subquery"
)
node.restart_clickhouse(kill=True)
node.query("insert into test select 1")
result = node.query("select * from mv")
assert int(result) == 1

View File

@ -21,3 +21,5 @@ $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 ASYNC"
$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 FORMAT TabSeparated"
$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 SYNC FORMAT TabSeparated"
$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 1 TEST" &>/dev/null
clickhouse_test_wait_queries 60

View File

@ -23,7 +23,7 @@ function run_and_check()
echo "Checking $*"
# Run query with external table (implicit StorageMemory user)
$CLICKHOUSE_CURL -sS -F "s=@$tmp_file;" "$CLICKHOUSE_URL&s_structure=key+Int&query=SELECT+count()+FROM+s&memory_profiler_sample_probability=1&query_id=$query_id&$*" -o /dev/null
$CLICKHOUSE_CURL -sS -F "s=@$tmp_file;" "$CLICKHOUSE_URL&s_structure=key+Int&query=SELECT+count()+FROM+s&memory_profiler_sample_probability=1&max_untracked_memory=0&query_id=$query_id&$*" -o /dev/null
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @- <<<'SYSTEM FLUSH LOGS'

View File

@ -84,14 +84,14 @@ let render_data_query = `
SELECT groupArray([d, n, fail]) FROM
(
SELECT n, check_start_time::Date - start_date AS d, max(test_status LIKE 'F%' OR test_status LIKE 'E%') AS fail
FROM "gh-data".checks
FROM "default".checks
INNER JOIN
(
SELECT test_name, toUInt16(rowNumberInAllBlocks()) AS n FROM
(
SELECT DISTINCT test_name
FROM "gh-data".checks
FROM "default".checks
WHERE match(test_name, '^\\d+_') AND check_name ILIKE '%stateless%' AND check_start_time > now() - INTERVAL 1 DAY
ORDER BY test_name
)
@ -112,7 +112,7 @@ let test_names_query = `
SELECT test_name, toUInt16(rowNumberInAllBlocks()) AS n FROM
(
SELECT DISTINCT test_name
FROM "gh-data".checks
FROM "default".checks
WHERE match(test_name, '^\\d+_') AND check_name ILIKE '%stateless%' AND check_start_time > now() - INTERVAL 1 DAY
ORDER BY test_name
) FORMAT JSONCompact`;