diff --git a/docker/test/integration/runner/compose/docker_compose_mongo.yml b/docker/test/integration/runner/compose/docker_compose_mongo.yml index 732734abcbe..0bdd054420a 100644 --- a/docker/test/integration/runner/compose/docker_compose_mongo.yml +++ b/docker/test/integration/runner/compose/docker_compose_mongo.yml @@ -9,3 +9,9 @@ services: ports: - ${MONGO_EXTERNAL_PORT}:${MONGO_INTERNAL_PORT} command: --profile=2 --verbose + + mongo2: + image: mongo:5.0 + restart: always + ports: + - "27018:27017" diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 1f27072f142..33615080df4 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -388,12 +388,6 @@ void LocalServer::setupUsers() } -String LocalServer::getQueryTextPrefix() -{ - return getInitialCreateTableQuery(); -} - - void LocalServer::connect() { connection_parameters = ConnectionParameters(config()); @@ -463,6 +457,10 @@ try } #endif + String initial_query = getInitialCreateTableQuery(); + if (!initial_query.empty()) + processQueryText(initial_query); + if (is_interactive && !delayed_interactive) { runInteractive(); diff --git a/programs/local/LocalServer.h b/programs/local/LocalServer.h index ce0df06c86a..06e3746eacd 100644 --- a/programs/local/LocalServer.h +++ b/programs/local/LocalServer.h @@ -37,7 +37,6 @@ protected: void processError(const String & query) const override; String getName() const override { return "local"; } - String getQueryTextPrefix() override; void printHelpMessage(const OptionsDescription & options_description) override; void addOptions(OptionsDescription & options_description) override; diff --git a/src/Access/EnabledQuota.cpp b/src/Access/EnabledQuota.cpp index f0c6004bd77..5551b6dca40 100644 --- a/src/Access/EnabledQuota.cpp +++ b/src/Access/EnabledQuota.cpp @@ -65,9 +65,7 @@ struct EnabledQuota::Impl end = end + duration * n; if (end_of_interval.compare_exchange_strong(end_loaded, end.time_since_epoch())) { - /// We reset counters only if the interval's end has been calculated before. - /// If it hasn't we just calculate the interval's end for the first time and don't reset counters yet. - need_reset_counters = (end_loaded.count() != 0); + need_reset_counters = true; break; } end = std::chrono::system_clock::time_point{end_loaded}; @@ -93,11 +91,20 @@ struct EnabledQuota::Impl { for (const auto & interval : intervals.intervals) { + if (!interval.end_of_interval.load().count()) + { + /// We need to calculate end of the interval if it hasn't been calculated before. + bool dummy; + getEndOfInterval(interval, current_time, dummy); + } + auto quota_type_i = static_cast(quota_type); QuotaValue used = (interval.used[quota_type_i] += value); QuotaValue max = interval.max[quota_type_i]; + if (!max) continue; + if (used > max) { bool counters_were_reset = false; @@ -123,10 +130,19 @@ struct EnabledQuota::Impl auto quota_type_i = static_cast(quota_type); for (const auto & interval : intervals.intervals) { + if (!interval.end_of_interval.load().count()) + { + /// We need to calculate end of the interval if it hasn't been calculated before. + bool dummy; + getEndOfInterval(interval, current_time, dummy); + } + QuotaValue used = interval.used[quota_type_i]; QuotaValue max = interval.max[quota_type_i]; + if (!max) continue; + if (used > max) { bool counters_were_reset = false; diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index b97d8342186..4052531c493 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1494,17 +1494,14 @@ void ClientBase::runNonInteractive() { auto process_multi_query_from_file = [&](const String & file) { - auto text = getQueryTextPrefix(); String queries_from_file; ReadBufferFromFile in(file); readStringUntilEOF(queries_from_file, in); - text += queries_from_file; - return executeMultiQuery(text); + return executeMultiQuery(queries_from_file); }; - /// Read all queries into `text`. for (const auto & queries_file : queries_files) { for (const auto & interleave_file : interleave_queries_files) @@ -1519,9 +1516,6 @@ void ClientBase::runNonInteractive() } String text; - if (is_multiquery) - text = getQueryTextPrefix(); - if (config().has("query")) { text += config().getRawString("query"); /// Poco configuration should not process substitutions in form of ${...} inside query. diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index bad1395e699..4c5d29b390b 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -78,9 +78,6 @@ protected: String & query_to_execute, ASTPtr & parsed_query, const String & all_queries_text, std::optional & current_exception); - /// For non-interactive multi-query mode get queries text prefix. - virtual String getQueryTextPrefix() { return ""; } - static void clearTerminal(); void showClientVersion(); @@ -100,9 +97,10 @@ protected: const std::vector & external_tables_arguments) = 0; virtual void processConfig() = 0; -private: +protected: bool processQueryText(const String & text); +private: void receiveResult(ASTPtr parsed_query); bool receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled); void receiveLogs(ASTPtr parsed_query); diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 54785f92926..84a796d03d9 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -601,6 +601,7 @@ M(631, UNKNOWN_FILE_SIZE) \ M(632, UNEXPECTED_DATA_AFTER_PARSED_VALUE) \ M(633, QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW) \ + M(634, MONGODB_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.cpp b/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.cpp index 70a58971d3f..383ca3db6f4 100644 --- a/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.cpp +++ b/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.cpp @@ -157,7 +157,12 @@ void ArithmeticOperationsInAgrFuncMatcher::visit(const ASTFunction & func, ASTPt void ArithmeticOperationsInAgrFuncMatcher::visit(ASTPtr & ast, Data & data) { if (const auto * function_node = ast->as()) + { + if (function_node->is_window_function) + return; + visit(*function_node, ast, data); + } } bool ArithmeticOperationsInAgrFuncMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &) diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index 0440c52797c..802bf4e43ce 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -51,7 +51,7 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v } } } - else if (function->name == "toUInt8" || function->name == "toInt8") + else if (function->name == "toUInt8" || function->name == "toInt8" || function->name == "identity") { if (const auto * expr_list = function->arguments->as()) { diff --git a/src/Interpreters/RewriteAnyFunctionVisitor.cpp b/src/Interpreters/RewriteAnyFunctionVisitor.cpp index eed6368ae54..5eb14aa4252 100644 --- a/src/Interpreters/RewriteAnyFunctionVisitor.cpp +++ b/src/Interpreters/RewriteAnyFunctionVisitor.cpp @@ -63,7 +63,12 @@ bool extractIdentifiers(const ASTFunction & func, std::unordered_set & void RewriteAnyFunctionMatcher::visit(ASTPtr & ast, Data & data) { if (auto * func = ast->as()) + { + if (func->is_window_function) + return; + visit(*func, ast, data); + } } void RewriteAnyFunctionMatcher::visit(const ASTFunction & func, ASTPtr & ast, Data & data) diff --git a/src/Interpreters/RewriteSumIfFunctionVisitor.cpp b/src/Interpreters/RewriteSumIfFunctionVisitor.cpp index 7b322ca1585..7f725c1d8a5 100644 --- a/src/Interpreters/RewriteSumIfFunctionVisitor.cpp +++ b/src/Interpreters/RewriteSumIfFunctionVisitor.cpp @@ -10,7 +10,12 @@ namespace DB void RewriteSumIfFunctionMatcher::visit(ASTPtr & ast, Data & data) { if (auto * func = ast->as()) + { + if (func->is_window_function) + return; + visit(*func, ast, data); + } } void RewriteSumIfFunctionMatcher::visit(const ASTFunction & func, ASTPtr & ast, Data &) diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index bf20bef6992..b571a8e8e10 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -225,7 +225,13 @@ void removeColumnNullability(ColumnWithTypeAndName & column) if (column.column && column.column->isNullable()) { + column.column = column.column->convertToFullColumnIfConst(); const auto * nullable_col = checkAndGetColumn(*column.column); + if (!nullable_col) + { + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Column '{}' is expected to be nullable", column.dumpStructure()); + } + MutableColumnPtr mutable_column = nullable_col->getNestedColumn().cloneEmpty(); insertFromNullableOrDefault(mutable_column, nullable_col); column.column = std::move(mutable_column); diff --git a/src/Processors/Transforms/MongoDBSource.cpp b/src/Processors/Transforms/MongoDBSource.cpp index 30ad9746520..4f5be41a89d 100644 --- a/src/Processors/Transforms/MongoDBSource.cpp +++ b/src/Processors/Transforms/MongoDBSource.cpp @@ -36,6 +36,7 @@ namespace ErrorCodes extern const int MONGODB_CANNOT_AUTHENTICATE; extern const int NOT_FOUND_COLUMN_IN_BLOCK; extern const int UNKNOWN_TYPE; + extern const int MONGODB_ERROR; } @@ -327,6 +328,14 @@ Chunk MongoDBSource::generate() for (auto & document : response.documents()) { + if (document->exists("ok") && document->exists("$err") + && document->exists("code") && document->getInteger("ok") == 0) + { + auto code = document->getInteger("code"); + const Poco::MongoDB::Element::Ptr value = document->get("$err"); + auto message = static_cast &>(*value).value(); + throw Exception(ErrorCodes::MONGODB_ERROR, "Got error from MongoDB: {}, code: {}", message, code); + } ++num_rows; for (const auto idx : collections::range(0, size)) diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 8389c432db2..ade89ea7228 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -86,7 +86,7 @@ std::optional getExternalDataSourceConfiguration(const configuration.username = config.getString(collection_prefix + ".user", ""); configuration.password = config.getString(collection_prefix + ".password", ""); configuration.database = config.getString(collection_prefix + ".database", ""); - configuration.table = config.getString(collection_prefix + ".table", ""); + configuration.table = config.getString(collection_prefix + ".table", config.getString(collection_prefix + ".collection", "")); configuration.schema = config.getString(collection_prefix + ".schema", ""); configuration.addresses_expr = config.getString(collection_prefix + ".addresses_expr", ""); diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index b214caa9a12..502f8b800e3 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -40,7 +40,6 @@ struct StorageMySQLConfiguration : ExternalDataSourceConfiguration struct StorageMongoDBConfiguration : ExternalDataSourceConfiguration { - String collection; String options; }; diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 9b48f3fc3b3..fe0f9b8d4b4 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -67,9 +67,12 @@ void StorageMongoDB::connectIfNotConnected() if (!authenticated) { # if POCO_VERSION >= 0x01070800 - Poco::MongoDB::Database poco_db(database_name); - if (!poco_db.authenticate(*connection, username, password, Poco::MongoDB::Database::AUTH_SCRAM_SHA1)) - throw Exception("Cannot authenticate in MongoDB, incorrect user or password", ErrorCodes::MONGODB_CANNOT_AUTHENTICATE); + if (!username.empty() && !password.empty()) + { + Poco::MongoDB::Database poco_db(database_name); + if (!poco_db.authenticate(*connection, username, password, Poco::MongoDB::Database::AUTH_SCRAM_SHA1)) + throw Exception("Cannot authenticate in MongoDB, incorrect user or password", ErrorCodes::MONGODB_CANNOT_AUTHENTICATE); + } # else authenticate(*connection, database_name, username, password); # endif @@ -112,9 +115,7 @@ StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, C for (const auto & [arg_name, arg_value] : storage_specific_args) { - if (arg_name == "collection") - configuration.collection = arg_value->as()->value.safeGet(); - else if (arg_name == "options") + if (arg_name == "options") configuration.options = arg_value->as()->value.safeGet(); else throw Exception(ErrorCodes::BAD_ARGUMENTS, @@ -139,7 +140,7 @@ StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, C configuration.host = parsed_host_port.first; configuration.port = parsed_host_port.second; configuration.database = engine_args[1]->as().value.safeGet(); - configuration.collection = engine_args[2]->as().value.safeGet(); + configuration.table = engine_args[2]->as().value.safeGet(); configuration.username = engine_args[3]->as().value.safeGet(); configuration.password = engine_args[4]->as().value.safeGet(); @@ -163,7 +164,7 @@ void registerStorageMongoDB(StorageFactory & factory) configuration.host, configuration.port, configuration.database, - configuration.collection, + configuration.table, configuration.username, configuration.password, configuration.options, diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 784edfc1917..a75e0a2b4ea 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -573,7 +573,7 @@ class TestCase: database = args.testcase_database # This is for .sh tests - os.environ["CLICKHOUSE_LOG_COMMENT"] = self.case_file + os.environ["CLICKHOUSE_LOG_COMMENT"] = args.testcase_basename params = { 'client': client + ' --database=' + database, diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0817cc882b4..66bc8a0ab09 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -725,6 +725,8 @@ class ClickHouseCluster: env_variables['MONGO_HOST'] = self.mongo_host env_variables['MONGO_EXTERNAL_PORT'] = str(self.mongo_port) env_variables['MONGO_INTERNAL_PORT'] = "27017" + env_variables['MONGO_EXTERNAL_PORT_2'] = "27018" + env_variables['MONGO_INTERNAL_PORT_2'] = "27017" self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')]) self.base_mongo_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')] @@ -2107,7 +2109,7 @@ class ClickHouseInstance: except Exception as e: logging.warning(f"Current start attempt failed. Will kill {pid} just in case.") self.exec_in_container(["bash", "-c", f"kill -9 {pid}"], user='root', nothrow=True) - time.sleep(time_to_sleep) + time.sleep(time_to_sleep) raise Exception("Cannot start ClickHouse, see additional info in logs") diff --git a/tests/integration/test_storage_mongodb/configs/named_collections.xml b/tests/integration/test_storage_mongodb/configs/named_collections.xml index feb6b55af02..5f7db390982 100644 --- a/tests/integration/test_storage_mongodb/configs/named_collections.xml +++ b/tests/integration/test_storage_mongodb/configs/named_collections.xml @@ -6,7 +6,7 @@ mongo1 27017 test - simple_table
+ simple_table diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index 1a5de353d7d..2d27ec18018 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -20,8 +20,12 @@ def started_cluster(request): cluster.shutdown() -def get_mongo_connection(started_cluster, secure=False): - connection_str = 'mongodb://root:clickhouse@localhost:{}'.format(started_cluster.mongo_port) +def get_mongo_connection(started_cluster, secure=False, with_credentials=True): + connection_str = '' + if with_credentials: + connection_str = 'mongodb://root:clickhouse@localhost:{}'.format(started_cluster.mongo_port) + else: + connection_str = 'mongodb://localhost:27018' if secure: connection_str += '/?tls=true&tlsAllowInvalidCertificates=true' return pymongo.MongoClient(connection_str) @@ -138,4 +142,20 @@ def test_predefined_connection_configuration(started_cluster): node = started_cluster.instances['node'] node.query("create table simple_mongo_table(key UInt64, data String) engine = MongoDB(mongo1)") + assert node.query("SELECT count() FROM simple_mongo_table") == '100\n' + simple_mongo_table.drop() + +@pytest.mark.parametrize('started_cluster', [False], indirect=['started_cluster']) +def test_no_credentials(started_cluster): + mongo_connection = get_mongo_connection(started_cluster, with_credentials=False) + db = mongo_connection['test'] + simple_mongo_table = db['simple_table'] + data = [] + for i in range(0, 100): + data.append({'key': i, 'data': hex(i * i)}) + simple_mongo_table.insert_many(data) + + node = started_cluster.instances['node'] + node.query("create table simple_mongo_table_2(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', '', '')") + assert node.query("SELECT count() FROM simple_mongo_table_2") == '100\n' simple_mongo_table.drop() diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index 07562557369..4811d0a02ad 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -174,6 +174,8 @@ select groupArray(number) over () from numbers(3); -- Seen errors like 'column `1` not found' from count(1). select count(1) over (rows unbounded preceding), max(number + 1) over () from numbers(3); 1 3 +2 3 +3 3 -- Should work in DISTINCT select distinct sum(0) over (rows unbounded preceding) from numbers(2); 0 diff --git a/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.reference b/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.reference index 67f2590a0c6..a7903610a42 100644 --- a/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.reference +++ b/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.reference @@ -6,3 +6,7 @@ 42 42 42 +SELECT + x, + concat(x, \'_\') +FROM test diff --git a/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql b/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql index ad3d417bc26..d2041a612a6 100644 --- a/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql +++ b/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql @@ -11,4 +11,7 @@ select if(toUInt8(1), 42, y) from test; select if(toInt8(1), 42, y) from test; select if(toUInt8(toUInt8(0)), y, 42) from test; select if(cast(cast(0, 'UInt8'), 'UInt8'), y, 42) from test; + +explain syntax select x, if((select hasColumnInTable(currentDatabase(), 'test', 'y')), y, x || '_') from test; + drop table if exists t; diff --git a/tests/queries/0_stateless/02129_window_functions_disable_optimizations.reference b/tests/queries/0_stateless/02129_window_functions_disable_optimizations.reference new file mode 100644 index 00000000000..f66c81021c9 --- /dev/null +++ b/tests/queries/0_stateless/02129_window_functions_disable_optimizations.reference @@ -0,0 +1,36 @@ +1 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 0.5 30 15 +1 1 30 45 +2 0.5 30 60 +3 1 30 90 +4 0.5 30 105 +5 1 30 135 +6 0.5 30 150 +7 1 30 180 +8 0.5 30 195 +9 1 30 225 +0 0 0 +1 1 1 +2 0 1 +3 0 1 +4 0 1 +5 0 1 +6 0 1 +7 0 1 +8 0 1 +9 0 1 +5772761.230862 +5773916.014064 +5775070.797267 +5776226.273617 +5777381.749967 +5778537.226317 diff --git a/tests/queries/0_stateless/02129_window_functions_disable_optimizations.sql b/tests/queries/0_stateless/02129_window_functions_disable_optimizations.sql new file mode 100644 index 00000000000..cfe9f20d378 --- /dev/null +++ b/tests/queries/0_stateless/02129_window_functions_disable_optimizations.sql @@ -0,0 +1,27 @@ +SET optimize_rewrite_sum_if_to_count_if = 1; + +SELECT if(number % 10 = 0, 1, 0) AS dummy, +sum(dummy) OVER w +FROM numbers(10) +WINDOW w AS (ORDER BY number ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW); + +SET optimize_arithmetic_operations_in_aggregate_functions=1; +SELECT + *, + if((number % 2) = 0, 0.5, 1) AS a, + 30 AS b, + sum(a * b) OVER (ORDER BY number ASC) AS s +FROM numbers(10); + +SET optimize_aggregators_of_group_by_keys=1; + +SELECT + *, + if(number = 1, 1, 0) as a, + max(a) OVER (ORDER BY number ASC) AS s +FROM numbers(10); + +SET optimize_group_by_function_keys = 1; +SELECT round(sum(log(2) * number), 6) AS k FROM numbers(10000) +GROUP BY (number % 2) * (number % 3), number % 3, number % 2 +HAVING sum(log(2) * number) > 346.57353 ORDER BY k; diff --git a/tests/queries/0_stateless/02133_issue_32458.reference b/tests/queries/0_stateless/02133_issue_32458.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02133_issue_32458.sql b/tests/queries/0_stateless/02133_issue_32458.sql new file mode 100644 index 00000000000..16af361db7a --- /dev/null +++ b/tests/queries/0_stateless/02133_issue_32458.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (`id` Int32, `key` String) ENGINE = Memory; +CREATE TABLE t2 (`id` Int32, `key` String) ENGINE = Memory; + +INSERT INTO t1 VALUES (0, ''); +INSERT INTO t2 VALUES (0, ''); + +SELECT * FROM t1 ANY INNER JOIN t2 ON ((NULL = t1.key) = t2.id) AND (('' = t1.key) = t2.id); + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/02140_clickhouse_local_queries_file_table.reference b/tests/queries/0_stateless/02140_clickhouse_local_queries_file_table.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02140_clickhouse_local_queries_file_table.sh b/tests/queries/0_stateless/02140_clickhouse_local_queries_file_table.sh new file mode 100755 index 00000000000..377cbb13688 --- /dev/null +++ b/tests/queries/0_stateless/02140_clickhouse_local_queries_file_table.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --queries-file <(echo 'select 1') --queries-file <(echo 'select 2') --format Null diff --git a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference new file mode 100644 index 00000000000..e4c93e9e1c5 --- /dev/null +++ b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference @@ -0,0 +1 @@ +CREATE TABLE _local.table\n(\n `key` String\n)\nENGINE = File(\'TSVWithNamesAndTypes\', \'/dev/null\') diff --git a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh new file mode 100755 index 00000000000..fc71f779fa1 --- /dev/null +++ b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create table table'