From caeb31940ce9fce6d98ab9629881a099a349703c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 Jan 2024 20:20:58 +0000 Subject: [PATCH 01/21] Allow to build without libssl --- contrib/libssh-cmake/CMakeLists.txt | 2 +- src/Access/Authentication.cpp | 4 ++-- src/Access/AuthenticationData.cpp | 4 ++-- src/Access/UsersConfigAccessStorage.cpp | 2 +- src/Client/Connection.cpp | 4 ++-- src/Client/ConnectionParameters.cpp | 2 +- src/Common/SSH/Wrappers.cpp | 2 +- src/Common/SSH/Wrappers.h | 2 +- src/Common/config.h.in | 1 + src/configure_config.cmake | 3 +++ 10 files changed, 15 insertions(+), 11 deletions(-) diff --git a/contrib/libssh-cmake/CMakeLists.txt b/contrib/libssh-cmake/CMakeLists.txt index 7a3816d4dce..eee3df832fa 100644 --- a/contrib/libssh-cmake/CMakeLists.txt +++ b/contrib/libssh-cmake/CMakeLists.txt @@ -1,4 +1,4 @@ -option (ENABLE_SSH "Enable support for SSH keys and protocol" ON) +option (ENABLE_SSH "Enable support for SSH keys and protocol" ${ENABLE_LIBRARIES}) if (NOT ENABLE_SSH) message(STATUS "Not using SSH") diff --git a/src/Access/Authentication.cpp b/src/Access/Authentication.cpp index 372334bb152..47187d83154 100644 --- a/src/Access/Authentication.cpp +++ b/src/Access/Authentication.cpp @@ -73,7 +73,7 @@ namespace return checkPasswordDoubleSHA1MySQL(scramble, scrambled_password, Util::encodeDoubleSHA1(password_plaintext)); } -#if USE_SSL +#if USE_SSH bool checkSshSignature(const std::vector & keys, std::string_view signature, std::string_view original) { for (const auto & key: keys) @@ -243,7 +243,7 @@ bool Authentication::areCredentialsValid( throw Authentication::Require("ClickHouse X.509 Authentication"); case AuthenticationType::SSH_KEY: -#if USE_SSL +#if USE_SSH return checkSshSignature(auth_data.getSSHKeys(), ssh_credentials->getSignature(), ssh_credentials->getOriginal()); #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL"); diff --git a/src/Access/AuthenticationData.cpp b/src/Access/AuthenticationData.cpp index 1036e1e027e..da90a0f5842 100644 --- a/src/Access/AuthenticationData.cpp +++ b/src/Access/AuthenticationData.cpp @@ -320,7 +320,7 @@ std::shared_ptr AuthenticationData::toAST() const } case AuthenticationType::SSH_KEY: { -#if USE_SSL +#if USE_SSH for (const auto & key : getSSHKeys()) node->children.push_back(std::make_shared(key.getBase64(), key.getKeyType())); @@ -353,7 +353,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que /// For this type of authentication we have ASTPublicSSHKey as children for ASTAuthenticationData if (query.type && query.type == AuthenticationType::SSH_KEY) { -#if USE_SSL +#if USE_SSH AuthenticationData auth_data(*query.type); std::vector keys; diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 2b0fb3f9b2e..e9b2e1397ab 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -209,7 +209,7 @@ namespace } else if (has_ssh_keys) { -#if USE_SSL +#if USE_SSH user->auth_data = AuthenticationData{AuthenticationType::SSH_KEY}; Poco::Util::AbstractConfiguration::Keys entries; diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 352d2a53892..4848f4c7ee5 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -316,7 +316,7 @@ void Connection::sendHello() "Inter-server secret support is disabled, because ClickHouse was built without SSL library"); #endif } -#if USE_SSL +#if USE_SSH /// Just inform server that we will authenticate using SSH keys. else if (!ssh_private_key.isEmpty()) { @@ -346,7 +346,7 @@ void Connection::sendAddendum() void Connection::performHandshakeForSSHAuth() { -#if USE_SSL +#if USE_SSH String challenge; { writeVarUInt(Protocol::Client::SSHChallengeRequest, *out); diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index cadabc98447..a0ae47f0620 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -79,7 +79,7 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati } else { -#if USE_SSL +#if USE_SSH std::string filename = config.getString("ssh-key-file"); std::string passphrase; if (config.has("ssh-key-passphrase")) diff --git a/src/Common/SSH/Wrappers.cpp b/src/Common/SSH/Wrappers.cpp index b38dd1f4036..463338dbe3f 100644 --- a/src/Common/SSH/Wrappers.cpp +++ b/src/Common/SSH/Wrappers.cpp @@ -1,5 +1,5 @@ #include -# if USE_SSL +# if USE_SSH # include # pragma GCC diagnostic push diff --git a/src/Common/SSH/Wrappers.h b/src/Common/SSH/Wrappers.h index 9036fac3163..699bba2b042 100644 --- a/src/Common/SSH/Wrappers.h +++ b/src/Common/SSH/Wrappers.h @@ -1,7 +1,7 @@ #pragma once #include #include "config.h" -#if USE_SSL +#if USE_SSH # include # include diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 5b3388a3b7d..8e513dca145 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -5,6 +5,7 @@ #cmakedefine01 USE_CPUID #cmakedefine01 USE_BASE64 #cmakedefine01 USE_SSL +#cmakedefine01 USE_SSH #cmakedefine01 USE_HDFS #cmakedefine01 USE_AWS_S3 #cmakedefine01 USE_AZURE_BLOB_STORAGE diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 7de2d5a9fdd..141e51badbb 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -164,6 +164,9 @@ endif () if (ENABLE_OPENSSL) set(USE_OPENSSL_INTREE 1) endif () +if (TARGET ch_contrib::ssh) + set(USE_SSH 1) +endif() if (TARGET ch_contrib::fiu) set(FIU_ENABLE 1) endif() From 94eb33ef27a9ab7c4a99af40772ea287e67efcbf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 26 Jan 2024 14:29:50 +0100 Subject: [PATCH 02/21] Improve pytest --pdb experience by preserving dockerd on SIGINT (v2) Previous fix (#43392) was not correct, since terminal sends SIGINT to all processes in a group, so simply ignoring it in parent won't work. So instead dockerd will be run in a new session. Signed-off-by: Azat Khuzhin --- docker/test/integration/runner/dockerd-entrypoint.sh | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index b05aef76faf..8882daa38ea 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -23,13 +23,15 @@ if [ -f /sys/fs/cgroup/cgroup.controllers ]; then > /sys/fs/cgroup/cgroup.subtree_control fi -# In case of test hung it is convenient to use pytest --pdb to debug it, -# and on hung you can simply press Ctrl-C and it will spawn a python pdb, -# but on SIGINT dockerd will exit, so ignore it to preserve the daemon. -trap '' INT # Binding to an IP address without --tlsverify is deprecated. Startup is intentionally being slowed # unless --tls=false or --tlsverify=false is set -dockerd --host=unix:///var/run/docker.sock --tls=false --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log & +# +# In case of test hung it is convenient to use pytest --pdb to debug it, +# and on hung you can simply press Ctrl-C and it will spawn a python pdb, +# but on SIGINT dockerd will exit, so we spawn new session to ignore SIGINT by +# docker. +# Note, that if you will run it via runner, it will send SIGINT to docker anyway. +setsid dockerd --host=unix:///var/run/docker.sock --tls=false --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log & set +e reties=0 From 5227d7c9bce3d1b9cac2b8d3b8ef0c7b61a2a516 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 26 Jan 2024 11:31:44 +0100 Subject: [PATCH 03/21] Fix 02475_bson_each_row_format flakiness (due to small parsing block) CI: https://s3.amazonaws.com/clickhouse-test-reports/59170/c82050d1925439f0ede2b32acb5b1b8df4acae5d/stateless_tests__release_/run.log Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02475_bson_each_row_format.sh | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/02475_bson_each_row_format.sh b/tests/queries/0_stateless/02475_bson_each_row_format.sh index aa58d27fa50..f5c48608639 100755 --- a/tests/queries/0_stateless/02475_bson_each_row_format.sh +++ b/tests/queries/0_stateless/02475_bson_each_row_format.sh @@ -5,6 +5,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# In case of parallel parsing and small block +# (--min_chunk_bytes_for_parallel_parsing) we may have multiple blocks, and +# this will break sorting order, so let's limit number of threads to avoid +# reordering. +CLICKHOUSE_CLIENT+="--allow_repeated_settings --max_threads 1" + echo "Integers" $CLICKHOUSE_CLIENT -q "insert into function file(02475_data.bsonEachRow) select number::Bool as bool, number::Int8 as int8, number::UInt8 as uint8, number::Int16 as int16, number::UInt16 as uint16, number::Int32 as int32, number::UInt32 as uint32, number::Int64 as int64, number::UInt64 as uint64 from numbers(5) settings engine_file_truncate_on_insert=1" $CLICKHOUSE_CLIENT -q "select * from file(02475_data.bsonEachRow, auto, 'bool Bool, int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64')" From 6795d0207f98657e499c66871678c23412ac04ba Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Jan 2024 16:37:54 +0000 Subject: [PATCH 04/21] Update test_reload_after_fail_in_cache_dictionary for analyzer --- tests/analyzer_integration_broken_tests.txt | 1 - tests/integration/test_dictionaries_update_and_reload/test.py | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 23f22209451..e1d4de59a23 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -1,7 +1,6 @@ test_access_for_functions/test.py::test_access_rights_for_function test_build_sets_from_multiple_threads/test.py::test_set test_concurrent_backups_s3/test.py::test_concurrent_backups -test_dictionaries_update_and_reload/test.py::test_reload_after_fail_in_cache_dictionary test_distributed_backward_compatability/test.py::test_distributed_in_tuple test_distributed_type_object/test.py::test_distributed_type_object test_executable_table_function/test.py::test_executable_function_input_python diff --git a/tests/integration/test_dictionaries_update_and_reload/test.py b/tests/integration/test_dictionaries_update_and_reload/test.py index 3d96d0b8dd4..648ea847afb 100644 --- a/tests/integration/test_dictionaries_update_and_reload/test.py +++ b/tests/integration/test_dictionaries_update_and_reload/test.py @@ -281,7 +281,7 @@ def test_reload_after_fail_in_cache_dictionary(started_cluster): query_and_get_error = instance.query_and_get_error # Can't get a value from the cache dictionary because the source (table `test.xypairs`) doesn't respond. - expected_error = "Table test.xypairs does not exist" + expected_error = "UNKNOWN_TABLE" update_error = "Could not update cache dictionary cache_xypairs now" assert expected_error in query_and_get_error( "SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(1))" From 4273ae4f4ff401691acf3712d862336e1bf6cfb6 Mon Sep 17 00:00:00 2001 From: Peignon Melvyn Date: Fri, 26 Jan 2024 21:27:20 +0100 Subject: [PATCH 05/21] Update view.md --- docs/en/sql-reference/statements/alter/view.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/view.md b/docs/en/sql-reference/statements/alter/view.md index 517e64e3e5b..59045afdeb6 100644 --- a/docs/en/sql-reference/statements/alter/view.md +++ b/docs/en/sql-reference/statements/alter/view.md @@ -8,8 +8,6 @@ sidebar_label: VIEW You can modify `SELECT` query that was specified when a [materialized view](../create/view.md#materialized) was created with the `ALTER TABLE … MODIFY QUERY` statement without interrupting ingestion process. -The `allow_experimental_alter_materialized_view_structure` setting must be enabled. - This command is created to change materialized view created with `TO [db.]name` clause. It does not change the structure of the underling storage table and it does not change the columns' definition of the materialized view, because of this the application of this command is very limited for materialized views are created without `TO [db.]name` clause. **Example with TO table** From 44e42052b184834466970a7d95a3943eba572dec Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 27 Jan 2024 13:12:21 +0100 Subject: [PATCH 06/21] Fix perf tests after sumMap starts to filter out -0. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Before perf tests was relying on the following: SELECT sumMap(['foo', 'bar'], [-0., -0.]) ┌─sumMap(['foo', 'bar'], [-0., -0.])─┐ │ (['bar','foo'],[-0,-0]) │ └────────────────────────────────────┘ While it got changed, and now: ┌─sumMap(['foo', 'bar'], [-0., -0.])─┐ │ ([],[]) │ └────────────────────────────────────┘ But it works for nan: SELECT sumMap(['foo', 'bar'], [nan, nan]) ┌─sumMap(['foo', 'bar'], [nan, nan])─┐ │ (['bar','foo'],[nan,nan]) │ └────────────────────────────────────┘ Signed-off-by: Azat Khuzhin --- tests/performance/scripts/compare.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/performance/scripts/compare.sh b/tests/performance/scripts/compare.sh index 7dc522dca7a..92ba383f965 100755 --- a/tests/performance/scripts/compare.sh +++ b/tests/performance/scripts/compare.sh @@ -444,10 +444,10 @@ create view query_logs as create table query_run_metric_arrays engine File(TSV, 'analyze/query-run-metric-arrays.tsv') as with ( - -- sumMapState with the list of all keys with '-0.' values. Negative zero is because - -- sumMap removes keys with positive zeros. + -- sumMapState with the list of all keys with 'nan' values. 'nan' is because + -- sumMap removes keys with positive/negative zeros. with (select groupUniqArrayArray(mapKeys(ProfileEvents)) from query_logs) as all_names - select arrayReduce('sumMapState', [(all_names, arrayMap(x->-0., all_names))]) + select arrayReduce('sumMapState', [(all_names, arrayMap(x->nan, all_names))]) ) as all_metrics select test, query_index, version, query_id, (finalizeAggregation( @@ -460,13 +460,13 @@ create table query_run_metric_arrays engine File(TSV, 'analyze/query-run-metric- ), arrayReduce('sumMapState', [( ['client_time', 'server_time', 'memory_usage'], - arrayMap(x->if(x != 0., x, -0.), [ + arrayMap(x->if(x != 0., x, nan), [ toFloat64(query_runs.time), toFloat64(query_duration_ms / 1000.), toFloat64(memory_usage)]))]) ] )) as metrics_tuple).1 metric_names, - metrics_tuple.2 metric_values + arrayMap(x->if(isNaN(x),0,x), metrics_tuple.2) metric_values from query_logs right join query_runs on query_logs.query_id = query_runs.query_id From d94547d338897ef6a2bb53a65946b53fab7358d0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 27 Jan 2024 16:00:49 +0100 Subject: [PATCH 07/21] Use fresh ZooKeeper client on DROP (to have higher chances on success) In case of DROP the client can be expired, and even though StorageReplicatedMergeTree::drop() uses getZooKeeperIfTableShutDown(), which creates new client if current is expired, it is not enough, since current_zookeeper (cached client) will be used implicitly from dropAllData(). This had been found by stress tests, that leads to DROP query hang [1]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/59255/94eb33ef27a9ab7c4a99af40772ea287e67efcbf/stress_test__tsan_.html
stacktrace 2024.01.26 17:41:41.138577 [ 3319 ] {} DatabaseCatalog: Cannot drop table test_vzfk1xoc.alter_table1 (455a674c-161d-44d8-abc2-fd205bad1116). Will retry later.: Code: 999. Coordination::Exception: Session expired. (KEEPER_EXCEPTION), Stack trace (when copying this message, always include the lines below): 0. ./contrib/llvm-project/libcxx/include/exception:134: Poco::Exception::Exception(String const&, int) @ 0x000000001e06a5e3 in /usr/bin/clickhouse 1. ./build_docker/./src/Common/Exception.cpp:96: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000f697c74 in /usr/bin/clickhouse 2. ./src/Common/Exception.h:0: Coordination::Exception::Exception(char const (&) [16], Coordination::Error) @ 0x000000001adc298b in /usr/bin/clickhouse 3. ./src/Common/ZooKeeper/IKeeper.h:0: Coordination::ZooKeeper::pushRequest(Coordination::ZooKeeper::RequestInfo&&) @ 0x000000001adfb418 in /usr/bin/clickhouse 4. ./build_docker/./src/Common/ZooKeeper/ZooKeeperImpl.cpp:1343: Coordination::ZooKeeper::get(String const&, std::function, std::shared_ptr>) @ 0x000000001adfd8e5 in /usr/bin/clickhouse 5. ./contrib/llvm-project/libcxx/include/__functional/function.h:818: ? @ 0x000000001ad890bd in /usr/bin/clickhouse 6. ./contrib/llvm-project/libcxx/include/__functional/function.h:818: ? @ 0x000000001ad88a36 in /usr/bin/clickhouse 7. ./build_docker/./src/Common/ZooKeeper/ZooKeeper.cpp:580: zkutil::ZooKeeper::tryGetWatch(String const&, String&, Coordination::Stat*, std::function, Coordination::Error*) @ 0x000000001ad898bf in /usr/bin/clickhouse 8. ./build_docker/./src/Common/ZooKeeper/ZooKeeper.cpp:570: zkutil::ZooKeeper::tryGet(String const&, String&, Coordination::Stat*, std::shared_ptr const&, Coordination::Error*) @ 0x000000001ad89554 in /usr/bin/clickhouse 9. ./build_docker/./src/Common/ZooKeeper/ZooKeeperWithFaultInjection.cpp:0: DB::ZooKeeperWithFaultInjection::tryGet(String const&, String&, Coordination::Stat*, std::shared_ptr const&, Coordination::Error*) @ 0x000000001ae110dc in /usr/bin/clickhouse 10. ./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:701: DB::StorageReplicatedMergeTree::unlockSharedDataByID(String, String const&, DB::MergeTreePartInfo const&, String const&, String const&, std::shared_ptr const&, DB::MergeTreeSettings const&, std::shared_ptr, String const&, StrongTypedef) @ 0x000000001975527e in /usr/bin/clickhouse 11. ./build_docker/./src/Storages/StorageReplicatedMergeTree.cpp:0: DB::StorageReplicatedMergeTree::removeSharedDetachedPart(std::shared_ptr, String const&, String const&, String const&, String const&, String const&, std::shared_ptr const&, std::shared_ptr const&) @ 0x000000001976c980 in /usr/bin/clickhouse 12. ./build_docker/./src/Storages/StorageReplicatedMergeTree.cpp:10171: DB::StorageReplicatedMergeTree::removeDetachedPart(std::shared_ptr, String const&, String const&) @ 0x000000001976bdc6 in /usr/bin/clickhouse 13. ./build_docker/./src/Storages/MergeTree/MergeTreeData.cpp:2798: DB::MergeTreeData::dropAllData() @ 0x0000000019c643c8 in /usr/bin/clickhouse 14. ./build_docker/./src/Storages/StorageReplicatedMergeTree.cpp:1117: DB::StorageReplicatedMergeTree::drop() @ 0x0000000019664e0a in /usr/bin/clickhouse 15. ./build_docker/./src/Interpreters/DatabaseCatalog.cpp:0: DB::DatabaseCatalog::dropTableFinally(DB::DatabaseCatalog::TableMarkedAsDropped const&) @ 0x0000000017bf1fac in /usr/bin/clickhouse 16. ./build_docker/./src/Interpreters/DatabaseCatalog.cpp:0: DB::DatabaseCatalog::dropTableDataTask() @ 0x0000000017bf175e in /usr/bin/clickhouse 17. ./contrib/llvm-project/libcxx/include/__functional/function.h:717: ? @ 0x0000000017bf94c2 in /usr/bin/clickhouse 18. ./contrib/llvm-project/libcxx/include/__functional/function.h:0: ? @ 0x00000000170bd0b8 in /usr/bin/clickhouse 19. ./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:701: DB::BackgroundSchedulePool::threadFunction() @ 0x00000000170c06ce in /usr/bin/clickhouse 20. ./build_docker/./src/Core/BackgroundSchedulePool.cpp:0: void std::__function::__policy_invoker::__call_impl::ThreadFromGlobalPoolImpl, StrongTypedef, char const*)::$_0>(DB::BackgroundSchedulePool::BackgroundSchedulePool(unsigned long, StrongTypedef, StrongTypedef, char const*)::$_0&&)::'lambda'(), void ()>>(std::__function::__policy_storage const*) @ 0x00000000170c0feb in /usr/bin/clickhouse 21. ./base/base/../base/wide_integer_impl.h:809: ThreadPoolImpl::worker(std::__list_iterator) @ 0x000000000f7c6a2f in /usr/bin/clickhouse 22. ./contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:302: void* std::__thread_proxy[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>>(void*) @ 0x000000000f7cc8f2 in /usr/bin/clickhouse 23. ? @ 0x000000000733904f in /usr/bin/clickhouse 24. ? @ 0x00007f28b2319ac3 25. ? @ 0x00007f28b23ab850
Signed-off-by: Azat Khuzhin --- src/Storages/StorageReplicatedMergeTree.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c82721d2a18..695b78a10db 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1096,6 +1096,8 @@ void StorageReplicatedMergeTree::drop() /// Table can be shut down, restarting thread is not active /// and calling StorageReplicatedMergeTree::getZooKeeper()/getAuxiliaryZooKeeper() won't suffice. zookeeper = getZooKeeperIfTableShutDown(); + /// Update zookeeper client, since existing may be expired, while ZooKeeper is required inside dropAllData(). + current_zookeeper = zookeeper; /// If probably there is metadata in ZooKeeper, we don't allow to drop the table. if (!zookeeper) From cf1df96cc1c19f9fec621ba1a407148474edba01 Mon Sep 17 00:00:00 2001 From: Peignon Melvyn Date: Sat, 27 Jan 2024 17:27:00 +0100 Subject: [PATCH 08/21] Update view.md Improved documentation. --- docs/en/sql-reference/statements/create/view.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index f6158acd9a4..028d0b09a1a 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -97,7 +97,7 @@ This feature is deprecated and will be removed in the future. For your convenience, the old documentation is located [here](https://pastila.nl/?00f32652/fdf07272a7b54bda7e13b919264e449f.md) -## Refreshable Materialized View {#refreshable-materialized-view} +## Refreshable Materialized View [Experimental] {#refreshable-materialized-view} ```sql CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name @@ -120,7 +120,8 @@ Differences from regular non-refreshable materialized views: :::note Refreshable materialized views are a work in progress. Setting `allow_experimental_refreshable_materialized_view = 1` is required for creating one. Current limitations: - * not compatible with Replicated database or table engines, + * not compatible with Replicated database or table engines + * It is not supported in ClickHouse Cloud * require [Atomic database engine](../../../engines/database-engines/atomic.md), * no retries for failed refresh - we just skip to the next scheduled refresh time, * no limit on number of concurrent refreshes. From 9d3b4e530810b2d05f62706a22062519ba18b30b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 27 Jan 2024 16:40:11 +0000 Subject: [PATCH 09/21] lower memory usage --- src/Columns/ColumnAggregateFunction.cpp | 4 ++-- src/Columns/ColumnArray.cpp | 12 +++++++++--- src/Columns/ColumnArray.h | 1 + src/Columns/ColumnConst.cpp | 2 +- src/Columns/ColumnDecimal.cpp | 4 ++-- src/Columns/ColumnDecimal.h | 1 + src/Columns/ColumnFixedString.cpp | 2 +- src/Columns/ColumnFixedString.h | 5 +++++ src/Columns/ColumnLowCardinality.h | 2 ++ src/Columns/ColumnMap.cpp | 5 +++++ src/Columns/ColumnMap.h | 1 + src/Columns/ColumnNullable.cpp | 8 +++++++- src/Columns/ColumnNullable.h | 1 + src/Columns/ColumnString.cpp | 7 ++++++- src/Columns/ColumnString.h | 1 + src/Columns/ColumnTuple.cpp | 7 +++++++ src/Columns/ColumnTuple.h | 1 + src/Columns/ColumnVector.cpp | 8 ++++---- src/Columns/ColumnVector.h | 5 +++++ src/Columns/IColumn.h | 5 +---- src/Common/PODArray.h | 7 +++++++ src/Core/Block.cpp | 2 +- src/DataTypes/Serializations/SerializationString.cpp | 2 +- src/Storages/MergeTree/MergeTreeReadTask.cpp | 4 ++++ src/Storages/MergeTree/MergeTreeSequentialSource.cpp | 5 +++++ 25 files changed, 81 insertions(+), 21 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 2018015b46d..43e55494747 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -337,7 +337,7 @@ ColumnPtr ColumnAggregateFunction::indexImpl(const PaddedPODArray & indexe assert(limit <= indexes.size()); auto res = createView(); - res->data.resize(limit); + res->data.resize_exact(limit); for (size_t i = 0; i < limit; ++i) res->data[i] = data[indexes[i]]; @@ -626,7 +626,7 @@ void ColumnAggregateFunction::getPermutation(PermutationSortDirection /*directio size_t /*limit*/, int /*nan_direction_hint*/, IColumn::Permutation & res) const { size_t s = data.size(); - res.resize(s); + res.resize_exact(s); iota(res.data(), s, IColumn::Permutation::value_type(0)); } diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 1cb8188bce6..fddfa2ac6b2 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -109,7 +109,7 @@ MutableColumnPtr ColumnArray::cloneResized(size_t to_size) const offset = getOffsets().back(); } - res->getOffsets().resize(to_size); + res->getOffsets().resize_exact(to_size); for (size_t i = from_size; i < to_size; ++i) res->getOffsets()[i] = offset; } @@ -427,6 +427,12 @@ void ColumnArray::reserve(size_t n) getData().reserve(n); /// The average size of arrays is not taken into account here. Or it is considered to be no more than 1. } +void ColumnArray::shrinkToFit() +{ + getOffsets().shrink_to_fit(); + getData().shrinkToFit(); +} + void ColumnArray::ensureOwnership() { getData().ensureOwnership(); @@ -603,7 +609,7 @@ void ColumnArray::expand(const IColumn::Filter & mask, bool inverted) ssize_t index = mask.size() - 1; ssize_t from = offsets_data.size() - 1; - offsets_data.resize(mask.size()); + offsets_data.resize_exact(mask.size()); UInt64 last_offset = offsets_data[from]; while (index >= 0) { @@ -831,7 +837,7 @@ ColumnPtr ColumnArray::indexImpl(const PaddedPODArray & indexes, size_t limit auto res = ColumnArray::create(data->cloneEmpty()); Offsets & res_offsets = res->getOffsets(); - res_offsets.resize(limit); + res_offsets.resize_exact(limit); size_t current_offset = 0; for (size_t i = 0; i < limit; ++i) diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 2a9bfa405e5..407f44a6f3c 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -108,6 +108,7 @@ public: void updatePermutationWithCollation(const Collator & collator, PermutationSortDirection direction, PermutationSortStability stability, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override; void reserve(size_t n) override; + void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; diff --git a/src/Columns/ColumnConst.cpp b/src/Columns/ColumnConst.cpp index 9aa0f5cfa49..57d02e72820 100644 --- a/src/Columns/ColumnConst.cpp +++ b/src/Columns/ColumnConst.cpp @@ -128,7 +128,7 @@ MutableColumns ColumnConst::scatter(ColumnIndex num_columns, const Selector & se void ColumnConst::getPermutation(PermutationSortDirection /*direction*/, PermutationSortStability /*stability*/, size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const { - res.resize(s); + res.resize_exact(s); iota(res.data(), s, IColumn::Permutation::value_type(0)); } diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 20fc5d8e1fe..aef81727ffd 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -159,7 +159,7 @@ void ColumnDecimal::getPermutation(IColumn::PermutationSortDirection directio }; size_t data_size = data.size(); - res.resize(data_size); + res.resize_exact(data_size); if (limit >= data_size) limit = 0; @@ -318,7 +318,7 @@ MutableColumnPtr ColumnDecimal::cloneResized(size_t size) const if (size > 0) { auto & new_col = static_cast(*res); - new_col.data.resize(size); + new_col.data.resize_exact(size); size_t count = std::min(this->size(), size); diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index fb24ae4554b..840dc23dc36 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -55,6 +55,7 @@ public: size_t allocatedBytes() const override { return data.allocated_bytes(); } void protect() override { data.protect(); } void reserve(size_t n) override { data.reserve(n); } + void shrinkToFit() override { data.shrink_to_fit(); } void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast(src).getData()[n]); } void insertData(const char * src, size_t /*length*/) override; diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index a18e5c522a1..1ba59ce4f4b 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -39,7 +39,7 @@ MutableColumnPtr ColumnFixedString::cloneResized(size_t size) const if (size > 0) { auto & new_col = assert_cast(*new_col_holder); - new_col.chars.resize(size * n); + new_col.chars.resize_exact(size * n); size_t count = std::min(this->size(), size); memcpy(new_col.chars.data(), chars.data(), count * n * sizeof(chars[0])); diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 445432b7b28..70c26ba8496 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -181,6 +181,11 @@ public: chars.reserve(n * size); } + void shrinkToFit() override + { + chars.shrink_to_fit(); + } + void resize(size_t size) { chars.resize(n * size); diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 91bd5945fd9..c6fcfc0a4b7 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -159,6 +159,7 @@ public: } void reserve(size_t n) override { idx.reserve(n); } + void shrinkToFit() override { idx.shrinkToFit(); } /// Don't count the dictionary size as it can be shared between different blocks. size_t byteSize() const override { return idx.getPositions()->byteSize(); } @@ -295,6 +296,7 @@ public: void popBack(size_t n) { positions->popBack(n); } void reserve(size_t n) { positions->reserve(n); } + void shrinkToFit() { positions->shrinkToFit(); } UInt64 getMaxPositionForCurrentType() const; diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index ddcde43ca23..61abe9d2598 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -229,6 +229,11 @@ void ColumnMap::reserve(size_t n) nested->reserve(n); } +void ColumnMap::shrinkToFit() +{ + nested->shrinkToFit(); +} + void ColumnMap::ensureOwnership() { nested->ensureOwnership(); diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index fde8a7e0e67..118c5b3d3df 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -83,6 +83,7 @@ public: void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override; void reserve(size_t n) override; + void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 4ee6bb3d586..1a2bc378ff1 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -97,7 +97,7 @@ MutableColumnPtr ColumnNullable::cloneResized(size_t new_size) const if (new_size > 0) { - new_null_map->getData().resize(new_size); + new_null_map->getData().resize_exact(new_size); size_t count = std::min(size(), new_size); memcpy(new_null_map->getData().data(), getNullMapData().data(), count * sizeof(getNullMapData()[0])); @@ -678,6 +678,12 @@ void ColumnNullable::reserve(size_t n) getNullMapData().reserve(n); } +void ColumnNullable::shrinkToFit() +{ + getNestedColumn().shrinkToFit(); + getNullMapData().shrink_to_fit(); +} + void ColumnNullable::ensureOwnership() { getNestedColumn().ensureOwnership(); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index b57fdf3064d..988783abfa3 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -109,6 +109,7 @@ public: void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override; void reserve(size_t n) override; + void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 50fe90ad8ef..b1ce50f9a2e 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -71,8 +71,8 @@ MutableColumnPtr ColumnString::cloneResized(size_t to_size) const /// Empty strings are just zero terminating bytes. res->chars.resize_fill(res->chars.size() + to_size - from_size); + res->offsets.resize_exact(to_size); - res->offsets.resize(to_size); for (size_t i = from_size; i < to_size; ++i) { ++offset; @@ -494,6 +494,11 @@ void ColumnString::reserve(size_t n) offsets.reserve(n); } +void ColumnString::shrinkToFit() +{ + chars.shrink_to_fit(); + offsets.shrink_to_fit(); +} void ColumnString::getExtremes(Field & min, Field & max) const { diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 345513248fd..f2844b40aff 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -259,6 +259,7 @@ public: ColumnPtr compress() const override; void reserve(size_t n) override; + void shrinkToFit() override; void getExtremes(Field & min, Field & max) const override; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 356bb0493d2..55899bbd50b 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -444,6 +444,13 @@ void ColumnTuple::reserve(size_t n) getColumn(i).reserve(n); } +void ColumnTuple::shrinkToFit() +{ + const size_t tuple_size = columns.size(); + for (size_t i = 0; i < tuple_size; ++i) + getColumn(i).shrinkToFit(); +} + void ColumnTuple::ensureOwnership() { const size_t tuple_size = columns.size(); diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 79099f4c098..d5d0915de96 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -91,6 +91,7 @@ public: void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges& equal_ranges) const override; void reserve(size_t n) override; + void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index b1cf449dfde..802d4293b90 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -237,7 +237,7 @@ void ColumnVector::getPermutation(IColumn::PermutationSortDirection direction size_t limit, int nan_direction_hint, IColumn::Permutation & res) const { size_t data_size = data.size(); - res.resize(data_size); + res.resize_exact(data_size); if (data_size == 0) return; @@ -424,7 +424,7 @@ MutableColumnPtr ColumnVector::cloneResized(size_t size) const if (size > 0) { auto & new_col = static_cast(*res); - new_col.data.resize(size); + new_col.data.resize_exact(size); size_t count = std::min(this->size(), size); memcpy(new_col.data.data(), data.data(), count * sizeof(data[0])); @@ -628,8 +628,8 @@ inline void doFilterAligned(const UInt8 *& filt_pos, const UInt8 *& filt_end_ali filt_pos += SIMD_ELEMENTS; data_pos += SIMD_ELEMENTS; } - /// resize to the real size. - res_data.resize(current_offset); + /// Resize to the real size. + res_data.resize_exact(current_offset); } ) diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 517375f8eb4..cfc2ad6272e 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -182,6 +182,11 @@ public: data.reserve(n); } + void shrinkToFit() override + { + data.shrink_to_fit(); + } + const char * getFamilyName() const override { return TypeName.data(); } TypeIndex getDataType() const override { return TypeToTypeIndex; } diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 1ccdf255bf4..4f992263605 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -399,10 +399,7 @@ public: /// Requests the removal of unused capacity. /// It is a non-binding request to reduce the capacity of the underlying container to its size. - virtual MutablePtr shrinkToFit() const - { - return cloneResized(size()); - } + virtual void shrinkToFit() {} /// If we have another column as a source (owner of data), copy all data to ourself and reset source. virtual void ensureOwnership() {} diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index 6a048d1c6c0..1a4047a2588 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -238,6 +239,12 @@ public: resize_assume_reserved(n); } + template + void shrink_to_fit(TAllocatorParams &&... allocator_params) + { + realloc(PODArrayDetails::minimum_memory_for_elements(size(), ELEMENT_SIZE, pad_left, pad_right), std::forward(allocator_params)...); + } + void resize_assume_reserved(const size_t n) /// NOLINT { c_end = c_start + PODArrayDetails::byte_size(n, ELEMENT_SIZE); diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index a7d5b0a869f..dfd60b994f4 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -601,7 +601,7 @@ Block Block::shrinkToFit() const { Columns new_columns(data.size(), nullptr); for (size_t i = 0; i < data.size(); ++i) - new_columns[i] = data[i].column->shrinkToFit(); + new_columns[i] = data[i].column->cloneResized(data[i].column->size()); return cloneWithColumns(new_columns); } diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index b2b083fd466..e2935a78c32 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -208,7 +208,7 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt data[offset - 1] = 0; } - data.resize(offset); + data.resize_exact(offset); } diff --git a/src/Storages/MergeTree/MergeTreeReadTask.cpp b/src/Storages/MergeTree/MergeTreeReadTask.cpp index dcfed700fac..41c7531b6a6 100644 --- a/src/Storages/MergeTree/MergeTreeReadTask.cpp +++ b/src/Storages/MergeTree/MergeTreeReadTask.cpp @@ -184,7 +184,11 @@ MergeTreeReadTask::BlockAndProgress MergeTreeReadTask::read(const BlockSizeParam Block block; if (read_result.num_rows != 0) + { + for (const auto & column : read_result.columns) + column->assumeMutableRef().shrinkToFit(); block = sample_block.cloneWithColumns(read_result.columns); + } BlockAndProgress res = { .block = std::move(block), diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 29af7fb4820..d0fbc316024 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -140,6 +140,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( if (storage.supportsSubcolumns()) options.withSubcolumns(); + columns_for_reader = storage_snapshot->getColumnsByNames(options, columns_to_read); } else @@ -156,6 +157,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( read_settings.local_fs_method = LocalFSReadMethod::pread; if (read_with_direct_io) read_settings.direct_io_threshold = 1; + /// Configure throttling switch (type) { @@ -224,7 +226,10 @@ try for (size_t i = 0; i < num_columns; ++i) { if (header.has(it->name)) + { + columns[i]->assumeMutableRef().shrinkToFit(); res_columns.emplace_back(std::move(columns[i])); + } ++it; } From b7a6dbfbb9c31970aaeefd3fc1423c5de9d014b4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 27 Jan 2024 18:22:37 +0100 Subject: [PATCH 10/21] Check engine to avoid possible seg fault --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 9ce1c856622..5b04ffb2b17 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -716,7 +716,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti setEngine(create); /// We have to check access rights again (in case engine was changed). - if (create.storage) + if (create.storage && create.storage->engine) { auto source_access_type = StorageFactory::instance().getSourceAccessType(create.storage->engine->name); if (source_access_type != AccessType::NONE) From 9dd6362ae7488169b4dc8be9be92e3a982131cdb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 13:44:56 +0100 Subject: [PATCH 11/21] Play UI: improve rendering of errors from JSON formats --- programs/server/play.html | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/programs/server/play.html b/programs/server/play.html index e6662693bd7..507a96382a7 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -993,7 +993,16 @@ function renderError(response) { clear(); - document.getElementById('error').innerText = response ? response : "No response."; + + let message = response; + try { + let json = JSON.parse(response); + if (json.exception) { + message = json.exception; + } + } catch (e) {} + + document.getElementById('error').innerText = message ? message : "No response."; document.getElementById('error').style.display = 'block'; document.getElementById('logo-container').style.display = 'none'; } From e0f0b21e907eb0d20655f83641f26261793910cf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 14:24:13 +0100 Subject: [PATCH 12/21] Remove KQL --- src/Interpreters/executeQuery.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 533d58aaa8f..3ff3db6b8ff 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -102,6 +102,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int QUERY_WAS_CANCELLED; extern const int INCORRECT_DATA; + extern const int SUPPORT_IS_DISABLED; } @@ -709,10 +710,7 @@ static std::tuple executeQueryImpl( { if (settings.dialect == Dialect::kusto && !internal) { - ParserKQLStatement parser(end, settings.allow_settings_after_format_in_insert); - - /// TODO: parser should fail early when max_query_size limit is reached. - ast = parseKQLQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Kusto dialect is disabled until these two bugs will be fixed: https://github.com/ClickHouse/ClickHouse/issues/59037 and https://github.com/ClickHouse/ClickHouse/issues/59036"); } else if (settings.dialect == Dialect::prql && !internal) { From 27cd21848360815c5db49647b5d8efc6e55f2b99 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 15:10:57 +0100 Subject: [PATCH 13/21] No debug symbols in Rust --- rust/prql/Cargo.toml | 2 +- rust/skim/Cargo.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/rust/prql/Cargo.toml b/rust/prql/Cargo.toml index f29aa4aaef9..ca8a19b41ac 100644 --- a/rust/prql/Cargo.toml +++ b/rust/prql/Cargo.toml @@ -13,7 +13,7 @@ serde_json = "1.0" crate-type = ["staticlib"] [profile.release] -debug = true +debug = false [profile.release-thinlto] inherits = "release" diff --git a/rust/skim/Cargo.toml b/rust/skim/Cargo.toml index 22af40c3e33..0839b888d8b 100644 --- a/rust/skim/Cargo.toml +++ b/rust/skim/Cargo.toml @@ -17,7 +17,7 @@ cxx-build = "1.0.83" crate-type = ["staticlib"] [profile.release] -debug = true +debug = false [profile.release-thinlto] inherits = "release" From b3b2b7a1e6352379d90d72bc773737f386ec1c36 Mon Sep 17 00:00:00 2001 From: Maksim Alekseev Date: Sun, 28 Jan 2024 17:29:08 +0300 Subject: [PATCH 14/21] Fix architecture name in select of Rust target --- contrib/corrosion-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/corrosion-cmake/CMakeLists.txt b/contrib/corrosion-cmake/CMakeLists.txt index 04871c761ab..9b98ed6efb3 100644 --- a/contrib/corrosion-cmake/CMakeLists.txt +++ b/contrib/corrosion-cmake/CMakeLists.txt @@ -28,7 +28,7 @@ elseif(CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-aarch64") set(Rust_CARGO_TARGET "aarch64-unknown-linux-gnu") elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "x86_64")) set(Rust_CARGO_TARGET "x86_64-apple-darwin") -elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "darwin")) +elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "aarch64")) set(Rust_CARGO_TARGET "aarch64-apple-darwin") elseif((CMAKE_TOOLCHAIN_FILE MATCHES "freebsd") AND (CMAKE_TOOLCHAIN_FILE MATCHES "x86_64")) set(Rust_CARGO_TARGET "x86_64-unknown-freebsd") From d36c92a21900c7f47680ee67cb93864d02105bdf Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Sun, 28 Jan 2024 15:09:39 +0000 Subject: [PATCH 15/21] CI: Add rust dir to build digest --- tests/ci/ci_config.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index ece7f2f7bae..df72c02cb7d 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -213,6 +213,7 @@ class BuildConfig: "./programs", "./packages", "./docker/packager/packager", + "./rust", ], exclude_files=[".md"], docker=["clickhouse/binary-builder"], From aa6c7e78beb843b1c830a947615836a89291519b Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 28 Jan 2024 15:47:17 +0000 Subject: [PATCH 16/21] fix deadlock in AsyncLoader::stop() --- src/Common/AsyncLoader.cpp | 44 ++++++++++++++++++------- src/Common/AsyncLoader.h | 5 --- src/Common/tests/gtest_async_loader.cpp | 12 +++++-- 3 files changed, 43 insertions(+), 18 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index e9de95363bc..140194e10b4 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -195,13 +196,6 @@ void LoadTask::remove() } } -void LoadTask::detach() -{ - jobs.clear(); - goal_jobs.clear(); -} - - AsyncLoader::AsyncLoader(std::vector pool_initializers, bool log_failures_, bool log_progress_) : log_failures(log_failures_) , log_progress(log_progress_) @@ -214,7 +208,22 @@ AsyncLoader::AsyncLoader(std::vector pool_initializers, bool lo AsyncLoader::~AsyncLoader() { - stop(); + // All `LoadTask` objects should be destructed before AsyncLoader destruction because they hold a reference. + // To make sure we check for all pending jobs to be finished. + std::unique_lock lock{mutex}; + if (scheduled_jobs.empty() && finished_jobs.empty()) + return; + + std::vector scheduled; + std::vector finished; + scheduled.reserve(scheduled_jobs.size()); + finished.reserve(finished_jobs.size()); + for (const auto & [job, _] : scheduled_jobs) + scheduled.push_back(job->name); + for (const auto & job : finished_jobs) + finished.push_back(job->name); + LOG_ERROR(log, "Bug. Destruction with pending ({}) and finished ({}) load jobs.", fmt::join(scheduled, ", "), fmt::join(finished, ", ")); + abort(); } void AsyncLoader::start() @@ -236,6 +245,17 @@ void AsyncLoader::wait() for (auto & p : pools) p.thread_pool->wait(); lock.lock(); + + // If there is no way for all jobs to finish, throw LOGICAL_ERROR instead of deadlock + if (!scheduled_jobs.empty() && !hasWorker(lock)) + { + std::vector names; + names.reserve(scheduled_jobs.size()); + for (const auto & [job, _] : scheduled_jobs) + names.push_back(job->name); + LOG_ERROR(log, "Waiting for load jobs to finish while being stopped: {}.", fmt::join(names, ", ")); + abort(); + } } } @@ -243,10 +263,12 @@ void AsyncLoader::stop() { { std::unique_lock lock{mutex}; - is_running = false; - // NOTE: there is no need to notify because workers never wait + is_running = false; // NOTE: there is no need to notify because workers never wait } - wait(); + + // Wait for all currently running jobs to finish (and do NOT wait all pending jobs) + for (auto & p : pools) + p.thread_pool->wait(); } void AsyncLoader::schedule(LoadTask & task) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index c2a9c901f1c..b1b336d24dc 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -198,10 +198,6 @@ public: // Remove all jobs of this task from AsyncLoader. void remove(); - // Do not track jobs in this task. - // WARNING: Jobs will never be removed() and are going to be stored as finished jobs until ~AsyncLoader(). - void detach(); - // Return the final jobs in this tasks. This job subset should be used as `dependencies` for dependent jobs or tasks: // auto load_task = loadSomethingAsync(async_loader, load_after_task.goals(), something); const LoadJobSet & goals() const { return goal_jobs.empty() ? jobs : goal_jobs; } @@ -333,7 +329,6 @@ private: public: AsyncLoader(std::vector pool_initializers, bool log_failures_, bool log_progress_); - // Stops AsyncLoader before destruction // WARNING: all tasks instances should be destructed before associated AsyncLoader. ~AsyncLoader(); diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index d978d23750c..950c7bbab76 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -622,7 +622,13 @@ TEST(AsyncLoader, CustomDependencyFailure) auto dependent_job1 = makeLoadJob({ collect_job }, "dependent_job1", dependent_job_func); auto dependent_job2 = makeLoadJob({ collect_job }, "dependent_job2", dependent_job_func); auto dependent_job3 = makeLoadJob({ collect_job }, "dependent_job3", dependent_job_func); - auto task = t.schedule({ dependent_job1, dependent_job2, dependent_job3 }); // Other jobs should be discovery automatically + auto task = t.schedule({ + dependent_job1, dependent_job2, dependent_job3, + collect_job, + late_dep1, late_dep2, late_dep3, + good_dep1, good_dep2, good_dep3, + evil_dep1, evil_dep2, evil_dep3, + }); t.loader.wait(collect_job, true); canceled_sync.arrive_and_wait(); // (A) @@ -1022,8 +1028,10 @@ TEST(AsyncLoader, SetMaxThreads) }; // Generate enough independent jobs + std::vector tasks; + tasks.reserve(1000); for (int i = 0; i < 1000; i++) - t.schedule({makeLoadJob({}, "job", job_func)})->detach(); + tasks.push_back(t.schedule({makeLoadJob({}, "job", job_func)})); t.loader.start(); while (sync_index < syncs.size()) From 4dcf27e5b20f3c00e3cec066eb0c462b6dda3f38 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 18:27:15 +0100 Subject: [PATCH 17/21] /binary: allow specifying user/password/host --- programs/server/binary.html | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/programs/server/binary.html b/programs/server/binary.html index 988dd33a72a..74095dff537 100644 --- a/programs/server/binary.html +++ b/programs/server/binary.html @@ -60,10 +60,16 @@ /// If it is hosted on server, assume that it is the address of ClickHouse. if (location.protocol != 'file:') { host = location.origin; - user = 'default'; add_http_cors_header = false; } + if (window.location.search) { + const params = new URLSearchParams(window.location.search); + if (params.has('host')) { host = params.get('host'); } + if (params.has('user')) { user = params.get('user'); } + if (params.has('password')) { password = params.get('password'); } + } + let map = L.map('space', { crs: L.CRS.Simple, center: [-512, 512], From c7663c6d92b7841c118f23cc5c1f66589dff1c8b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 18:31:10 +0100 Subject: [PATCH 18/21] Remove tests --- .../02366_kql_create_table.reference | 4 - .../0_stateless/02366_kql_create_table.sql | 29 -- .../0_stateless/02366_kql_datatype.reference | 105 ----- .../0_stateless/02366_kql_datatype.sql | 117 ------ .../0_stateless/02366_kql_distinct.reference | 27 -- .../0_stateless/02366_kql_distinct.sql | 28 -- .../0_stateless/02366_kql_extend.reference | 32 -- .../queries/0_stateless/02366_kql_extend.sql | 61 --- .../02366_kql_func_binary.reference | 7 - .../0_stateless/02366_kql_func_binary.sql | 8 - .../02366_kql_func_datetime.reference | 76 ---- .../0_stateless/02366_kql_func_datetime.sql | 86 ----- .../02366_kql_func_dynamic.reference | 152 -------- .../0_stateless/02366_kql_func_dynamic.sql | 161 -------- .../0_stateless/02366_kql_func_ip.reference | 123 ------ .../queries/0_stateless/02366_kql_func_ip.sql | 131 ------- .../0_stateless/02366_kql_func_math.reference | 4 - .../0_stateless/02366_kql_func_math.sql | 7 - .../02366_kql_func_scalar.reference | 16 - .../0_stateless/02366_kql_func_scalar.sql | 26 -- .../02366_kql_func_string.reference | 360 ------------------ .../0_stateless/02366_kql_func_string.sql | 313 --------------- .../02366_kql_makeseries.reference | 60 --- .../0_stateless/02366_kql_makeseries.sql | 77 ---- .../0_stateless/02366_kql_mvexpand.reference | 65 ---- .../0_stateless/02366_kql_mvexpand.sql | 35 -- ...02366_kql_native_interval_format.reference | 23 -- .../02366_kql_native_interval_format.sql.j2 | 16 - .../02366_kql_operator_in_sql.reference | 60 --- .../0_stateless/02366_kql_operator_in_sql.sql | 42 -- .../0_stateless/02366_kql_summarize.reference | 92 ----- .../0_stateless/02366_kql_summarize.sql | 102 ----- .../0_stateless/02366_kql_tabular.reference | 139 ------- .../queries/0_stateless/02366_kql_tabular.sql | 88 ----- 34 files changed, 2672 deletions(-) delete mode 100644 tests/queries/0_stateless/02366_kql_create_table.reference delete mode 100644 tests/queries/0_stateless/02366_kql_create_table.sql delete mode 100644 tests/queries/0_stateless/02366_kql_datatype.reference delete mode 100644 tests/queries/0_stateless/02366_kql_datatype.sql delete mode 100644 tests/queries/0_stateless/02366_kql_distinct.reference delete mode 100644 tests/queries/0_stateless/02366_kql_distinct.sql delete mode 100644 tests/queries/0_stateless/02366_kql_extend.reference delete mode 100644 tests/queries/0_stateless/02366_kql_extend.sql delete mode 100644 tests/queries/0_stateless/02366_kql_func_binary.reference delete mode 100644 tests/queries/0_stateless/02366_kql_func_binary.sql delete mode 100644 tests/queries/0_stateless/02366_kql_func_datetime.reference delete mode 100644 tests/queries/0_stateless/02366_kql_func_datetime.sql delete mode 100644 tests/queries/0_stateless/02366_kql_func_dynamic.reference delete mode 100644 tests/queries/0_stateless/02366_kql_func_dynamic.sql delete mode 100644 tests/queries/0_stateless/02366_kql_func_ip.reference delete mode 100644 tests/queries/0_stateless/02366_kql_func_ip.sql delete mode 100644 tests/queries/0_stateless/02366_kql_func_math.reference delete mode 100644 tests/queries/0_stateless/02366_kql_func_math.sql delete mode 100644 tests/queries/0_stateless/02366_kql_func_scalar.reference delete mode 100644 tests/queries/0_stateless/02366_kql_func_scalar.sql delete mode 100644 tests/queries/0_stateless/02366_kql_func_string.reference delete mode 100644 tests/queries/0_stateless/02366_kql_func_string.sql delete mode 100644 tests/queries/0_stateless/02366_kql_makeseries.reference delete mode 100644 tests/queries/0_stateless/02366_kql_makeseries.sql delete mode 100644 tests/queries/0_stateless/02366_kql_mvexpand.reference delete mode 100644 tests/queries/0_stateless/02366_kql_mvexpand.sql delete mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.reference delete mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 delete mode 100644 tests/queries/0_stateless/02366_kql_operator_in_sql.reference delete mode 100644 tests/queries/0_stateless/02366_kql_operator_in_sql.sql delete mode 100644 tests/queries/0_stateless/02366_kql_summarize.reference delete mode 100644 tests/queries/0_stateless/02366_kql_summarize.sql delete mode 100644 tests/queries/0_stateless/02366_kql_tabular.reference delete mode 100644 tests/queries/0_stateless/02366_kql_tabular.sql diff --git a/tests/queries/0_stateless/02366_kql_create_table.reference b/tests/queries/0_stateless/02366_kql_create_table.reference deleted file mode 100644 index 35136b5ff42..00000000000 --- a/tests/queries/0_stateless/02366_kql_create_table.reference +++ /dev/null @@ -1,4 +0,0 @@ --- test create table -- -Theodore -Diaz -Theodore Diaz 28 diff --git a/tests/queries/0_stateless/02366_kql_create_table.sql b/tests/queries/0_stateless/02366_kql_create_table.sql deleted file mode 100644 index b266679b06a..00000000000 --- a/tests/queries/0_stateless/02366_kql_create_table.sql +++ /dev/null @@ -1,29 +0,0 @@ -DROP TABLE IF EXISTS Customers; -CREATE TABLE Customers -( - FirstName Nullable(String), - LastName String, - Occupation String, - Education String, - Age Nullable(UInt8) -) ENGINE = Memory; - -INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28),('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38); -Select '-- test create table --' ; -Select * from kql(Customers|project FirstName) limit 1;; -DROP TABLE IF EXISTS kql_table1; -CREATE TABLE kql_table1 ENGINE = Memory AS select *, now() as new_column From kql(Customers | project LastName | filter LastName=='Diaz'); -select LastName from kql_table1 limit 1; -DROP TABLE IF EXISTS kql_table2; -CREATE TABLE kql_table2 -( - FirstName Nullable(String), - LastName String, - Age Nullable(UInt8) -) ENGINE = Memory; -INSERT INTO kql_table2 select * from kql(Customers|project FirstName,LastName,Age | filter FirstName=='Theodore'); -select * from kql_table2 limit 1; --- select * from kql(Customers | where FirstName !in ("test", "test2")); -DROP TABLE IF EXISTS Customers; -DROP TABLE IF EXISTS kql_table1; -DROP TABLE IF EXISTS kql_table2; \ No newline at end of file diff --git a/tests/queries/0_stateless/02366_kql_datatype.reference b/tests/queries/0_stateless/02366_kql_datatype.reference deleted file mode 100644 index fe666f3734c..00000000000 --- a/tests/queries/0_stateless/02366_kql_datatype.reference +++ /dev/null @@ -1,105 +0,0 @@ --- bool -true -\N --- int -123 -\N --- long -123 -255 --1 -\N -456 --- real -0.01 -\N -nan -inf --inf --- datetime -2015-12-31 23:59:59.900000000 -2015-12-31 00:00:00.000000000 -2014-05-25 08:20:03.123456000 -2014-11-08 15:55:55.000000000 -2014-11-08 15:55:00.000000000 -2014-11-08 00:00:00.000000000 -\N -2014-05-25 08:20:03.123456000 -2014-11-08 15:55:55.123456000 --- time -1216984.12345 -45055.123 -86400 --86400 -6.000000000000001e-9 -6e-7 -172800 -259200 --- guid -\N --- timespan (time) -172800 -1800 -10 -0.1 -0.00001 -1e-7 -1120343 --- null -1 -\N \N \N \N \N --- decimal -\N -123.345 -100000 --- dynamic -\N -1 -86400 -[1,2,3] -[[1],[2],[3]] -['a','b','c'] --- cast functions -true -1 --- tobool("false") -false -1 --- tobool(1) -true -1 --- tobool(123) -true -1 --- tobool("abc") -\N -\N --- todouble() -123.4 -\N --- toreal() -123.4 -\N --- toint() -1 -\N --- tostring() -123 -1 --- todatetime() -1 -\N --- make_timespan() -01:12:00 01:12:30 1.12:30:55 --- totimespan() -1e-7 -60 -\N -1120343 --- tolong() -123 -\N --- todecimal() -123.345 -\N -\N diff --git a/tests/queries/0_stateless/02366_kql_datatype.sql b/tests/queries/0_stateless/02366_kql_datatype.sql deleted file mode 100644 index ecd29504298..00000000000 --- a/tests/queries/0_stateless/02366_kql_datatype.sql +++ /dev/null @@ -1,117 +0,0 @@ -set dialect = 'kusto'; - -print '-- bool' -print bool(true); -print bool(true); -print bool(null); -print '-- int'; -print int(123); -print int(null); -print int('4'); -- { clientError BAD_ARGUMENTS } -print '-- long'; -print long(123); -print long(0xff); -print long(-1); -print long(null); -print 456; -print '-- real'; -print real(0.01); -print real(null); -print real(nan); -print real(+inf); -print real(-inf); -print double('4.2'); -- { clientError BAD_ARGUMENTS } -print '-- datetime'; -print datetime(2015-12-31 23:59:59.9); -print datetime(2015-12-31); -print datetime('2014-05-25T08:20:03.123456'); -print datetime('2014-11-08 15:55:55'); -print datetime('2014-11-08 15:55'); -print datetime('2014-11-08'); -print datetime(null); -print datetime('2014-05-25T08:20:03.123456Z'); -print datetime('2014-11-08 15:55:55.123456Z'); -print '-- time'; -print time('14.02:03:04.12345'); -print time('12:30:55.123'); -print time(1d); -print time(-1d); -print time(6nanoseconds); -print time(6tick); -print time(2); -print time(2) + 1d; -print '-- guid' -print guid(74be27de-1e4e-49d9-b579-fe0b331d3642); -print guid(null); -print '-- timespan (time)'; -print timespan(2d); -- 2 days ---print timespan(1.5h); -- 1.5 hour -print timespan(30m); -- 30 minutes -print timespan(10s); -- 10 seconds ---print timespan(0.1s); -- 0.1 second -print timespan(100ms); -- 100 millisecond -print timespan(10microsecond); -- 10 microseconds -print timespan(1tick); -- 100 nanoseconds ---print timespan(1.5h) / timespan(30m); -print timespan('12.23:12:23') / timespan(1s); -print '-- null'; -print isnull(null); -print bool(null), int(null), long(null), real(null), double(null); -print '-- decimal'; -print decimal(null); -print decimal(123.345); -print decimal(1e5); -print '-- dynamic'; -- no support for mixed types and bags for now -print dynamic(null); -print dynamic(1); -print dynamic(timespan(1d)); -print dynamic([1,2,3]); -print dynamic([[1], [2], [3]]); -print dynamic(['a', "b", 'c']); -print '-- cast functions' -print '--tobool("true")'; -- == true -print tobool('true'); -- == true -print tobool('true') == toboolean('true'); -- == true -print '-- tobool("false")'; -- == false -print tobool('false'); -- == false -print tobool('false') == toboolean('false'); -- == false -print '-- tobool(1)'; -- == true -print tobool(1); -- == true -print tobool(1) == toboolean(1); -- == true -print '-- tobool(123)'; -- == true -print tobool(123); -- == true -print tobool(123) == toboolean(123); -- == true -print '-- tobool("abc")'; -- == null -print tobool('abc'); -- == null -print tobool('abc') == toboolean('abc'); -- == null -print '-- todouble()'; -print todouble('123.4'); -print todouble('abc') == null; -print '-- toreal()'; -print toreal("123.4"); -print toreal('abc') == null; -print '-- toint()'; -print toint("123") == int(123); -print toint('abc'); -print '-- tostring()'; -print tostring(123); -print tostring(null) == ''; -print '-- todatetime()'; -print todatetime("2015-12-24") == datetime(2015-12-24); -print todatetime('abc') == null; -print '-- make_timespan()'; -print v1=make_timespan(1,12), v2=make_timespan(1,12,30), v3=make_timespan(1,12,30,55.123); -print '-- totimespan()'; -print totimespan(1tick); -print totimespan('0.00:01:00'); -print totimespan('abc'); -print totimespan('12.23:12:23') / totimespan(1s); --- print totimespan(strcat('12.', '23', ':12:', '23')) / timespan(1s); -> 1120343 -print '-- tolong()'; -print tolong('123'); -print tolong('abc'); -print '-- todecimal()'; -print todecimal(123.345); -print todecimal(null); -print todecimal('abc'); --- print todecimal(4 * 2 + 3); -> 11 diff --git a/tests/queries/0_stateless/02366_kql_distinct.reference b/tests/queries/0_stateless/02366_kql_distinct.reference deleted file mode 100644 index 2100f44f18c..00000000000 --- a/tests/queries/0_stateless/02366_kql_distinct.reference +++ /dev/null @@ -1,27 +0,0 @@ --- distinct * -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Peter Nara Skilled Manual Graduate Degree 26 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 -\N why Professional Partial College 38 --- distinct one column -- -Skilled Manual -Management abcd defg -Professional --- distinct two column -- -Skilled Manual Bachelors -Management abcd defg Bachelors -Skilled Manual Graduate Degree -Professional Graduate Degree -Professional Partial College --- distinct with where -- -Skilled Manual Bachelors -Management abcd defg Bachelors -Skilled Manual Graduate Degree -Professional Graduate Degree -Professional Partial College --- distinct with where, order -- -Skilled Manual Bachelors -Skilled Manual Graduate Degree -Professional Graduate Degree diff --git a/tests/queries/0_stateless/02366_kql_distinct.sql b/tests/queries/0_stateless/02366_kql_distinct.sql deleted file mode 100644 index 3c997eb4865..00000000000 --- a/tests/queries/0_stateless/02366_kql_distinct.sql +++ /dev/null @@ -1,28 +0,0 @@ -DROP TABLE IF EXISTS Customers; -CREATE TABLE Customers -( - FirstName Nullable(String), - LastName String, - Occupation String, - Education String, - Age Nullable(UInt8) -) ENGINE = Memory; - -INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28), ('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38); - -set dialect = 'kusto'; - -print '-- distinct * --'; -Customers | distinct *; - -print '-- distinct one column --'; -Customers | distinct Occupation; - -print '-- distinct two column --'; -Customers | distinct Occupation, Education; - -print '-- distinct with where --'; -Customers where Age <30 | distinct Occupation, Education; - -print '-- distinct with where, order --'; -Customers |where Age <30 | order by Age| distinct Occupation, Education; diff --git a/tests/queries/0_stateless/02366_kql_extend.reference b/tests/queries/0_stateless/02366_kql_extend.reference deleted file mode 100644 index 2936c9ea19c..00000000000 --- a/tests/queries/0_stateless/02366_kql_extend.reference +++ /dev/null @@ -1,32 +0,0 @@ --- extend #1 -- -Aldi Apple 4 2016-09-10 400 -Costco Apple 2 2016-09-11 200 --- extend #2 -- -Apple 200 -Apple 400 --- extend #3 -- -Apple cost 480 on average based on 5 samples. -Snargaluff cost 28080 on average based on 5 samples. --- extend #4 -- -1 --- extend #5 -- -Aldi Apple 4 2016-09-10 Apple was purchased from Aldi for $4 on 2016-09-10 400 -Costco Apple 2 2016-09-11 Apple was purchased from Costco for $2 on 2016-09-11 200 --- extend #6 -- -Aldi Apple 2016-09-10 400 -Costco Apple 2016-09-11 200 -Aldi Apple 2016-09-10 600 -Costco Snargaluff 2016-09-12 10000 -Aldi Apple 2016-09-12 700 -Aldi Snargaluff 2016-09-11 40000 -Costco Snargaluff 2016-09-12 10400 -Aldi Apple 2016-09-12 500 -Aldi Snargaluff 2016-09-11 60000 -Costco Snargaluff 2016-09-10 20000 --- extend #7 -- -5 --- extend #8 -- --- extend #9 -- --- extend #10 -- --- extend #11 -- -5 [2,1] diff --git a/tests/queries/0_stateless/02366_kql_extend.sql b/tests/queries/0_stateless/02366_kql_extend.sql deleted file mode 100644 index 0a3c1f3dcd4..00000000000 --- a/tests/queries/0_stateless/02366_kql_extend.sql +++ /dev/null @@ -1,61 +0,0 @@ --- datatable(Supplier:string, Fruit:string, Price: real, Purchase:datetime) --- [ --- 'Aldi','Apple',4,'2016-09-10', --- 'Costco','Apple',2,'2016-09-11', --- 'Aldi','Apple',6,'2016-09-10', --- 'Costco','Snargaluff',100,'2016-09-12', --- 'Aldi','Apple',7,'2016-09-12', --- 'Aldi','Snargaluff',400,'2016-09-11', --- 'Costco','Snargaluff',104,'2016-09-12', --- 'Aldi','Apple',5,'2016-09-12', --- 'Aldi','Snargaluff',600,'2016-09-11', --- 'Costco','Snargaluff',200,'2016-09-10', --- ] - - -DROP TABLE IF EXISTS Ledger; -CREATE TABLE Ledger -( - Supplier Nullable(String), - Fruit String , - Price Float64, - Purchase Date -) ENGINE = Memory; -INSERT INTO Ledger VALUES ('Aldi','Apple',4,'2016-09-10'), ('Costco','Apple',2,'2016-09-11'), ('Aldi','Apple',6,'2016-09-10'), ('Costco','Snargaluff',100,'2016-09-12'), ('Aldi','Apple',7,'2016-09-12'), ('Aldi','Snargaluff',400,'2016-09-11'),('Costco','Snargaluff',104,'2016-09-12'),('Aldi','Apple',5,'2016-09-12'),('Aldi','Snargaluff',600,'2016-09-11'),('Costco','Snargaluff',200,'2016-09-10'); - --- This test requies sorting after some of aggregations but I don't know KQL, sorry -set max_bytes_before_external_group_by = 0; -set dialect = 'kusto'; - -print '-- extend #1 --'; -Ledger | extend PriceInCents = 100 * Price | take 2; - -print '-- extend #2 --'; -Ledger | extend PriceInCents = 100 * Price | sort by PriceInCents asc | project Fruit, PriceInCents | take 2; - -print '-- extend #3 --'; -Ledger | extend PriceInCents = 100 * Price | sort by PriceInCents asc | project Fruit, PriceInCents | summarize AveragePrice = avg(PriceInCents), Purchases = count() by Fruit | extend Sentence = strcat(Fruit, ' cost ', tostring(AveragePrice), ' on average based on ', tostring(Purchases), ' samples.') | project Sentence; - -print '-- extend #4 --'; -Ledger | extend a = Price | extend b = a | extend c = a, d = b + 500 | extend Pass = bool(b == a and c == a and d == b + 500) | summarize binary_all_and(Pass); - -print '-- extend #5 --'; -Ledger | take 2 | extend strcat(Fruit, ' was purchased from ', Supplier, ' for $', tostring(Price), ' on ', tostring(Purchase)) | extend PriceInCents = 100 * Price; - -print '-- extend #6 --'; -Ledger | extend Price = 100 * Price; - -print '-- extend #7 --'; -print a = 4 | extend a = 5; - -print '-- extend #8 --'; --- print x = 5 | extend array_sort_desc(range(0, x), range(1, x + 1)) - -print '-- extend #9 --'; -print x = 19 | extend = 4 + ; -- { clientError SYNTAX_ERROR } - -print '-- extend #10 --'; -Ledger | extend PriceInCents = * Price | sort by PriceInCents asc | project Fruit, PriceInCents | summarize AveragePrice = avg(PriceInCents), Purchases = count() by Fruit | extend Sentence = strcat(Fruit, ' cost ', tostring(AveragePrice), ' on average based on ', tostring(Purchases), ' samples.') | project Sentence; -- { clientError SYNTAX_ERROR } - -print '-- extend #11 --'; -- should ideally return this in the future: 5 [2,1] because of the alias ex -print x = 5 | extend ex = array_sort_desc(dynamic([1, 2]), dynamic([3, 4])); diff --git a/tests/queries/0_stateless/02366_kql_func_binary.reference b/tests/queries/0_stateless/02366_kql_func_binary.reference deleted file mode 100644 index 6276cd6d867..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_binary.reference +++ /dev/null @@ -1,7 +0,0 @@ - -- binary functions -4 7 -1 -1 -1 -7 3 -1 diff --git a/tests/queries/0_stateless/02366_kql_func_binary.sql b/tests/queries/0_stateless/02366_kql_func_binary.sql deleted file mode 100644 index 824022b564c..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_binary.sql +++ /dev/null @@ -1,8 +0,0 @@ -set dialect='kusto'; -print ' -- binary functions'; -print binary_and(4,7), binary_or(4,7); -print binary_shift_left(1, 1) == binary_shift_left(1, 65); -print binary_shift_right(2, 1) == binary_shift_right(2, 65); -print binary_shift_right(binary_shift_left(1, 65), 65) == 1; -print binary_xor(2, 5), bitset_count_ones(42); -print bitset_count_ones(binary_shift_left(binary_and(4,7), 1)); diff --git a/tests/queries/0_stateless/02366_kql_func_datetime.reference b/tests/queries/0_stateless/02366_kql_func_datetime.reference deleted file mode 100644 index 40d8d7e19ac..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_datetime.reference +++ /dev/null @@ -1,76 +0,0 @@ --- dayofmonth() -31 --- dayofweek() -4.00:00:00 --- dayofyear() -365 --- getmonth() -10 --- getyear() -2015 --- hoursofday() -23 --- startofday() -2017-01-01 00:00:00.000000000 -2016-12-31 00:00:00.000000000 -2017-01-02 00:00:00.000000000 --- endofday() -2017-01-01 23:59:59.999999000 -2016-12-31 23:59:59.999999000 -2017-01-02 23:59:59.999999000 --- endofmonth() -2017-01-31 23:59:59.999999000 -2016-12-31 23:59:59.999999000 -2017-02-28 23:59:59.999999000 -2022-09-30 23:59:59.999999000 --- startofweek() -2017-01-01 00:00:00.000000000 -2016-12-25 00:00:00.000000000 -2017-01-08 00:00:00.000000000 --- endofweek() -2017-01-07 23:59:59.999999000 -2016-12-31 23:59:59.999999000 -2017-01-14 23:59:59.999999000 --- startofyear() -2017-01-01 00:00:00.000000000 -2016-01-01 00:00:00.000000000 -2018-01-01 00:00:00.000000000 --- endofyear() -2017-12-31 23:59:59.999999000 -2016-12-31 23:59:59.999999000 -2018-12-31 23:59:59.999999000 --- unixtime_seconds_todatetime() -2019-01-01 00:00:00.000000000 -1970-01-02 00:00:00.000000000 -1969-12-31 00:00:00.000000000 --- unixtime_microseconds_todatetime -2019-01-01 00:00:00.000000 --- unixtime_milliseconds_todatetime() -2019-01-01 00:00:00.000 --- unixtime_nanoseconds_todatetime() -2019-01-01 00:00:00.000000000 --- weekofyear() -52 --- monthofyear() -12 --- weekofyear() -52 --- now() -1 --- make_datetime() -1 -2017-10-01 12:10:00.0000000 -2017-10-01 12:11:00.0000000 --- format_datetime -15-12-14 02:03:04.1234500 -17-01-29 [09:00:05] 2017-01-29 [09:00:05] 17-01-29 [09:00:05 AM] --- format_timespan() -02:03:04.1234500 -29.09:00:05:12 --- ago() --- datetime_diff() -17 2 13 4 29 2 5 10 --- datetime_part() -2017 4 10 44 30 303 01 02 03 --- datetime_add() -2018-01-01 00:00:00.0000000 2017-04-01 00:00:00.0000000 2017-02-01 00:00:00.0000000 2017-01-08 00:00:00.0000000 2017-01-02 00:00:00.0000000 2017-01-01 01:00:00.0000000 2017-01-01 00:01:00.0000000 2017-01-01 00:00:01.0000000 diff --git a/tests/queries/0_stateless/02366_kql_func_datetime.sql b/tests/queries/0_stateless/02366_kql_func_datetime.sql deleted file mode 100644 index b1fba4166a9..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_datetime.sql +++ /dev/null @@ -1,86 +0,0 @@ -set dialect = 'kusto'; - -print '-- dayofmonth()'; -print dayofmonth(datetime(2015-12-31)); -print '-- dayofweek()'; -print dayofweek(datetime(2015-12-31)); -print '-- dayofyear()'; -print dayofyear(datetime(2015-12-31)); -print '-- getmonth()'; -print getmonth(datetime(2015-10-12)); -print '-- getyear()'; -print getyear(datetime(2015-10-12)); -print '-- hoursofday()'; -print hourofday(datetime(2015-12-31 23:59:59.9)); -print '-- startofday()'; -print startofday(datetime(2017-01-01 10:10:17)); -print startofday(datetime(2017-01-01 10:10:17), -1); -print startofday(datetime(2017-01-01 10:10:17), 1); -print '-- endofday()'; -print endofday(datetime(2017-01-01 10:10:17)); -print endofday(datetime(2017-01-01 10:10:17), -1); -print endofday(datetime(2017-01-01 10:10:17), 1); -print '-- endofmonth()'; -print endofmonth(datetime(2017-01-01 10:10:17)); -print endofmonth(datetime(2017-01-01 10:10:17), -1); -print endofmonth(datetime(2017-01-01 10:10:17), 1); -print endofmonth(datetime(2022-09-23)); -print '-- startofweek()'; -print startofweek(datetime(2017-01-01 10:10:17)); -print startofweek(datetime(2017-01-01 10:10:17), -1); -print startofweek(datetime(2017-01-01 10:10:17), 1); -print '-- endofweek()'; -print endofweek(datetime(2017-01-01 10:10:17)); -print endofweek(datetime(2017-01-01 10:10:17), -1); -print endofweek(datetime(2017-01-01 10:10:17), 1); -print '-- startofyear()'; -print startofyear(datetime(2017-01-01 10:10:17)); -print startofyear(datetime(2017-01-01 10:10:17), -1); -print startofyear(datetime(2017-01-01 10:10:17), 1); -print '-- endofyear()'; -print endofyear(datetime(2017-01-01 10:10:17)); -print endofyear(datetime(2017-01-01 10:10:17), -1); -print endofyear(datetime(2017-01-01 10:10:17), 1); -print '-- unixtime_seconds_todatetime()'; -print unixtime_seconds_todatetime(1546300800); -print unixtime_seconds_todatetime(1d); -print unixtime_seconds_todatetime(-1d); -print '-- unixtime_microseconds_todatetime'; -print unixtime_microseconds_todatetime(1546300800000000); -print '-- unixtime_milliseconds_todatetime()'; -print unixtime_milliseconds_todatetime(1546300800000); -print '-- unixtime_nanoseconds_todatetime()'; -print unixtime_nanoseconds_todatetime(1546300800000000000); -print '-- weekofyear()'; -print week_of_year(datetime(2000-01-01)); -print '-- monthofyear()'; -print monthofyear(datetime(2015-12-31)); -print '-- weekofyear()'; -print week_of_year(datetime(2000-01-01)); -print '-- now()'; -print getyear(now(-2d))>1900; -print '-- make_datetime()'; -print make_datetime(2017,10,01,12,10) == datetime(2017-10-01 12:10:00); -print year_month_day_hour_minute = make_datetime(2017,10,01,12,10); -print year_month_day_hour_minute_second = make_datetime(2017,10,01,12,11,0.1234567); -print '-- format_datetime'; -print format_datetime(datetime(2015-12-14 02:03:04.12345), 'y-M-d h:m:s.fffffff'); -print v1=format_datetime(datetime(2017-01-29 09:00:05),'yy-MM-dd [HH:mm:ss]'), v2=format_datetime(datetime(2017-01-29 09:00:05), 'yyyy-M-dd [H:mm:ss]'), v3=format_datetime(datetime(2017-01-29 09:00:05), 'yy-MM-dd [hh:mm:ss tt]'); -print '-- format_timespan()'; -print format_timespan(time('14.02:03:04.12345'), 'h:m:s.fffffff'); -print v1=format_timespan(time('29.09:00:05.12345'), 'dd.hh:mm:ss:FF'); --- print v2=format_timespan(time('29.09:00:05.12345'), 'ddd.h:mm:ss [fffffff]'); == '029.9:00:05 [1234500]' -print '-- ago()'; --- print ago(1d) - now(); -print '-- datetime_diff()'; -print year = datetime_diff('year',datetime(2017-01-01),datetime(2000-12-31)), quarter = datetime_diff('quarter',datetime(2017-07-01),datetime(2017-03-30)), month = datetime_diff('month',datetime(2017-01-01),datetime(2015-12-30)), week = datetime_diff('week',datetime(2017-10-29 00:00),datetime(2017-09-30 23:59)), day = datetime_diff('day',datetime(2017-10-29 00:00),datetime(2017-09-30 23:59)), hour = datetime_diff('hour',datetime(2017-10-31 01:00),datetime(2017-10-30 23:59)), minute = datetime_diff('minute',datetime(2017-10-30 23:05:01),datetime(2017-10-30 23:00:59)), second = datetime_diff('second',datetime(2017-10-30 23:00:10.100),datetime(2017-10-30 23:00:00.900)); --- millisecond = datetime_diff('millisecond',datetime(2017-10-30 23:00:00.200100),datetime(2017-10-30 23:00:00.100900)), --- microsecond = datetime_diff('microsecond',datetime(2017-10-30 23:00:00.1009001),datetime(2017-10-30 23:00:00.1008009)), --- nanosecond = datetime_diff('nanosecond',datetime(2017-10-30 23:00:00.0000000),datetime(2017-10-30 23:00:00.0000007)) -print '-- datetime_part()'; -print year = datetime_part("year", datetime(2017-10-30 01:02:03.7654321)),quarter = datetime_part("quarter", datetime(2017-10-30 01:02:03.7654321)),month = datetime_part("month", datetime(2017-10-30 01:02:03.7654321)),weekOfYear = datetime_part("week_of_year", datetime(2017-10-30 01:02:03.7654321)),day = datetime_part("day", datetime(2017-10-30 01:02:03.7654321)),dayOfYear = datetime_part("dayOfYear", datetime(2017-10-30 01:02:03.7654321)),hour = datetime_part("hour", datetime(2017-10-30 01:02:03.7654321)),minute = datetime_part("minute", datetime(2017-10-30 01:02:03.7654321)),second = datetime_part("second", datetime(2017-10-30 01:02:03.7654321)); --- millisecond = datetime_part("millisecond", dt), --- microsecond = datetime_part("microsecond", dt), --- nanosecond = datetime_part("nanosecond", dt) -print '-- datetime_add()'; -print year = datetime_add('year',1,make_datetime(2017,1,1)),quarter = datetime_add('quarter',1,make_datetime(2017,1,1)),month = datetime_add('month',1,make_datetime(2017,1,1)),week = datetime_add('week',1,make_datetime(2017,1,1)),day = datetime_add('day',1,make_datetime(2017,1,1)),hour = datetime_add('hour',1,make_datetime(2017,1,1)),minute = datetime_add('minute',1,make_datetime(2017,1,1)),second = datetime_add('second',1,make_datetime(2017,1,1)); \ No newline at end of file diff --git a/tests/queries/0_stateless/02366_kql_func_dynamic.reference b/tests/queries/0_stateless/02366_kql_func_dynamic.reference deleted file mode 100644 index 564f1eebc4b..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_dynamic.reference +++ /dev/null @@ -1,152 +0,0 @@ --- constant index value -1 c ['A',NULL,'C'] --- array_length() -1 -1 --- array_sum() -1 -1 --- array_index_of() -3 -1 --- array_iif() -[1,5,3] -[1,5,3] -[1,5,NULL] -[NULL,NULL,NULL] --- array_concat() -[1,2,3,4,5,6] --- array_reverse() -[] -[1] -[4,3,2,1] -['example','an','is','this'] --- array_rotate_left() -[] -[] -[] -[3,4,5,1,2] -[1,2,3,4,5] -[3,4,5,1,2] -[4,5,1,2,3] -[1,2,3,4,5] -[4,5,1,2,3] --- array_rotate_right() -[] -[] -[] -[4,5,1,2,3] -[1,2,3,4,5] -[4,5,1,2,3] -[3,4,5,1,2] -[1,2,3,4,5] -[3,4,5,1,2] --- array_shift_left() -[] -[] -[] -[3,4,5,NULL,NULL] -[NULL,NULL,1,2,3] -[3,4,5,-1,-1] -['c','',''] --- array_shift_right() -[] -[] -[] -[3,4,5,NULL,NULL] -[NULL,NULL,1,2,3] -[3,4,5,-1,-1] -['c','',''] --- array_slice() -[3,4] --- array_split() -[[1],[2,3],[4,5]] -[[1,2],[3,4,5]] -[[1],[2,3],[4,5]] -[[1,2,3,4],[],[4,5]] --- array_sort_asc() -(['a','c','c','d',NULL]) -([1,2,3,4]) -['a','b','c'] -(['p','q','r'],['hello','clickhouse','world']) -([NULL,'a','c','c','d']) -([NULL,'a','c','c','d']) -([NULL,NULL,NULL]) -[1,2,3,NULL,NULL] -['a','e','b','c','d'] -(['George','John','Paul','Ringo']) -(['blue','green','yellow',NULL,NULL]) -([NULL,NULL,'blue','green','yellow']) --- array_sort_desc() -(['d','c','c','a',NULL]) -([4,3,2,1]) -['c','b','a'] -(['r','q','p'],['world','clickhouse','hello']) -([NULL,'d','c','c','a']) -([NULL,'d','c','c','a']) -([NULL,NULL,NULL]) -[3,2,1,NULL,NULL] -['d','c','b','e','a'] -(['Ringo','Paul','John','George']) -(['yellow','green','blue',NULL,NULL]) -([NULL,NULL,'yellow','green','blue']) --- jaccard_index() -0.75 -0 -0 -nan -0 -0.75 -0.25 --- pack_array() -1 2 4 [1,2,4] -['ab','0.0.0.42','4.2'] --- repeat() -[] -[1,1,1] -['asd','asd','asd'] -[86400,86400,86400] -[true,true,true] -[NULL] -[NULL] --- set_difference() -[] -[] -[] -[] -[4,5,6] -[4] -[1,3] -[1,2,3] -['d','s'] -['Chewbacca','Han Solo'] --- set_has_element() -0 -1 -0 -1 -0 --- set_intersect() -[] -[1,2,3] -[1,2,3] -[] -[5] -[] -['a'] -['Darth Vader'] --- set_union() -[] -[1,2,3] -[1,2,3,4,5,6] -[1,2,3,4] -[1,2,3,4,5] -[1,2,3] -['a','d','f','s'] -['Chewbacca','Darth Sidious','Darth Vader','Han Solo'] --- zip() -[] -[[1,2],[3,4],[5,6]] -[['Darth','Vader','has a suit'],['Master','Yoda','doesn\'t have a suit']] -[[1,10],[2,20],[3,NULL]] -[[NULL,1],[NULL,2],[NULL,3]] diff --git a/tests/queries/0_stateless/02366_kql_func_dynamic.sql b/tests/queries/0_stateless/02366_kql_func_dynamic.sql deleted file mode 100644 index b0956f032d0..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_dynamic.sql +++ /dev/null @@ -1,161 +0,0 @@ -DROP TABLE IF EXISTS array_test; -CREATE TABLE array_test (floats Array(Float64), - strings Array(String), - nullable_strings Array(Nullable(String)) - ) ENGINE=Memory; -INSERT INTO array_test VALUES([1.0, 2.5], ['a', 'c'], ['A', NULL, 'C']); -set dialect = 'kusto'; -print '-- constant index value'; -array_test | project floats[0], strings[1], nullable_strings; -print '-- array_length()'; -print array_length(dynamic(['John', 'Denver', 'Bob', 'Marley'])) == 4; -print array_length(dynamic([1, 2, 3])) == 3; -print '-- array_sum()'; -print array_sum(dynamic([2, 5, 3])) == 10; -print array_sum(dynamic([2.5, 5.5, 3])) == 11; -print '-- array_index_of()'; -print array_index_of(dynamic(['John', 'Denver', 'Bob', 'Marley']), 'Marley'); -print array_index_of(dynamic([1, 2, 3]), 2); -print '-- array_iif()'; -print array_iif(dynamic([true,false,true]), dynamic([1,2,3]), dynamic([4,5,6])); -print array_iif(dynamic([1,0,1]), dynamic([1,2,3]), dynamic([4,5,6])); -print array_iif(dynamic([true,false,true]), dynamic([1,2]), dynamic([4,5,6])); -print array_iif(dynamic(['a','b','c']), dynamic([1,2,3]), dynamic([4,5,6])); -print '-- array_concat()'; -print array_concat(dynamic([1,2,3]),dynamic([4,5,6])); -print '-- array_reverse()'; -print array_reverse(dynamic([])); -print array_reverse(dynamic([1])); -print array_reverse(dynamic([1,2,3,4])); -print array_reverse(dynamic(["this", "is", "an", "example"])); -print '-- array_rotate_left()'; -print array_rotate_left(dynamic([]), 0); -print array_rotate_left(dynamic([]), 500); -print array_rotate_left(dynamic([]), -500); -print array_rotate_left(dynamic([1,2,3,4,5]), 2); -print array_rotate_left(dynamic([1,2,3,4,5]), 5); -print array_rotate_left(dynamic([1,2,3,4,5]), 7); -print array_rotate_left(dynamic([1,2,3,4,5]), -2); -print array_rotate_left(dynamic([1,2,3,4,5]), -5); -print array_rotate_left(dynamic([1,2,3,4,5]), -7); -print '-- array_rotate_right()'; -print array_rotate_right(dynamic([]), 0); -print array_rotate_right(dynamic([]), 500); -print array_rotate_right(dynamic([]), -500); -print array_rotate_right(dynamic([1,2,3,4,5]), 2); -print array_rotate_right(dynamic([1,2,3,4,5]), 5); -print array_rotate_right(dynamic([1,2,3,4,5]), 7); -print array_rotate_right(dynamic([1,2,3,4,5]), -2); -print array_rotate_right(dynamic([1,2,3,4,5]), -5); -print array_rotate_right(dynamic([1,2,3,4,5]), -7); -print '-- array_shift_left()'; -print array_shift_left(dynamic([]), 0); -print array_shift_left(dynamic([]), 555); -print array_shift_left(dynamic([]), -555); -print array_shift_left(dynamic([1,2,3,4,5]), 2); -print array_shift_left(dynamic([1,2,3,4,5]), -2); -print array_shift_left(dynamic([1,2,3,4,5]), 2, -1); -print array_shift_left(dynamic(['a', 'b', 'c']), 2); -print '-- array_shift_right()'; -print array_shift_left(dynamic([]), 0); -print array_shift_left(dynamic([]), 555); -print array_shift_left(dynamic([]), -555); -print array_shift_right(dynamic([1,2,3,4,5]), -2); -print array_shift_right(dynamic([1,2,3,4,5]), 2); -print array_shift_right(dynamic([1,2,3,4,5]), -2, -1); -print array_shift_right(dynamic(['a', 'b', 'c']), -2); -print '-- array_slice()'; ---print array_slice(dynamic([1,2,3]), 1, 2); -- will enable whe analyzer dixed -print array_slice(dynamic([1,2,3,4,5]), -3, -2); -print '-- array_split()'; -print array_split(dynamic([1,2,3,4,5]), dynamic([1,-2])); -print array_split(dynamic([1,2,3,4,5]), 2); -print array_split(dynamic([1,2,3,4,5]), dynamic([1,3])); -print array_split(dynamic([1,2,3,4,5]), dynamic([-1,-2])); -print '-- array_sort_asc()'; -print array_sort_asc(dynamic([null, 'd', 'a', 'c', 'c'])); -print array_sort_asc(dynamic([4, 1, 3, 2])); -print array_sort_asc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))[0]; -print array_sort_asc(dynamic(['q', 'p', 'r']), dynamic(['clickhouse','hello', 'world'])); -print array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , false); -print array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2); -print array_sort_asc( dynamic([null, null, null]) , false); -print array_sort_asc(dynamic([2, 1, null,3, null]), dynamic([20, 10, 40, 30, 50]), 1 < 2)[0]; -print array_sort_asc(dynamic(['1','3','4','5','2']),dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]))[3]; -print array_sort_asc(split("John,Paul,George,Ringo", ",")); -print array_sort_asc(dynamic([null,"blue","yellow","green",null])); -print array_sort_asc(dynamic([null,"blue","yellow","green",null]), false); -print '-- array_sort_desc()'; -print array_sort_desc(dynamic([null, 'd', 'a', 'c', 'c'])); -print array_sort_desc(dynamic([4, 1, 3, 2])); -print array_sort_desc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))[0]; -print array_sort_desc(dynamic(['q', 'p', 'r']), dynamic(['clickhouse','hello', 'world'])); -print array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , false); -print array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2); -print array_sort_desc( dynamic([null, null, null]) , false); -print array_sort_desc(dynamic([2, 1, null,3, null]), dynamic([20, 10, 40, 30, 50]), 1 < 2)[0]; -print array_sort_desc(dynamic(['1','3','4','5','2']),dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]))[3]; -print array_sort_desc(split("John,Paul,George,Ringo", ",")); -print array_sort_desc(dynamic([null,"blue","yellow","green",null])); -print array_sort_desc(dynamic([null,"blue","yellow","green",null]), false); -print '-- jaccard_index()'; -print jaccard_index(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3, 4, 4, 4])); -print jaccard_index(dynamic([1, 2, 3]), dynamic([])); -print jaccard_index(dynamic([]), dynamic([1, 2, 3, 4])); -print jaccard_index(dynamic([]), dynamic([])); -print jaccard_index(dynamic([1, 2, 3]), dynamic([4, 5, 6, 7])); -print jaccard_index(dynamic(['a', 's', 'd']), dynamic(['f', 'd', 's', 'a'])); -print jaccard_index(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])); -print '-- pack_array()'; -print pack_array(); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -print x = 1 | extend y = x * 2 | extend z = y * 2 | extend pack_array(x,y,z); -print pack_array(strcat('a', 'b'), format_ipv4(42), tostring(4.2)); -print '-- repeat()'; -print repeat(1, 0); -print repeat(1, 3); -print repeat("asd", 3); -print repeat(timespan(1d), 3); -print repeat(true, 3); -print repeat(1, -3); -print repeat(6.7,-4); -print '-- set_difference()'; -print set_difference(dynamic([]), dynamic([])); -print set_difference(dynamic([]), dynamic([9])); -print set_difference(dynamic([]), dynamic(["asd"])); -print set_difference(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])); -print array_sort_asc(set_difference(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[0]; -print set_difference(dynamic([4]), dynamic([1, 2, 3])); -print array_sort_asc(set_difference(dynamic([1, 2, 3, 4, 5]), dynamic([5]), dynamic([2, 4])))[0]; -print array_sort_asc(set_difference(dynamic([1, 2, 3]), dynamic([])))[0]; -print array_sort_asc(set_difference(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])))[0]; -print array_sort_asc(set_difference(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])))[0]; -print '-- set_has_element()'; -print set_has_element(dynamic([]), 9); -print set_has_element(dynamic(["this", "is", "an", "example"]), "example"); -print set_has_element(dynamic(["this", "is", "an", "example"]), "examplee"); -print set_has_element(dynamic([1, 2, 3]), 2); -print set_has_element(dynamic([1, 2, 3, 4.2]), 4); -print '-- set_intersect()'; -print set_intersect(dynamic([]), dynamic([])); -print array_sort_asc(set_intersect(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])))[0]; -print array_sort_asc(set_intersect(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[0]; -print set_intersect(dynamic([4]), dynamic([1, 2, 3])); -print set_intersect(dynamic([1, 2, 3, 4, 5]), dynamic([1, 3, 5]), dynamic([2, 5])); -print set_intersect(dynamic([1, 2, 3]), dynamic([])); -print set_intersect(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])); -print set_intersect(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])); -print '-- set_union()'; -print set_union(dynamic([]), dynamic([])); -print array_sort_asc(set_union(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])))[0]; -print array_sort_asc(set_union(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[0]; -print array_sort_asc(set_union(dynamic([4]), dynamic([1, 2, 3])))[0]; -print array_sort_asc(set_union(dynamic([1, 3, 4]), dynamic([5]), dynamic([2, 4])))[0]; -print array_sort_asc(set_union(dynamic([1, 2, 3]), dynamic([])))[0]; -print array_sort_asc(set_union(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])))[0]; -print array_sort_asc(set_union(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])))[0]; -print '-- zip()'; -print zip(dynamic([]), dynamic([])); -print zip(dynamic([1,3,5]), dynamic([2,4,6])); -print zip(dynamic(['Darth','Master']), dynamic(['Vader','Yoda']), dynamic(['has a suit','doesn\'t have a suit'])); -print zip(dynamic([1,2,3]), dynamic([10,20])); -print zip(dynamic([]), dynamic([1,2,3])); \ No newline at end of file diff --git a/tests/queries/0_stateless/02366_kql_func_ip.reference b/tests/queries/0_stateless/02366_kql_func_ip.reference deleted file mode 100644 index 2a0bbf53fff..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_ip.reference +++ /dev/null @@ -1,123 +0,0 @@ --- ipv4_is_private(\'127.0.0.1\') -0 --- ipv4_is_private(\'10.1.2.3\') -1 --- ipv4_is_private(\'192.168.1.1/24\') -1 -ipv4_is_private(strcat(\'192.\',\'168.\',\'1.\',\'1\',\'/24\')) -1 --- ipv4_is_private(\'abc\') -\N --- ipv4_netmask_suffix(\'192.168.1.1/24\') -24 --- ipv4_netmask_suffix(\'192.168.1.1\') -32 --- ipv4_netmask_suffix(\'127.0.0.1/16\') -16 --- ipv4_netmask_suffix(\'abc\') -\N -ipv4_netmask_suffix(strcat(\'127.\', \'0.\', \'0.1/16\')) -16 --- ipv4_is_in_range(\'127.0.0.1\', \'127.0.0.1\') -1 --- ipv4_is_in_range(\'192.168.1.6\', \'192.168.1.1/24\') -1 --- ipv4_is_in_range(\'192.168.1.1\', \'192.168.2.1/24\') -0 --- ipv4_is_in_range(strcat(\'192.\',\'168.\', \'1.1\'), \'192.168.2.1/24\') -0 --- ipv4_is_in_range(\'abc\', \'127.0.0.1\') -\N --- parse_ipv6(127.0.0.1) -0000:0000:0000:0000:0000:ffff:7f00:0001 --- parse_ipv6(fe80::85d:e82c:9446:7994) -fe80:0000:0000:0000:085d:e82c:9446:7994 --- parse_ipv4(\'127.0.0.1\') -2130706433 --- parse_ipv4(\'192.1.168.1\') < parse_ipv4(\'192.1.168.2\') -1 --- parse_ipv4(arrayStringConcat([\'127\', \'0\', \'0\', \'1\'], \'.\')) --- parse_ipv4_mask(\'127.0.0.1\', 24) == 2130706432 -2130706432 --- parse_ipv4_mask(\'abc\', 31) -\N -\N --- parse_ipv4_mask(\'192.1.168.2\', 31) == parse_ipv4_mask(\'192.1.168.3\', 31) -3221334018 -3221334018 --- ipv4_is_match(\'127.0.0.1\', \'127.0.0.1\') -1 --- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\') -0 --- ipv4_is_match(\'192.168.1.1/24\', \'192.168.1.255/24\') -1 --- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\', 24) -1 --- ipv4_is_match(\'abc\', \'def\', 24) -\N --- ipv4_compare() -0 --1 -1 -0 -0 -0 -0 -0 -0 -0 -0 --- format_ipv4() -192.168.1.0 -192.168.1.1 -192.168.1.0 -192.168.1.0 -1 -1 -127.0.0.0 --- format_ipv4_mask() -192.168.1.0/24 -192.168.1.0/24 -192.168.1.0/24 -192.168.1.1/32 -192.168.1.0/24 -1 -1 -127.0.0.0/24 --- parse_ipv6_mask() -0000:0000:0000:0000:0000:0000:0000:0000 -fe80:0000:0000:0000:085d:e82c:9446:7900 -0000:0000:0000:0000:0000:ffff:c0a8:ff00 -0000:0000:0000:0000:0000:ffff:c0a8:ff00 -0000:0000:0000:0000:0000:ffff:ffff:ffff -fe80:0000:0000:0000:085d:e82c:9446:7994 -fe80:0000:0000:0000:085d:e82c:9446:7900 -0000:0000:0000:0000:0000:ffff:c0a8:ffff -0000:0000:0000:0000:0000:ffff:c0a8:ff00 --- ipv6_is_match() -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/02366_kql_func_ip.sql b/tests/queries/0_stateless/02366_kql_func_ip.sql deleted file mode 100644 index c9b335f203a..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_ip.sql +++ /dev/null @@ -1,131 +0,0 @@ -set dialect='kusto'; -print '-- ipv4_is_private(\'127.0.0.1\')'; -print ipv4_is_private('127.0.0.1'); -print '-- ipv4_is_private(\'10.1.2.3\')'; -print ipv4_is_private('10.1.2.3'); -print '-- ipv4_is_private(\'192.168.1.1/24\')'; -print ipv4_is_private('192.168.1.1/24'); -print 'ipv4_is_private(strcat(\'192.\',\'168.\',\'1.\',\'1\',\'/24\'))'; -print ipv4_is_private(strcat('192.','168.','1.','1','/24')); -print '-- ipv4_is_private(\'abc\')'; -print ipv4_is_private('abc'); -- == null - -print '-- ipv4_netmask_suffix(\'192.168.1.1/24\')'; -print ipv4_netmask_suffix('192.168.1.1/24'); -- == 24 -print '-- ipv4_netmask_suffix(\'192.168.1.1\')'; -print ipv4_netmask_suffix('192.168.1.1'); -- == 32 -print '-- ipv4_netmask_suffix(\'127.0.0.1/16\')'; -print ipv4_netmask_suffix('127.0.0.1/16'); -- == 16 -print '-- ipv4_netmask_suffix(\'abc\')'; -print ipv4_netmask_suffix('abc'); -- == null -print 'ipv4_netmask_suffix(strcat(\'127.\', \'0.\', \'0.1/16\'))'; -print ipv4_netmask_suffix(strcat('127.', '0.', '0.1/16')); -- == 16 - -print '-- ipv4_is_in_range(\'127.0.0.1\', \'127.0.0.1\')'; -print ipv4_is_in_range('127.0.0.1', '127.0.0.1'); -- == true -print '-- ipv4_is_in_range(\'192.168.1.6\', \'192.168.1.1/24\')'; -print ipv4_is_in_range('192.168.1.6', '192.168.1.1/24'); -- == true -print '-- ipv4_is_in_range(\'192.168.1.1\', \'192.168.2.1/24\')'; -print ipv4_is_in_range('192.168.1.1', '192.168.2.1/24'); -- == false -print '-- ipv4_is_in_range(strcat(\'192.\',\'168.\', \'1.1\'), \'192.168.2.1/24\')'; -print ipv4_is_in_range(strcat('192.','168.', '1.1'), '192.168.2.1/24'); -- == false -print '-- ipv4_is_in_range(\'abc\', \'127.0.0.1\')'; -- == null -print ipv4_is_in_range('abc', '127.0.0.1'); - -print '-- parse_ipv6(127.0.0.1)'; -print parse_ipv6('127.0.0.1'); -print '-- parse_ipv6(fe80::85d:e82c:9446:7994)'; -print parse_ipv6('fe80::85d:e82c:9446:7994'); -print '-- parse_ipv4(\'127.0.0.1\')'; -print parse_ipv4('127.0.0.1'); -print '-- parse_ipv4(\'192.1.168.1\') < parse_ipv4(\'192.1.168.2\')'; -print parse_ipv4('192.1.168.1') < parse_ipv4('192.1.168.2'); -print '-- parse_ipv4(arrayStringConcat([\'127\', \'0\', \'0\', \'1\'], \'.\'))'; -print parse_ipv4(arrayStringConcat(['127', '0', '0', '1'], '.')); -- { clientError UNKNOWN_FUNCTION } - -print '-- parse_ipv4_mask(\'127.0.0.1\', 24) == 2130706432'; -print parse_ipv4_mask('127.0.0.1', 24); -print '-- parse_ipv4_mask(\'abc\', 31)'; -print parse_ipv4_mask('abc', 31) -print '-- parse_ipv4_mask(\'192.1.168.2\', 1000)'; -print parse_ipv4_mask('192.1.168.2', 1000); -print '-- parse_ipv4_mask(\'192.1.168.2\', 31) == parse_ipv4_mask(\'192.1.168.3\', 31)'; ---print parse_ipv4_mask('192.1.168.2', 31) == parse_ipv4_mask('192.1.168.3', 31); // this qual failed in analyzer 3221334018 -print parse_ipv4_mask('192.1.168.2', 31); -print parse_ipv4_mask('192.1.168.3', 31); -print '-- ipv4_is_match(\'127.0.0.1\', \'127.0.0.1\')'; -print ipv4_is_match('127.0.0.1', '127.0.0.1'); -print '-- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\')'; -print ipv4_is_match('192.168.1.1', '192.168.1.255'); -print '-- ipv4_is_match(\'192.168.1.1/24\', \'192.168.1.255/24\')'; -print ipv4_is_match('192.168.1.1/24', '192.168.1.255/24'); -print '-- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\', 24)'; -print ipv4_is_match('192.168.1.1', '192.168.1.255', 24); -print '-- ipv4_is_match(\'abc\', \'def\', 24)'; -print ipv4_is_match('abc', 'dev', 24); -print '-- ipv4_compare()'; -print ipv4_compare('127.0.0.1', '127.0.0.1'); -print ipv4_compare('192.168.1.1', '192.168.1.255'); -print ipv4_compare('192.168.1.255', '192.168.1.1'); -print ipv4_compare('192.168.1.1/24', '192.168.1.255/24'); -print ipv4_compare('192.168.1.1', '192.168.1.255', 24); -print ipv4_compare('192.168.1.1/24', '192.168.1.255'); -print ipv4_compare('192.168.1.1', '192.168.1.255/24'); -print ipv4_compare('192.168.1.1/30', '192.168.1.255/24'); -print ipv4_compare('192.168.1.1', '192.168.1.0', 31); -print ipv4_compare('192.168.1.1/24', '192.168.1.255', 31); -print ipv4_compare('192.168.1.1', '192.168.1.255', 24); -print '-- format_ipv4()'; -print format_ipv4('192.168.1.255', 24); -print format_ipv4('192.168.1.1', 32); -print format_ipv4('192.168.1.1/24', 32); -print format_ipv4(3232236031, 24); -print format_ipv4('192.168.1.1/24', -1) == ''; -print format_ipv4('abc', 24) == ''; -print format_ipv4(strcat('127.0', '.0.', '1', '/32'), 12 + 12); -print '-- format_ipv4_mask()'; -print format_ipv4_mask('192.168.1.255', 24); -print format_ipv4_mask(3232236031, 24); -print format_ipv4_mask('192.168.1.1', 24); -print format_ipv4_mask('192.168.1.1', 32); -print format_ipv4_mask('192.168.1.1/24', 32); -print format_ipv4_mask('192.168.1.1/24', -1) == ''; -print format_ipv4_mask('abc', 24) == ''; -print format_ipv4_mask(strcat('127.0', '.0.', '1', '/32'), 12 + 12); -print '-- parse_ipv6_mask()'; -print parse_ipv6_mask("127.0.0.1", 24); -print parse_ipv6_mask("fe80::85d:e82c:9446:7994", 120); -print parse_ipv6_mask("192.168.255.255", 120); -print parse_ipv6_mask("192.168.255.255/24", 124); -print parse_ipv6_mask("255.255.255.255", 128); -print parse_ipv6_mask("fe80::85d:e82c:9446:7994", 128); -print parse_ipv6_mask("fe80::85d:e82c:9446:7994/120", 124); -print parse_ipv6_mask("::192.168.255.255", 128); -print parse_ipv6_mask("::192.168.255.255/24", 128); -print '-- ipv6_is_match()'; -print ipv6_is_match('::ffff:7f00:1', '127.0.0.1') == true; -print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995') == false; -print ipv6_is_match('192.168.1.1/24', '192.168.1.255/24') == true; -print ipv6_is_match('fe80::85d:e82c:9446:7994/127', 'fe80::85d:e82c:9446:7995/127') == true; -print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995', 127) == true; -print ipv6_is_match('192.168.1.1', '192.168.1.1'); -- // Equal IPs -print ipv6_is_match('192.168.1.1/24', '192.168.1.255'); -- // 24 bit IP4-prefix is used for comparison -print ipv6_is_match('192.168.1.1', '192.168.1.255/24'); -- // 24 bit IP4-prefix is used for comparison -print ipv6_is_match('192.168.1.1/30', '192.168.1.255/24'); -- // 24 bit IP4-prefix is used for comparison -print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7994'); -- // Equal IPs -print ipv6_is_match('fe80::85d:e82c:9446:7994/120', 'fe80::85d:e82c:9446:7998'); -- // 120 bit IP6-prefix is used for comparison -print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7998/120'); -- // 120 bit IP6-prefix is used for comparison -print ipv6_is_match('fe80::85d:e82c:9446:7994/120', 'fe80::85d:e82c:9446:7998/120'); -- // 120 bit IP6-prefix is used for comparison -print ipv6_is_match('192.168.1.1', '::ffff:c0a8:0101'); -- // Equal IPs -print ipv6_is_match('192.168.1.1/24', '::ffff:c0a8:01ff'); -- // 24 bit IP-prefix is used for comparison -print ipv6_is_match('::ffff:c0a8:0101', '192.168.1.255/24'); -- // 24 bit IP-prefix is used for comparison -print ipv6_is_match('::192.168.1.1/30', '192.168.1.255/24'); -- // 24 bit IP-prefix is used for comparison -print ipv6_is_match('192.168.1.1', '192.168.1.0', 31); -- // 31 bit IP4-prefix is used for comparison -print ipv6_is_match('192.168.1.1/24', '192.168.1.255', 31); -- // 24 bit IP4-prefix is used for comparison -print ipv6_is_match('192.168.1.1', '192.168.1.255', 24); -- // 24 bit IP4-prefix is used for comparison -print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995', 127); -- // 127 bit IP6-prefix is used for comparison -print ipv6_is_match('fe80::85d:e82c:9446:7994/127', 'fe80::85d:e82c:9446:7998', 120); -- // 120 bit IP6-prefix is used for comparison -print ipv6_is_match('fe80::85d:e82c:9446:7994/120', 'fe80::85d:e82c:9446:7998', 127); -- // 120 bit IP6-prefix is used for comparison -print ipv6_is_match('192.168.1.1/24', '::ffff:c0a8:01ff', 127); -- // 127 bit IP6-prefix is used for comparison -print ipv6_is_match('::ffff:c0a8:0101', '192.168.1.255', 120); -- // 120 bit IP6-prefix is used for comparison -print ipv6_is_match('::192.168.1.1/30', '192.168.1.255/24', 127); -- // 120 bit IP6-prefix is used for comparison \ No newline at end of file diff --git a/tests/queries/0_stateless/02366_kql_func_math.reference b/tests/queries/0_stateless/02366_kql_func_math.reference deleted file mode 100644 index 92f283abcb6..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_math.reference +++ /dev/null @@ -1,4 +0,0 @@ --- isnan -- -1 -0 -0 diff --git a/tests/queries/0_stateless/02366_kql_func_math.sql b/tests/queries/0_stateless/02366_kql_func_math.sql deleted file mode 100644 index 4e83622eb6b..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_math.sql +++ /dev/null @@ -1,7 +0,0 @@ -set dialect = 'kusto'; -print '-- isnan --'; -print isnan(double(nan)); -print isnan(4.2); -print isnan(4); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } -print isnan(real(+inf)); -print isnan(dynamic(null)); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } diff --git a/tests/queries/0_stateless/02366_kql_func_scalar.reference b/tests/queries/0_stateless/02366_kql_func_scalar.reference deleted file mode 100644 index b7fa62c5d43..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_scalar.reference +++ /dev/null @@ -1,16 +0,0 @@ --- bin_at() -4.5 --12:0:0 -2017-05-14 12:00:00.000000000 -2017-05-14 00:00:00.000000000 -2018-02-25 15:14:00.000000000 5 -2018-02-24 15:14:00.000000000 3 -2018-02-23 15:14:00.000000000 4 --- bin() -4 -1970-05-11 00:00:00.000000000 -336:0:0 -1970-05-11 13:45:07.345000000 -1970-05-11 13:45:07.345623000 -2022-09-26 10:13:23.987232000 -1970-05-11 13:45:07.456336000 diff --git a/tests/queries/0_stateless/02366_kql_func_scalar.sql b/tests/queries/0_stateless/02366_kql_func_scalar.sql deleted file mode 100644 index d7e94cfd9d1..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_scalar.sql +++ /dev/null @@ -1,26 +0,0 @@ -DROP TABLE IF EXISTS Bin_at_test; -CREATE TABLE Bin_at_test -( - `Date` DateTime('UTC'), - Num Nullable(UInt8) -) ENGINE = Memory; -INSERT INTO Bin_at_test VALUES ('2018-02-24T15:14:01',3), ('2018-02-23T16:14:01',4), ('2018-02-26T15:14:01',5); - -set dialect = 'kusto'; -print '-- bin_at()'; -print bin_at(6.5, 2.5, 7); -print bin_at(1h, 1d, 12h); -print bin_at(datetime(2017-05-15 10:20:00.0), 1d, datetime(1970-01-01 12:00:00.0)); -print bin_at(datetime(2017-05-17 10:20:00.0), 7d, datetime(2017-06-04 00:00:00.0)); -Bin_at_test | summarize sum(Num) by d = todatetime(bin_at(Date, 1d, datetime('2018-02-24 15:14:00'))) | order by d; -print '-- bin()'; -print bin(4.5, 1); -print bin(datetime(1970-05-11 13:45:07), 1d); -print bin(16d, 7d); -print bin(datetime(1970-05-11 13:45:07.345623), 1ms); --- print bin(datetime(2022-09-26 10:13:23.987234), 6ms); -> 2022-09-26 10:13:23.982000000 -print bin(datetime(1970-05-11 13:45:07.345623), 1microsecond); -print bin(datetime(2022-09-26 10:13:23.987234), 6microseconds); -print bin(datetime(1970-05-11 13:45:07.456345672), 16microseconds); --- print bin(datetime(2022-09-26 10:13:23.987234128), 1tick); -> 2022-09-26 10:13:23.987234100 --- print bin(datetime(2022-09-26 10:13:23.987234128), 99nanosecond); -> null diff --git a/tests/queries/0_stateless/02366_kql_func_string.reference b/tests/queries/0_stateless/02366_kql_func_string.reference deleted file mode 100644 index 9bdd38ca5db..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_string.reference +++ /dev/null @@ -1,360 +0,0 @@ --- test String Functions -- --- Customers |where Education contains \'degree\' -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 - --- Customers |where Education !contains \'degree\' -\N why Professional Partial College 38 -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers |where Education contains \'Degree\' -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 - --- Customers |where Education !contains \'Degree\' -\N why Professional Partial College 38 -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers | where FirstName endswith \'RE\' -Theodore Diaz Skilled Manual Bachelors 28 - --- Customers | where ! FirstName endswith \'RE\' -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - ---Customers | where FirstName endswith_cs \'re\' -Theodore Diaz Skilled Manual Bachelors 28 - --- Customers | where FirstName !endswith_cs \'re\' -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers | where Occupation == \'Skilled Manual\' -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Apple Skilled Manual Bachelors 28 - --- Customers | where Occupation != \'Skilled Manual\' -\N why Professional Partial College 38 -Latoya Shen Professional Graduate Degree 25 -Stephanie Cox Management abcd defg Bachelors 33 - --- Customers | where Occupation has \'skilled\' -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Apple Skilled Manual Bachelors 28 - --- Customers | where Occupation !has \'skilled\' -\N why Professional Partial College 38 -Latoya Shen Professional Graduate Degree 25 -Stephanie Cox Management abcd defg Bachelors 33 - --- Customers | where Occupation has \'Skilled\' -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Apple Skilled Manual Bachelors 28 - --- Customers | where Occupation !has \'Skilled\' -\N why Professional Partial College 38 -Latoya Shen Professional Graduate Degree 25 -Stephanie Cox Management abcd defg Bachelors 33 - --- Customers | where Occupation hasprefix_cs \'Ab\' - --- Customers | where Occupation !hasprefix_cs \'Ab\' -\N why Professional Partial College 38 -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers | where Occupation hasprefix_cs \'ab\' -Stephanie Cox Management abcd defg Bachelors 33 - --- Customers | where Occupation !hasprefix_cs \'ab\' -\N why Professional Partial College 38 -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Apple Skilled Manual Bachelors 28 - --- Customers | where Occupation hassuffix \'Ent\' -Stephanie Cox Management abcd defg Bachelors 33 - --- Customers | where Occupation !hassuffix \'Ent\' -\N why Professional Partial College 38 -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Apple Skilled Manual Bachelors 28 - --- Customers | where Occupation hassuffix \'ent\' -Stephanie Cox Management abcd defg Bachelors 33 - --- Customers | where Occupation hassuffix \'ent\' -Stephanie Cox Management abcd defg Bachelors 33 - --- Customers |where Education in (\'Bachelors\',\'High School\') -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers | where Education !in (\'Bachelors\',\'High School\') -\N why Professional Partial College 38 -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 - --- Customers | where FirstName matches regex \'P.*r\' -Peter Nara Skilled Manual Graduate Degree 26 - --- Customers | where FirstName startswith \'pet\' -Peter Nara Skilled Manual Graduate Degree 26 - --- Customers | where FirstName !startswith \'pet\' -Latoya Shen Professional Graduate Degree 25 -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers | where FirstName startswith_cs \'pet\' - --- Customers | where FirstName !startswith_cs \'pet\' -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers | where isempty(LastName) -Apple Skilled Manual Bachelors 28 - --- Customers | where isnotempty(LastName) -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Peter Nara Skilled Manual Graduate Degree 26 -Latoya Shen Professional Graduate Degree 25 -\N why Professional Partial College 38 - --- Customers | where isnotnull(FirstName) -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers | where isnull(FirstName) -\N why Professional Partial College 38 - --- Customers | project url_decode(\'https%3A%2F%2Fwww.test.com%2Fhello%20word\') | take 1 -https://www.test.com/hello word - --- Customers | project url_encode(\'https://www.test.com/hello word\') | take 1 -https%3A%2F%2Fwww.test.com%2Fhello%20word - --- Customers | project name_abbr = strcat(substring(FirstName,0,3), \' \', substring(LastName,2)) -\N -Lat en -Pet ra -The az -Ste x -App - --- Customers | project name = strcat(FirstName, \' \', LastName) -\N -Latoya Shen -Peter Nara -Theodore Diaz -Stephanie Cox -Apple - --- Customers | project FirstName, strlen(FirstName) -\N \N -Latoya 6 -Peter 5 -Theodore 8 -Stephanie 9 -Apple 5 - --- Customers | project strrep(FirstName,2,\'_\') -\N -Latoya_Latoya -Peter_Peter -Theodore_Theodore -Stephanie_Stephanie -Apple_Apple - --- Customers | project toupper(FirstName) -\N -LATOYA -PETER -THEODORE -STEPHANIE -APPLE - --- Customers | project tolower(FirstName) -\N -latoya -peter -theodore -stephanie -apple - --- support subquery for in orerator (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/in-cs-operator) (subquery need to be wraped with bracket inside bracket); TODO: case-insensitive not supported yet -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Apple Skilled Manual Bachelors 28 - --- has_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-all-operator); TODO: subquery not supported yet -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Apple Skilled Manual Bachelors 28 - --- has_any (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-anyoperator); TODO: subquery not supported yet -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Peter Nara Skilled Manual Graduate Degree 26 -Apple Skilled Manual Bachelors 28 - --- countof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/countoffunction) -3 -3 -1 - --- extract ( https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractfunction) -PINEAPPLE ice cream is 20 -PINEAPPLE -20 - -20 -\N -\N -\N -\N -\N -45.6 -45.6 - --- extract_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractallfunction); TODO: captureGroups not supported yet -[['T','h','e'],['p','ric','e'],['P','INEAPPL','E'],['i','c','e'],['c','rea','m']] - --- extract_json (https://learn.microsoft.com/en-us/azure/data-explorer/kusto/query/extractjsonfunction) - - -John -iPhone -\N -26 -26 -26 -26 -\N - --- split (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/splitfunction) -['aa','bb'] -['bbb'] -[''] -['a','','b'] -['aa','cc'] -['aabbcc'] -['aaa','bbb','ccc'] -[NULL] - --- strcat_delim (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/strcat-delimfunction); TODO: only support string now. -1-2-Ab - --- indexof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/indexoffunction); TODO: length and occurrence not supported yet -2 -2 --1 --- base64_encode_fromguid() -8jMxriJurkmwahbmqbIS6w== --- base64_decode_toarray() -[] -[75,117,115,116,111] --- base64_decode_toguid() -10e99626-bc2b-4c75-bb3e-fe606de25700 -1 --- base64_encode_tostring - -S3VzdG8x --- base64_decode_tostring - -Kusto1 --- parse_url() -{"Scheme":"scheme","Host":"","Port":"0","Path":"/this/is/a/path","Username":"username","Password":"password","Query Parameters":{"k1":"v1","k2":"v2"},"Fragment":"fragment"} --- parse_urlquery() -{"Query Parameters":{"k1":"v1","k2":"v2","k3":"v3"}} --- strcmp() -0 1 -1 1 --- substring() -CD --- translate() -kusto xxx --- trim() -https://www.ibm.com -Te st1 - asd -asd -sd --- trim_start() -www.ibm.com -Te st1// $ -asdw - -asd --- trim_end() -https -- Te st1 -wasd - -asd --- trim, trim_start, trim_end all at once ---https://bing.com-- -- https://bing.com-- --https://bing.com https://bing.com --- replace_regex -Number was: 1 --- has_any_index() -0 1 -1 -1 --- parse_version() -1000000020000000300000040 -1000000020000000000000000 -1000000020000000000000000 -\N -\N -\N -\N -1000000020000000300000004 -1000000020000000000000000 -1000000020000000300000000 -1000000000000000000000000 --- parse_json() -[1,2,3] -[{"a":123.5,"b":"{\\"c\\":456}"}] --- parse_command_line() -[NULL] -[NULL] --- reverse() -321 -43.321 - -dsa -][ -]3,2,1[ -]\'redaV\',\'htraD\'[ -000000000.00:00:21 51-01-7102 -Peter Nara Skilled Manual Graduate Degree 26 -Latoya Shen Professional Graduate Degree 25 --- parse_csv() -[''] -['aaa'] -['aa','b','cc'] -['record1','a','b','c'] diff --git a/tests/queries/0_stateless/02366_kql_func_string.sql b/tests/queries/0_stateless/02366_kql_func_string.sql deleted file mode 100644 index d251b04e08b..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_string.sql +++ /dev/null @@ -1,313 +0,0 @@ --- Tags: no-fasttest - -DROP TABLE IF EXISTS Customers; -CREATE TABLE Customers -( - FirstName Nullable(String), - LastName String, - Occupation String, - Education String, - Age Nullable(UInt8) -) ENGINE = Memory; - -INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28), ('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38); - --- datatable (Version:string) [ --- '1.2.3.4', --- '1.2', --- '1.2.3', --- '1' --- ] - -DROP TABLE IF EXISTS Versions; -CREATE TABLE Versions -( - Version String -) ENGINE = Memory; -INSERT INTO Versions VALUES ('1.2.3.4'),('1.2'),('1.2.3'),('1'); - - -set dialect='kusto'; -print '-- test String Functions --'; - -print '-- Customers |where Education contains \'degree\''; -Customers |where Education contains 'degree' | order by LastName; -print ''; -print '-- Customers |where Education !contains \'degree\''; -Customers |where Education !contains 'degree' | order by LastName; -print ''; -print '-- Customers |where Education contains \'Degree\''; -Customers |where Education contains 'Degree' | order by LastName; -print ''; -print '-- Customers |where Education !contains \'Degree\''; -Customers |where Education !contains 'Degree' | order by LastName; -print ''; -print '-- Customers | where FirstName endswith \'RE\''; -Customers | where FirstName endswith 'RE' | order by LastName; -print ''; -print '-- Customers | where ! FirstName endswith \'RE\''; -Customers | where FirstName ! endswith 'RE' | order by LastName; -print ''; -print '--Customers | where FirstName endswith_cs \'re\''; -Customers | where FirstName endswith_cs 're' | order by LastName; -print ''; -print '-- Customers | where FirstName !endswith_cs \'re\''; -Customers | where FirstName !endswith_cs 're' | order by LastName; -print ''; -print '-- Customers | where Occupation == \'Skilled Manual\''; -Customers | where Occupation == 'Skilled Manual' | order by LastName; -print ''; -print '-- Customers | where Occupation != \'Skilled Manual\''; -Customers | where Occupation != 'Skilled Manual' | order by LastName; -print ''; -print '-- Customers | where Occupation has \'skilled\''; -Customers | where Occupation has 'skilled' | order by LastName; -print ''; -print '-- Customers | where Occupation !has \'skilled\''; -Customers | where Occupation !has 'skilled' | order by LastName; -print ''; -print '-- Customers | where Occupation has \'Skilled\''; -Customers | where Occupation has 'Skilled'| order by LastName; -print ''; -print '-- Customers | where Occupation !has \'Skilled\''; -Customers | where Occupation !has 'Skilled'| order by LastName; -print ''; -print '-- Customers | where Occupation hasprefix_cs \'Ab\''; -Customers | where Occupation hasprefix_cs 'Ab'| order by LastName; -print ''; -print '-- Customers | where Occupation !hasprefix_cs \'Ab\''; -Customers | where Occupation !hasprefix_cs 'Ab'| order by LastName; -print ''; -print '-- Customers | where Occupation hasprefix_cs \'ab\''; -Customers | where Occupation hasprefix_cs 'ab'| order by LastName; -print ''; -print '-- Customers | where Occupation !hasprefix_cs \'ab\''; -Customers | where Occupation !hasprefix_cs 'ab'| order by LastName; -print ''; -print '-- Customers | where Occupation hassuffix \'Ent\''; -Customers | where Occupation hassuffix 'Ent'| order by LastName; -print ''; -print '-- Customers | where Occupation !hassuffix \'Ent\''; -Customers | where Occupation !hassuffix 'Ent'| order by LastName; -print ''; -print '-- Customers | where Occupation hassuffix \'ent\''; -Customers | where Occupation hassuffix 'ent'| order by LastName; -print ''; -print '-- Customers | where Occupation hassuffix \'ent\''; -Customers | where Occupation hassuffix 'ent'| order by LastName; -print ''; -print '-- Customers |where Education in (\'Bachelors\',\'High School\')'; -Customers |where Education in ('Bachelors','High School')| order by LastName; -print ''; -print '-- Customers | where Education !in (\'Bachelors\',\'High School\')'; -Customers | where Education !in ('Bachelors','High School')| order by LastName; -print ''; -print '-- Customers | where FirstName matches regex \'P.*r\''; -Customers | where FirstName matches regex 'P.*r'| order by LastName; -print ''; -print '-- Customers | where FirstName startswith \'pet\''; -Customers | where FirstName startswith 'pet'| order by LastName; -print ''; -print '-- Customers | where FirstName !startswith \'pet\''; -Customers | where FirstName !startswith 'pet'| order by LastName; -print ''; -print '-- Customers | where FirstName startswith_cs \'pet\''; -Customers | where FirstName startswith_cs 'pet'| order by LastName; -print ''; -print '-- Customers | where FirstName !startswith_cs \'pet\''; -Customers | where FirstName !startswith_cs 'pet'| order by LastName; -print ''; -print '-- Customers | where isempty(LastName)'; -Customers | where isempty(LastName); -print ''; -print '-- Customers | where isnotempty(LastName)'; -Customers | where isnotempty(LastName); -print ''; -print '-- Customers | where isnotnull(FirstName)'; -Customers | where isnotnull(FirstName)| order by LastName; -print ''; -print '-- Customers | where isnull(FirstName)'; -Customers | where isnull(FirstName)| order by LastName; -print ''; -print '-- Customers | project url_decode(\'https%3A%2F%2Fwww.test.com%2Fhello%20word\') | take 1'; -Customers | project url_decode('https%3A%2F%2Fwww.test.com%2Fhello%20word') | take 1; -print ''; -print '-- Customers | project url_encode(\'https://www.test.com/hello word\') | take 1'; -Customers | project url_encode('https://www.test.com/hello word') | take 1; -print ''; -print '-- Customers | project name_abbr = strcat(substring(FirstName,0,3), \' \', substring(LastName,2))'; -Customers | project name_abbr = strcat(substring(FirstName,0,3), ' ', substring(LastName,2))| order by LastName; -print ''; -print '-- Customers | project name = strcat(FirstName, \' \', LastName)'; -Customers | project name = strcat(FirstName, ' ', LastName)| order by LastName; -print ''; -print '-- Customers | project FirstName, strlen(FirstName)'; -Customers | project FirstName, strlen(FirstName)| order by LastName; -print ''; -print '-- Customers | project strrep(FirstName,2,\'_\')'; -Customers | project strrep(FirstName,2,'_')| order by LastName; -print ''; -print '-- Customers | project toupper(FirstName)'; -Customers | project toupper(FirstName)| order by LastName; -print ''; -print '-- Customers | project tolower(FirstName)'; -Customers | project tolower(FirstName)| order by LastName; -print ''; -print '-- support subquery for in orerator (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/in-cs-operator) (subquery need to be wraped with bracket inside bracket); TODO: case-insensitive not supported yet'; -Customers | where Age in ((Customers|project Age|where Age < 30)) | order by LastName; --- Customer | where LastName in~ ("diaz", "cox") -print ''; -print '-- has_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-all-operator); TODO: subquery not supported yet'; -Customers | where Occupation has_all ('manual', 'skilled') | order by LastName; -print ''; -print '-- has_any (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-anyoperator); TODO: subquery not supported yet'; -Customers|where Occupation has_any ('Skilled','abcd'); -print ''; -print '-- countof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/countoffunction)'; -Customers | project countof('The cat sat on the mat', 'at') | take 1; -Customers | project countof('The cat sat on the mat', 'at', 'normal') | take 1; -Customers | project countof('The cat sat on the mat', '\\s.he', 'regex') | take 1; -print ''; -print '-- extract ( https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractfunction)'; -print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 0, 'The price of PINEAPPLE ice cream is 20'); -print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 1, 'The price of PINEAPPLE ice cream is 20'); -print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 20'); -print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 3, 'The price of PINEAPPLE ice cream is 20'); -print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 20', typeof(real)); -print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(bool)); -print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(date)); -print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(guid)); -print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(int)); -print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(long)); -print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(real)); -print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(decimal)); -print ''; -print '-- extract_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractallfunction); TODO: captureGroups not supported yet'; -Customers | project extract_all('(\\w)(\\w+)(\\w)','The price of PINEAPPLE ice cream is 20') | take 1; -print ''; -print '-- extract_json (https://learn.microsoft.com/en-us/azure/data-explorer/kusto/query/extractjsonfunction)'; -print extract_json('', ''); -- { serverError BAD_ARGUMENTS } -print extract_json('a', ''); -- { serverError BAD_ARGUMENTS } -print extract_json('$.firstName', ''); -print extract_json('$.phoneNumbers[0].type', ''); -print extractjson('$.firstName', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}'); -print extract_json('$.phoneNumbers[0].type', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(string)); -print extract_json('$.phoneNumbers[0].type', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(int)); -print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}'); -print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(int)); -print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(long)); --- print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(bool)); -> true -print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(double)); -print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(guid)); --- print extract_json('$.phoneNumbers', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(dynamic)); we won't be able to handle this particular case for a while, because it should return a dictionary -print ''; -print '-- split (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/splitfunction)'; -Customers | project split('aa_bb', '_') | take 1; -Customers | project split('aaa_bbb_ccc', '_', 1) | take 1; -Customers | project split('', '_') | take 1; -Customers | project split('a__b', '_') | take 1; -Customers | project split('aabbcc', 'bb') | take 1; -Customers | project split('aabbcc', '') | take 1; -Customers | project split('aaa_bbb_ccc', '_', -1) | take 1; -Customers | project split('aaa_bbb_ccc', '_', 10) | take 1; -print ''; -print '-- strcat_delim (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/strcat-delimfunction); TODO: only support string now.'; -Customers | project strcat_delim('-', '1', '2', strcat('A','b')) | take 1; --- Customers | project strcat_delim('-', '1', '2', 'A' , 1s); -print ''; -print '-- indexof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/indexoffunction); TODO: length and occurrence not supported yet'; -Customers | project indexof('abcdefg','cde') | take 1; -Customers | project indexof('abcdefg','cde',2) | take 1; -Customers | project indexof('abcdefg','cde',6) | take 1; -print '-- base64_encode_fromguid()'; --- print base64_encode_fromguid(guid(null)); -print base64_encode_fromguid(guid('ae3133f2-6e22-49ae-b06a-16e6a9b212eb')); -print base64_encode_fromguid(dynamic(null)); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } -print base64_encode_fromguid("abcd1231"); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } -print '-- base64_decode_toarray()'; -print base64_decode_toarray(''); -print base64_decode_toarray('S3VzdG8='); -print '-- base64_decode_toguid()'; -print base64_decode_toguid("JpbpECu8dUy7Pv5gbeJXAA=="); -print base64_decode_toguid(base64_encode_fromguid(guid('ae3133f2-6e22-49ae-b06a-16e6a9b212eb'))) == guid('ae3133f2-6e22-49ae-b06a-16e6a9b212eb'); -print '-- base64_encode_tostring'; -print base64_encode_tostring(''); -print base64_encode_tostring('Kusto1'); -print '-- base64_decode_tostring'; -print base64_decode_tostring(''); -print base64_decode_tostring('S3VzdG8x'); -print '-- parse_url()'; -print parse_url('scheme://username:password@host:1234/this/is/a/path?k1=v1&k2=v2#fragment'); -print '-- parse_urlquery()'; -print parse_urlquery('k1=v1&k2=v2&k3=v3'); -print '-- strcmp()'; -print strcmp('ABC','ABC'), strcmp('abc','ABC'), strcmp('ABC','abc'), strcmp('abcde','abc'); -print '-- substring()'; -print substring("ABCD", -2, 2); -print '-- translate()'; -print translate('krasp', 'otsku', 'spark'), translate('abc', '', 'ab'), translate('abc', 'x', 'abc'); -print '-- trim()'; -print trim("--", "--https://www.ibm.com--"); -print trim("[^\w]+", strcat("- ","Te st", "1", "// $")); -print trim("", " asd "); -print trim("a$", "asd"); -print trim("^a", "asd"); -print '-- trim_start()'; -print trim_start("https://", "https://www.ibm.com"); -print trim_start("[^\w]+", strcat("- ","Te st", "1", "// $")); -print trim_start("asd$", "asdw"); -print trim_start("asd$", "asd"); -print trim_start("d$", "asd"); -print '-- trim_end()'; -print trim_end("://www.ibm.com", "https://www.ibm.com"); -print trim_end("[^\w]+", strcat("- ","Te st", "1", "// $")); -print trim_end("^asd", "wasd"); -print trim_end("^asd", "asd"); -print trim_end("^a", "asd"); -print '-- trim, trim_start, trim_end all at once'; -print str = "--https://bing.com--", pattern = '--' | extend start = trim_start(pattern, str), end = trim_end(pattern, str), both = trim(pattern, str); -print '-- replace_regex'; -print replace_regex(strcat('Number is ', '1'), 'is (\d+)', 'was: \1'); -print '-- has_any_index()'; -print has_any_index('this is an example', dynamic(['this', 'example'])), has_any_index("this is an example", dynamic(['not', 'example'])), has_any_index("this is an example", dynamic(['not', 'found'])), has_any_index("this is an example", dynamic([])); -print '-- parse_version()'; -print parse_version(42); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } --- print parse_version(''); -> NULL -print parse_version('1.2.3.40'); -print parse_version('1.2'); -print parse_version(strcat('1.', '2')); -print parse_version('1.2.4.5.6'); -print parse_version('moo'); -print parse_version('moo.boo.foo'); -print parse_version(strcat_delim('.', 'moo', 'boo', 'foo')); -Versions | project parse_version(Version); -print '-- parse_json()'; -print parse_json(dynamic([1, 2, 3])); -print parse_json('{"a":123.5, "b":"{\\"c\\":456}"}'); -print '-- parse_command_line()'; -print parse_command_line(55, 'windows'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } --- print parse_command_line((52 + 3) * 4 % 2, 'windows'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -print parse_command_line('', 'windows'); -print parse_command_line(strrep(' ', 6), 'windows'); --- print parse_command_line('echo \"hello world!\" print$?', 'windows'); -> ["echo","hello world!","print$?"] --- print parse_command_line("yolo swag 'asd bcd' \"moo moo \"", 'windows'); -> ["yolo","swag","'asd","bcd'","moo moo "] --- print parse_command_line(strcat_delim(' ', "yolo", "swag", "\'asd bcd\'", "\"moo moo \""), 'windows'); -> ["yolo","swag","'asd","bcd'","moo moo "] -print '-- reverse()'; -print reverse(123); -print reverse(123.34); -print reverse(''); -print reverse("asd"); -print reverse(dynamic([])); -print reverse(dynamic([1, 2, 3])); -print reverse(dynamic(['Darth', "Vader"])); -print reverse(datetime(2017-10-15 12:00)); --- print reverse(timespan(3h)); -> 00:00:30 -Customers | where Education contains 'degree' | order by reverse(FirstName); -print '-- parse_csv()'; -print parse_csv(''); -print parse_csv(65); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -print parse_csv('aaa'); -print result=parse_csv('aa,b,cc'); -print result_multi_record=parse_csv('record1,a,b,c\nrecord2,x,y,z'); --- print result=parse_csv('aa,"b,b,b",cc,"Escaping quotes: ""Title""","line1\nline2"'); -> ["aa","b,b,b","cc","Escaping quotes: \"Title\"","line1\nline2"] --- print parse_csv(strcat(strcat_delim(',', 'aa', '"b,b,b"', 'cc', '"Escaping quotes: ""Title"""', '"line1\nline2"'), '\r\n', strcat_delim(',', 'asd', 'qcf'))); -> ["aa","b,b,b","cc","Escaping quotes: \"Title\"","line1\nline2"] diff --git a/tests/queries/0_stateless/02366_kql_makeseries.reference b/tests/queries/0_stateless/02366_kql_makeseries.reference deleted file mode 100644 index 8e7fde997bf..00000000000 --- a/tests/queries/0_stateless/02366_kql_makeseries.reference +++ /dev/null @@ -1,60 +0,0 @@ --- from to -Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102] -Costco Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [0,2,0] -Aldi Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [0,500,0] -Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6] --- from -Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102] -Costco Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000'] [0,2] -Aldi Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000'] [0,500] -Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6] --- to -Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102] -Costco Apple ['2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [2,0] -Aldi Snargaluff ['2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [500,0] -Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6] --- without from/to -Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102] -Costco Apple ['2016-09-11 00:00:00.000000000'] [2] -Aldi Snargaluff ['2016-09-11 00:00:00.000000000'] [500] -Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6] --- without by -['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [70,334,54] --- without aggregation alias -Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102] -Aldi Snargaluff ['2016-09-11 00:00:00.000000000'] [500] -Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6] -Costco Apple ['2016-09-11 00:00:00.000000000'] [2] --- assign group alias -Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102] -Aldi Snargaluff ['2016-09-11 00:00:00.000000000'] [500] -Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6] -Costco Apple ['2016-09-11 00:00:00.000000000'] [2] --- 3d step -Costco Snargaluff ['2016-09-10 00:00:00.000000000'] [134.66666666666666] -Costco Apple ['2016-09-10 00:00:00.000000000'] [2] -Aldi Snargaluff ['2016-09-10 00:00:00.000000000'] [500] -Aldi Apple ['2016-09-10 00:00:00.000000000'] [5.5] --- numeric column -Costco Snargaluff [10,11,12,13,14] [200,0,102,0,0] -Aldi Snargaluff [10,11,12,13,14] [0,500,0,0,0] -Aldi Apple [10,11,12,13,14] [5,0,6,0,0] -Costco Apple [10,11,12,13,14] [0,2,0,0,0] --- from -Costco Snargaluff [10,11,12] [200,0,102] -Aldi Snargaluff [10,11] [0,500] -Aldi Apple [10,11,12] [5,0,6] -Costco Apple [10,11] [0,2] --- to -Costco Snargaluff [8,12,16] [200,102,0] -Aldi Snargaluff [8,12,16] [500,0,0] -Aldi Apple [8,12,16] [5,6,0] -Costco Apple [8,12,16] [2,0,0] --- without from/to -Costco Snargaluff [10,12] [200,102] -Aldi Snargaluff [10] [500] -Aldi Apple [10,12] [5,6] -Costco Apple [10] [2] --- without by -[10,12] [202,54] -['2017-01-01 00:00:00.000000000','2017-01-02 00:00:00.000000000','2017-01-03 00:00:00.000000000','2017-01-04 00:00:00.000000000','2017-01-05 00:00:00.000000000','2017-01-06 00:00:00.000000000','2017-01-07 00:00:00.000000000','2017-01-08 00:00:00.000000000','2017-01-09 00:00:00.000000000'] [4,3,5,0,10.5,4,3,8,6.5] diff --git a/tests/queries/0_stateless/02366_kql_makeseries.sql b/tests/queries/0_stateless/02366_kql_makeseries.sql deleted file mode 100644 index c9ca91c0be0..00000000000 --- a/tests/queries/0_stateless/02366_kql_makeseries.sql +++ /dev/null @@ -1,77 +0,0 @@ --- Azure Data Explore Test Data --- let make_series_test_table = datatable (Supplier:string, Fruit:string, Price: real, Purchase:datetime) --- [ --- 'Aldi','Apple',4,'2016-09-10', --- 'Costco','Apple',2,'2016-09-11', --- 'Aldi','Apple',6,'2016-09-10', --- 'Costco','Snargaluff',100,'2016-09-12', --- 'Aldi','Apple',7,'2016-09-12', --- 'Aldi','Snargaluff',400,'2016-09-11', --- 'Costco','Snargaluff',104,'2016-09-12', --- 'Aldi','Apple',5,'2016-09-12', --- 'Aldi','Snargaluff',600,'2016-09-11', --- 'Costco','Snargaluff',200,'2016-09-10', --- ]; -DROP TABLE IF EXISTS make_series_test_table; -CREATE TABLE make_series_test_table -( - Supplier Nullable(String), - Fruit String , - Price Float64, - Purchase Date -) ENGINE = Memory; -INSERT INTO make_series_test_table VALUES ('Aldi','Apple',4,'2016-09-10'), ('Costco','Apple',2,'2016-09-11'), ('Aldi','Apple',6,'2016-09-10'), ('Costco','Snargaluff',100,'2016-09-12'), ('Aldi','Apple',7,'2016-09-12'), ('Aldi','Snargaluff',400,'2016-09-11'),('Costco','Snargaluff',104,'2016-09-12'),('Aldi','Apple',5,'2016-09-12'),('Aldi','Snargaluff',600,'2016-09-11'),('Costco','Snargaluff',200,'2016-09-10'); -DROP TABLE IF EXISTS make_series_test_table2; -CREATE TABLE make_series_test_table2 -( - Supplier Nullable(String), - Fruit String , - Price Int32, - Purchase Int32 -) ENGINE = Memory; -INSERT INTO make_series_test_table2 VALUES ('Aldi','Apple',4,10),('Costco','Apple',2,11),('Aldi','Apple',6,10),('Costco','Snargaluff',100,12),('Aldi','Apple',7,12),('Aldi','Snargaluff',400,11),('Costco','Snargaluff',104,12),('Aldi','Apple',5,12),('Aldi','Snargaluff',600,11),('Costco','Snargaluff',200,10); -DROP TABLE IF EXISTS make_series_test_table3; -CREATE TABLE make_series_test_table3 -( - timestamp datetime, - metric Float64, -) ENGINE = Memory; -INSERT INTO make_series_test_table3 VALUES (parseDateTimeBestEffort('2016-12-31T06:00', 'UTC'), 50), (parseDateTimeBestEffort('2017-01-01', 'UTC'), 4), (parseDateTimeBestEffort('2017-01-02', 'UTC'), 3), (parseDateTimeBestEffort('2017-01-03', 'UTC'), 4), (parseDateTimeBestEffort('2017-01-03T03:00', 'UTC'), 6), (parseDateTimeBestEffort('2017-01-05', 'UTC'), 8), (parseDateTimeBestEffort('2017-01-05T13:40', 'UTC'), 13), (parseDateTimeBestEffort('2017-01-06', 'UTC'), 4), (parseDateTimeBestEffort('2017-01-07', 'UTC'), 3), (parseDateTimeBestEffort('2017-01-08', 'UTC'), 8), (parseDateTimeBestEffort('2017-01-08T21:00', 'UTC'), 8), (parseDateTimeBestEffort('2017-01-09', 'UTC'), 2), (parseDateTimeBestEffort('2017-01-09T12:00', 'UTC'), 11), (parseDateTimeBestEffort('2017-01-10T05:00', 'UTC'), 5); - --- This test requies sorting after some of aggregations but I don't know KQL, sorry -set max_bytes_before_external_group_by = 0; -set dialect = 'kusto'; - -print '-- from to'; -make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) to datetime(2016-09-13) step 1d by Supplier, Fruit | order by Supplier, Fruit; -print '-- from'; -make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) step 1d by Supplier, Fruit | order by Supplier, Fruit; -print '-- to'; -make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase to datetime(2016-09-13) step 1d by Supplier, Fruit | order by Supplier, Fruit; -print '-- without from/to'; -make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase step 1d by Supplier, Fruit | order by Supplier, Fruit; -print '-- without by'; -make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase step 1d; -print '-- without aggregation alias'; -make_series_test_table | make-series avg(Price) default=0 on Purchase step 1d by Supplier, Fruit; -print '-- assign group alias'; -make_series_test_table | make-series avg(Price) default=0 on Purchase step 1d by Supplier_Name = Supplier, Fruit; -print '-- 3d step'; -make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) to datetime(2016-09-13) step 3d by Supplier, Fruit | order by Supplier, Fruit; - -print '-- numeric column' -print '-- from to'; -make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase from 10 to 15 step 1.0 by Supplier, Fruit; -print '-- from'; -make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase from 10 step 1.0 by Supplier, Fruit; -print '-- to'; -make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase to 18 step 4.0 by Supplier, Fruit; -print '-- without from/to'; -make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase step 2.0 by Supplier, Fruit; -print '-- without by'; -make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase step 2.0; - -make_series_test_table3 | make-series avg(metric) default=0 on timestamp from datetime(2017-01-01) to datetime(2017-01-10) step 1d - --- print '-- summarize --' --- make_series_test_table | summarize count() by format_datetime(bin(Purchase, 1d), 'yy-MM-dd'); diff --git a/tests/queries/0_stateless/02366_kql_mvexpand.reference b/tests/queries/0_stateless/02366_kql_mvexpand.reference deleted file mode 100644 index 25be070eb0b..00000000000 --- a/tests/queries/0_stateless/02366_kql_mvexpand.reference +++ /dev/null @@ -1,65 +0,0 @@ --- mv-expand -- --- mv_expand_test_table | mv-expand c -- -1 ['Salmon','Steak','Chicken'] 1 [5,6,7,8] -1 ['Salmon','Steak','Chicken'] 2 [5,6,7,8] -1 ['Salmon','Steak','Chicken'] 3 [5,6,7,8] -1 ['Salmon','Steak','Chicken'] 4 [5,6,7,8] --- mv_expand_test_table | mv-expand c, d -- -1 ['Salmon','Steak','Chicken'] 1 5 -1 ['Salmon','Steak','Chicken'] 2 6 -1 ['Salmon','Steak','Chicken'] 3 7 -1 ['Salmon','Steak','Chicken'] 4 8 --- mv_expand_test_table | mv-expand b | mv-expand c -- -1 Salmon 1 [5,6,7,8] -1 Salmon 2 [5,6,7,8] -1 Salmon 3 [5,6,7,8] -1 Salmon 4 [5,6,7,8] -1 Steak 1 [5,6,7,8] -1 Steak 2 [5,6,7,8] -1 Steak 3 [5,6,7,8] -1 Steak 4 [5,6,7,8] -1 Chicken 1 [5,6,7,8] -1 Chicken 2 [5,6,7,8] -1 Chicken 3 [5,6,7,8] -1 Chicken 4 [5,6,7,8] --- mv_expand_test_table | mv-expand with_itemindex=index b, c, d -- -0 1 Salmon 1 5 -1 1 Steak 2 6 -2 1 Chicken 3 7 -3 1 4 8 --- mv_expand_test_table | mv-expand array_concat(c,d) -- -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 1 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 2 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 3 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 4 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 5 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 6 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 7 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 8 --- mv_expand_test_table | mv-expand x = c, y = d -- -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 1 5 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 2 6 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 3 7 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 4 8 --- mv_expand_test_table | mv-expand xy = array_concat(c, d) -- -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 1 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 2 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 3 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 4 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 5 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 6 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 7 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 8 --- mv_expand_test_table | mv-expand xy = array_concat(c, d) limit 2| summarize count() by xy -- -1 1 -2 1 --- mv_expand_test_table | mv-expand with_itemindex=index c,d to typeof(bool) -- -0 1 ['Salmon','Steak','Chicken'] 1 true -1 1 ['Salmon','Steak','Chicken'] 2 true -2 1 ['Salmon','Steak','Chicken'] 3 true -3 1 ['Salmon','Steak','Chicken'] 4 true --- mv_expand_test_table | mv-expand c to typeof(bool) -- -1 ['Salmon','Steak','Chicken'] [5,6,7,8] true -1 ['Salmon','Steak','Chicken'] [5,6,7,8] true -1 ['Salmon','Steak','Chicken'] [5,6,7,8] true -1 ['Salmon','Steak','Chicken'] [5,6,7,8] true diff --git a/tests/queries/0_stateless/02366_kql_mvexpand.sql b/tests/queries/0_stateless/02366_kql_mvexpand.sql deleted file mode 100644 index e7798609646..00000000000 --- a/tests/queries/0_stateless/02366_kql_mvexpand.sql +++ /dev/null @@ -1,35 +0,0 @@ --- datatable(a: int, b: dynamic, c: dynamic, d: dynamic) [ --- 1, dynamic(['Salmon', 'Steak', 'Chicken']), dynamic([1, 2, 3, 4]), dynamic([5, 6, 7, 8]) --- ] - -DROP TABLE IF EXISTS mv_expand_test_table; -CREATE TABLE mv_expand_test_table -( - a UInt8, - b Array(String), - c Array(Int8), - d Array(Int8) -) ENGINE = Memory; -INSERT INTO mv_expand_test_table VALUES (1, ['Salmon', 'Steak','Chicken'],[1,2,3,4],[5,6,7,8]); -set dialect='kusto'; -print '-- mv-expand --'; -print '-- mv_expand_test_table | mv-expand c --'; -mv_expand_test_table | mv-expand c; -print '-- mv_expand_test_table | mv-expand c, d --'; -mv_expand_test_table | mv-expand c, d; -print '-- mv_expand_test_table | mv-expand b | mv-expand c --'; -mv_expand_test_table | mv-expand b | mv-expand c; -print '-- mv_expand_test_table | mv-expand with_itemindex=index b, c, d --'; -mv_expand_test_table | mv-expand with_itemindex=index b, c, d; -print '-- mv_expand_test_table | mv-expand array_concat(c,d) --'; -mv_expand_test_table | mv-expand array_concat(c,d); -print '-- mv_expand_test_table | mv-expand x = c, y = d --'; -mv_expand_test_table | mv-expand x = c, y = d; -print '-- mv_expand_test_table | mv-expand xy = array_concat(c, d) --'; -mv_expand_test_table | mv-expand xy = array_concat(c, d); -print '-- mv_expand_test_table | mv-expand xy = array_concat(c, d) limit 2| summarize count() by xy --'; -mv_expand_test_table | mv-expand xy = array_concat(c, d) limit 2| summarize count() by xy; -print '-- mv_expand_test_table | mv-expand with_itemindex=index c,d to typeof(bool) --'; -mv_expand_test_table | mv-expand with_itemindex=index c,d to typeof(bool); -print '-- mv_expand_test_table | mv-expand c to typeof(bool) --'; -mv_expand_test_table | mv-expand c to typeof(bool); diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference deleted file mode 100644 index 8a12c6885c4..00000000000 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.reference +++ /dev/null @@ -1,23 +0,0 @@ -numeric -kusto -00:00:00 -00:00:00.0000001 -00:00:00.0010000 -00:00:42 -01:06:00 -2.18:00:00 -5.00:00:00 -7.00:00:00 -14.00:00:00 -('00:01:12','21.00:00:00','00:00:00.0000002') -numeric -99 -100 -1 -42 -66 -66 -5 -1 -2 -(72,3,200) diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 deleted file mode 100644 index 0731687222d..00000000000 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 +++ /dev/null @@ -1,16 +0,0 @@ -select value from system.settings where name = 'interval_output_format'; - -{% for format in ['kusto', 'numeric'] -%} -select '{{ format }}'; -set interval_output_format = '{{ format }}'; -select toIntervalNanosecond(99); -select toIntervalNanosecond(100); -select toIntervalMillisecond(1); -select toIntervalSecond(42); -select toIntervalMinute(66); -select toIntervalHour(66); -select toIntervalDay(5); -select toIntervalWeek(1); -select toIntervalWeek(2); -select toIntervalSecond(72) + toIntervalWeek(3) + toIntervalNanosecond(200); -{% endfor -%} diff --git a/tests/queries/0_stateless/02366_kql_operator_in_sql.reference b/tests/queries/0_stateless/02366_kql_operator_in_sql.reference deleted file mode 100644 index 4e0987aa5c3..00000000000 --- a/tests/queries/0_stateless/02366_kql_operator_in_sql.reference +++ /dev/null @@ -1,60 +0,0 @@ --- #1 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 --- #2 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Peter Nara Skilled Manual Graduate Degree 26 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #3 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #4 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #5 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #6 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #7 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #8 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #9 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Peter Nara Skilled Manual Graduate Degree 26 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #10 -- --- #11 -- --- #12 -- --- #13 -- --- #14 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #15 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 diff --git a/tests/queries/0_stateless/02366_kql_operator_in_sql.sql b/tests/queries/0_stateless/02366_kql_operator_in_sql.sql deleted file mode 100644 index 0b02faa0680..00000000000 --- a/tests/queries/0_stateless/02366_kql_operator_in_sql.sql +++ /dev/null @@ -1,42 +0,0 @@ -DROP TABLE IF EXISTS Customers; -CREATE TABLE Customers -( - FirstName Nullable(String), - LastName String, - Occupation String, - Education String, - Age Nullable(UInt8) -) ENGINE = Memory; - -INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28),('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38); -Select '-- #1 --' ; -select * from kql($$Customers | where FirstName !in ('Peter', 'Latoya')$$); -Select '-- #2 --' ; -select * from kql($$Customers | where FirstName !in ("test", "test2")$$); -Select '-- #3 --' ; -select * from kql($$Customers | where FirstName !contains 'Pet'$$); -Select '-- #4 --' ; -select * from kql($$Customers | where FirstName !contains_cs 'Pet'$$); -Select '-- #5 --' ; -select * from kql($$Customers | where FirstName !endswith 'ter'$$); -Select '-- #6 --' ; -select * from kql($$Customers | where FirstName !endswith_cs 'ter'$$); -Select '-- #7 --' ; -select * from kql($$Customers | where FirstName != 'Peter'$$); -Select '-- #8 --' ; -select * from kql($$Customers | where FirstName !has 'Peter'$$); -Select '-- #9 --' ; -select * from kql($$Customers | where FirstName !has_cs 'peter'$$); -Select '-- #10 --' ; --- select * from kql($$Customers | where FirstName !hasprefix 'Peter'$$); -- will enable when analyzer fixed `and` issue -Select '-- #11 --' ; ---select * from kql($$Customers | where FirstName !hasprefix_cs 'Peter'$$); -Select '-- #12 --' ; ---select * from kql($$Customers | where FirstName !hassuffix 'Peter'$$); -Select '-- #13 --' ; ---select * from kql($$Customers | where FirstName !hassuffix_cs 'Peter'$$); -Select '-- #14 --' ; -select * from kql($$Customers | where FirstName !startswith 'Peter'$$); -Select '-- #15 --' ; -select * from kql($$Customers | where FirstName !startswith_cs 'Peter'$$); -DROP TABLE IF EXISTS Customers; diff --git a/tests/queries/0_stateless/02366_kql_summarize.reference b/tests/queries/0_stateless/02366_kql_summarize.reference deleted file mode 100644 index aeb42feb6be..00000000000 --- a/tests/queries/0_stateless/02366_kql_summarize.reference +++ /dev/null @@ -1,92 +0,0 @@ --- test summarize -- -12 25 46 32.416666666666664 389 -Skilled Manual 5 26 36 30.2 151 -Professional 6 25 46 34.166666666666664 205 -Management abcd defg 1 33 33 33 33 -Skilled Manual 0 -Professional 2 -Management abcd defg 0 -Skilled Manual 36 -Professional 38 -Management abcd defg 33 -Skilled Manual 26 -Professional 25 -Management abcd defg 33 -Skilled Manual 30.2 -Professional 29.25 -Management abcd defg 33 -Skilled Manual 151 -Professional 117 -Management abcd defg 33 -4 -2 -40 2 -30 4 -20 6 -Skilled Manual 5 -Professional 6 -Management abcd defg 1 --- make_list() -- -Skilled Manual ['Bachelors','Graduate Degree','High School','Partial College','Bachelors'] -Professional ['Graduate Degree','Partial College','Partial College','Partial College','Partial College','Partial College'] -Management abcd defg ['Bachelors'] -Skilled Manual ['Bachelors','Graduate Degree'] -Professional ['Graduate Degree','Partial College'] -Management abcd defg ['Bachelors'] --- make_list_if() -- -Skilled Manual ['Edward','Christine'] -Professional ['Dalton','Angel'] -Management abcd defg ['Stephanie'] -Skilled Manual ['Edward'] -Professional ['Dalton'] -Management abcd defg ['Stephanie'] --- make_set() -- -Skilled Manual ['Graduate Degree','High School','Partial College','Bachelors'] -Professional ['Graduate Degree','Partial College'] -Management abcd defg ['Bachelors'] -Skilled Manual ['Graduate Degree','Bachelors'] -Professional ['Graduate Degree','Partial College'] -Management abcd defg ['Bachelors'] --- make_set_if() -- -Skilled Manual ['Partial College','High School'] -Professional ['Partial College'] -Management abcd defg ['Bachelors'] -Skilled Manual ['High School'] -Professional ['Partial College'] -Management abcd defg ['Bachelors'] --- stdev() -- -6.855102059227432 --- stdevif() -- -7.557189365836421 --- binary_all_and -- -42 --- binary_all_or -- -46 --- binary_all_xor -- -4 -43.8 -25.55 30.5 43.8 -30.5 -35 -[25,35,45] --- Summarize following sort -- -Skilled Manual 5 -Professional 6 -Management abcd defg 1 --- summarize with bin -- -0 1 -245000 2 -0 1 -245 2 -0 1 -245 2 -2015-10-12 00:00:00.000000000 -2016-10-12 00:00:00.000000000 --- make_list_with_nulls -- -['Theodore','Stephanie','Peter','Latoya','Joshua','Edward','Dalton','Christine','Cameron','Angel','Apple',NULL] -Skilled Manual ['Theodore','Peter','Edward','Christine','Apple'] -Professional ['Latoya','Joshua','Dalton','Cameron','Angel',NULL] -Management abcd defg ['Stephanie'] -Skilled Manual ['Theodore','Peter','Edward','Christine','Apple'] [28,26,36,33,28] -Professional ['Latoya','Joshua','Dalton','Cameron','Angel',NULL] [25,26,42,28,46,38] -Management abcd defg ['Stephanie'] [33] diff --git a/tests/queries/0_stateless/02366_kql_summarize.sql b/tests/queries/0_stateless/02366_kql_summarize.sql deleted file mode 100644 index bb12d1f251f..00000000000 --- a/tests/queries/0_stateless/02366_kql_summarize.sql +++ /dev/null @@ -1,102 +0,0 @@ --- datatable(FirstName:string, LastName:string, Occupation:string, Education:string, Age:int) [ --- 'Theodore', 'Diaz', 'Skilled Manual', 'Bachelors', 28, --- 'Stephanie', 'Cox', 'Management abcd defg', 'Bachelors', 33, --- 'Peter', 'Nara', 'Skilled Manual', 'Graduate Degree', 26, --- 'Latoya', 'Shen', 'Professional', 'Graduate Degree', 25, --- 'Joshua', 'Lee', 'Professional', 'Partial College', 26, --- 'Edward', 'Hernandez', 'Skilled Manual', 'High School', 36, --- 'Dalton', 'Wood', 'Professional', 'Partial College', 42, --- 'Christine', 'Nara', 'Skilled Manual', 'Partial College', 33, --- 'Cameron', 'Rodriguez', 'Professional', 'Partial College', 28, --- 'Angel', 'Stewart', 'Professional', 'Partial College', 46, --- 'Apple', '', 'Skilled Manual', 'Bachelors', 28, --- dynamic(null), 'why', 'Professional', 'Partial College', 38 --- ] - -DROP TABLE IF EXISTS Customers; -CREATE TABLE Customers -( - FirstName Nullable(String), - LastName String, - Occupation String, - Education String, - Age Nullable(UInt8) -) ENGINE = Memory; - -INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28),('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Joshua','Lee','Professional','Partial College',26),('Edward','Hernandez','Skilled Manual','High School',36),('Dalton','Wood','Professional','Partial College',42),('Christine','Nara','Skilled Manual','Partial College',33),('Cameron','Rodriguez','Professional','Partial College',28),('Angel','Stewart','Professional','Partial College',46),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38); - -drop table if exists EventLog; -create table EventLog -( - LogEntry String, - Created Int64 -) ENGINE = Memory; - -insert into EventLog values ('Darth Vader has entered the room.', 546), ('Rambo is suspciously looking at Darth Vader.', 245234), ('Darth Sidious electrocutes both using Force Lightning.', 245554); - -drop table if exists Dates; -create table Dates -( - EventTime DateTime, -) ENGINE = Memory; - -Insert into Dates VALUES ('2015-10-12') , ('2016-10-12') -Select '-- test summarize --' ; -set dialect='kusto'; -Customers | summarize count(), min(Age), max(Age), avg(Age), sum(Age); -Customers | summarize count(), min(Age), max(Age), avg(Age), sum(Age) by Occupation | order by Occupation; -Customers | summarize countif(Age>40) by Occupation | order by Occupation; -Customers | summarize MyMax = maxif(Age, Age<40) by Occupation | order by Occupation; -Customers | summarize MyMin = minif(Age, Age<40) by Occupation | order by Occupation; -Customers | summarize MyAvg = avgif(Age, Age<40) by Occupation | order by Occupation; -Customers | summarize MySum = sumif(Age, Age<40) by Occupation | order by Occupation; -Customers | summarize dcount(Education); -Customers | summarize dcountif(Education, Occupation=='Professional'); -Customers | summarize count_ = count() by bin(Age, 10) | order by count_ asc; -Customers | summarize job_count = count() by Occupation | where job_count > 0 | order by Occupation; -Customers | summarize 'Edu Count'=count() by Education | sort by 'Edu Count' desc; -- { clientError 62 } - -print '-- make_list() --'; -Customers | summarize f_list = make_list(Education) by Occupation | sort by Occupation; -Customers | summarize f_list = make_list(Education, 2) by Occupation | sort by Occupation; -print '-- make_list_if() --'; -Customers | summarize f_list = make_list_if(FirstName, Age>30) by Occupation | sort by Occupation; -Customers | summarize f_list = make_list_if(FirstName, Age>30, 1) by Occupation | sort by Occupation; -print '-- make_set() --'; -Customers | summarize f_list = make_set(Education) by Occupation | sort by Occupation; -Customers | summarize f_list = make_set(Education, 2) by Occupation | sort by Occupation; -print '-- make_set_if() --'; -Customers | summarize f_list = make_set_if(Education, Age>30) by Occupation | sort by Occupation; -Customers | summarize f_list = make_set_if(Education, Age>30, 1) by Occupation | sort by Occupation; -print '-- stdev() --'; -Customers | project Age | summarize stdev(Age); -print '-- stdevif() --'; -Customers | project Age | summarize stdevif(Age, Age%2==0); -print '-- binary_all_and --'; -Customers | project Age | where Age > 40 | summarize binary_all_and(Age); -print '-- binary_all_or --'; -Customers | project Age | where Age > 40 | summarize binary_all_or(Age); -print '-- binary_all_xor --'; -Customers | project Age | where Age > 40 | summarize binary_all_xor(Age); - -Customers | project Age | summarize percentile(Age, 95); -Customers | project Age | summarize percentiles(Age, 5, 50, 95)|project round(percentiles_Age[0],2),round(percentiles_Age[1],2),round(percentiles_Age[2],2); -Customers | project Age | summarize percentiles(Age, 5, 50, 95)[1]; -Customers | summarize w=count() by AgeBucket=bin(Age, 5) | summarize percentilew(AgeBucket, w, 75); -Customers | summarize w=count() by AgeBucket=bin(Age, 5) | summarize percentilesw(AgeBucket, w, 50, 75, 99.9); - -print '-- Summarize following sort --'; -Customers | sort by FirstName | summarize count() by Occupation | sort by Occupation; - -print '-- summarize with bin --'; -EventLog | summarize count=count() by bin(Created, 1000) | sort by count asc; -EventLog | summarize count=count() by bin(unixtime_seconds_todatetime(Created/1000), 1s) | sort by count asc; -EventLog | summarize count=count() by time_label=bin(Created/1000, 1s) | sort by count asc; -Dates | project bin(datetime(EventTime), 1m); -print '-- make_list_with_nulls --'; -Customers | summarize t = make_list_with_nulls(FirstName); -Customers | summarize f_list = make_list_with_nulls(FirstName) by Occupation | sort by Occupation; -Customers | summarize f_list = make_list_with_nulls(FirstName), a_list = make_list_with_nulls(Age) by Occupation | sort by Occupation; --- TODO: --- arg_max() --- arg_min() diff --git a/tests/queries/0_stateless/02366_kql_tabular.reference b/tests/queries/0_stateless/02366_kql_tabular.reference deleted file mode 100644 index e70c02ce34f..00000000000 --- a/tests/queries/0_stateless/02366_kql_tabular.reference +++ /dev/null @@ -1,139 +0,0 @@ --- test Query only has table name: -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management Bachelors 33 -Peter Nara Skilled Manual Graduate Degree 26 -Latoya Shen Professional Graduate Degree 25 -Joshua Lee Professional Partial College 26 -Edward Hernandez Skilled Manual High School 36 -Dalton Wood Professional Partial College 42 -Christine Nara Skilled Manual Partial College 33 -Cameron Rodriguez Professional Partial College 28 -Angel Stewart Professional Partial College 46 --- Query has Column Selection -- -Theodore Diaz Skilled Manual -Stephanie Cox Management -Peter Nara Skilled Manual -Latoya Shen Professional -Joshua Lee Professional -Edward Hernandez Skilled Manual -Dalton Wood Professional -Christine Nara Skilled Manual -Cameron Rodriguez Professional -Angel Stewart Professional --- Query has limit -- -Theodore Diaz Skilled Manual -Stephanie Cox Management -Peter Nara Skilled Manual -Latoya Shen Professional -Joshua Lee Professional -Theodore Diaz Skilled Manual -Stephanie Cox Management -Peter Nara Skilled Manual -Latoya Shen Professional -Joshua Lee Professional --- Query has second limit with bigger value -- -Theodore Diaz Skilled Manual -Stephanie Cox Management -Peter Nara Skilled Manual -Latoya Shen Professional -Joshua Lee Professional --- Query has second limit with smaller value -- -Theodore Diaz Skilled Manual -Stephanie Cox Management -Peter Nara Skilled Manual --- Query has second Column selection -- -Theodore Diaz -Stephanie Cox -Peter Nara --- Query has second Column selection with extra column -- --- Query with desc sort -- -Theodore -Stephanie -Peter -Latoya -Joshua -Skilled Manual -Skilled Manual -Professional -Professional -Management --- Query with asc sort -- -Management -Professional -Professional -Skilled Manual -Skilled Manual --- Query with sort (without keyword asc desc) -- -Theodore -Stephanie -Peter -Latoya -Joshua -Skilled Manual -Skilled Manual -Professional -Professional -Management --- Query with sort 2 Columns with different direction -- -Stephanie Cox Management -Latoya Shen Professional -Joshua Lee Professional -Peter Nara Skilled Manual -Theodore Diaz Skilled Manual --- Query with second sort -- -Stephanie Cox Management -Latoya Shen Professional -Joshua Lee Professional -Peter Nara Skilled Manual -Theodore Diaz Skilled Manual --- Test String Equals (==) -- -Theodore Diaz Skilled Manual -Peter Nara Skilled Manual -Edward Hernandez Skilled Manual -Christine Nara Skilled Manual --- Test String Not equals (!=) -- -Stephanie Cox Management -Latoya Shen Professional -Joshua Lee Professional -Dalton Wood Professional -Cameron Rodriguez Professional -Angel Stewart Professional --- Test Filter using a list (in) -- -Theodore Diaz Skilled Manual Bachelors -Stephanie Cox Management Bachelors -Edward Hernandez Skilled Manual High School --- Test Filter using a list (!in) -- -Peter Nara Skilled Manual Graduate Degree -Latoya Shen Professional Graduate Degree -Joshua Lee Professional Partial College -Dalton Wood Professional Partial College -Christine Nara Skilled Manual Partial College -Cameron Rodriguez Professional Partial College -Angel Stewart Professional Partial College --- Test Filter using common string operations (contains_cs) -- -Joshua Lee Professional Partial College -Dalton Wood Professional Partial College -Christine Nara Skilled Manual Partial College -Cameron Rodriguez Professional Partial College -Angel Stewart Professional Partial College --- Test Filter using common string operations (startswith_cs) -- -Latoya Shen Professional Graduate Degree -Joshua Lee Professional Partial College -Dalton Wood Professional Partial College -Cameron Rodriguez Professional Partial College -Angel Stewart Professional Partial College --- Test Filter using common string operations (endswith_cs) -- -Latoya Shen Professional Graduate Degree -Joshua Lee Professional Partial College --- Test Filter using numerical equal (==) -- -Peter Nara Skilled Manual Graduate Degree 26 -Joshua Lee Professional Partial College 26 --- Test Filter using numerical great and less (> , <) -- -Stephanie Cox Management Bachelors 33 -Edward Hernandez Skilled Manual High School 36 -Christine Nara Skilled Manual Partial College 33 --- Test Filter using multi where -- -Dalton Wood Professional Partial College 42 -Angel Stewart Professional Partial College 46 --- Complex query with unknown function -- --- Missing column in front of startsWith -- diff --git a/tests/queries/0_stateless/02366_kql_tabular.sql b/tests/queries/0_stateless/02366_kql_tabular.sql deleted file mode 100644 index f73c4c09cca..00000000000 --- a/tests/queries/0_stateless/02366_kql_tabular.sql +++ /dev/null @@ -1,88 +0,0 @@ -DROP TABLE IF EXISTS Customers; -CREATE TABLE Customers -( - FirstName Nullable(String), - LastName String, - Occupation String, - Education String, - Age Nullable(UInt8) -) ENGINE = Memory; - -INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28), ('Stephanie','Cox','Management','Bachelors',33), ('Peter','Nara','Skilled Manual','Graduate Degree',26), ('Latoya','Shen','Professional','Graduate Degree',25), ('Joshua','Lee','Professional','Partial College',26), ('Edward','Hernandez','Skilled Manual','High School',36), ('Dalton','Wood','Professional','Partial College',42), ('Christine','Nara','Skilled Manual','Partial College',33), ('Cameron','Rodriguez','Professional','Partial College',28), ('Angel','Stewart','Professional','Partial College',46); - -set dialect='kusto'; -print '-- test Query only has table name: --'; -Customers; - -print '-- Query has Column Selection --'; -Customers | project FirstName,LastName,Occupation; - -print '-- Query has limit --'; -Customers | project FirstName,LastName,Occupation | take 5; -Customers | project FirstName,LastName,Occupation | limit 5; - -print '-- Query has second limit with bigger value --'; -Customers | project FirstName,LastName,Occupation | take 5 | take 7; - -print '-- Query has second limit with smaller value --'; -Customers | project FirstName,LastName,Occupation | take 5 | take 3; - -print '-- Query has second Column selection --'; -Customers | project FirstName,LastName,Occupation | take 3 | project FirstName,LastName; - -print '-- Query has second Column selection with extra column --'; -Customers| project FirstName,LastName,Occupation | take 3 | project FirstName,LastName,Education;-- { serverError 47 } - -print '-- Query with desc sort --'; -Customers | project FirstName | take 5 | sort by FirstName desc; -Customers | project Occupation | take 5 | order by Occupation desc; - -print '-- Query with asc sort --'; -Customers | project Occupation | take 5 | sort by Occupation asc; - -print '-- Query with sort (without keyword asc desc) --'; -Customers | project FirstName | take 5 | sort by FirstName; -Customers | project Occupation | take 5 | order by Occupation; - -print '-- Query with sort 2 Columns with different direction --'; -Customers | project FirstName,LastName,Occupation | take 5 | sort by Occupation asc, LastName desc; - -print '-- Query with second sort --'; -Customers | project FirstName,LastName,Occupation | take 5 | sort by Occupation desc |sort by Occupation asc, LastName desc; - -print '-- Test String Equals (==) --'; -Customers | project FirstName,LastName,Occupation | where Occupation == 'Skilled Manual'; - -print '-- Test String Not equals (!=) --'; -Customers | project FirstName,LastName,Occupation | where Occupation != 'Skilled Manual'; - -print '-- Test Filter using a list (in) --'; -Customers | project FirstName,LastName,Occupation,Education | where Education in ('Bachelors','High School'); - -print '-- Test Filter using a list (!in) --'; -set dialect='kusto'; -Customers | project FirstName,LastName,Occupation,Education | where Education !in ('Bachelors','High School'); - -print '-- Test Filter using common string operations (contains_cs) --'; -Customers | project FirstName,LastName,Occupation,Education | where Education contains_cs 'Coll'; - -print '-- Test Filter using common string operations (startswith_cs) --'; -Customers | project FirstName,LastName,Occupation,Education | where Occupation startswith_cs 'Prof'; - -print '-- Test Filter using common string operations (endswith_cs) --'; -Customers | project FirstName,LastName,Occupation,Education | where FirstName endswith_cs 'a'; - -print '-- Test Filter using numerical equal (==) --'; -Customers | project FirstName,LastName,Occupation,Education,Age | where Age == 26; - -print '-- Test Filter using numerical great and less (> , <) --'; -Customers | project FirstName,LastName,Occupation,Education,Age | where Age > 30 and Age < 40; - -print '-- Test Filter using multi where --'; -Customers | project FirstName,LastName,Occupation,Education,Age | where Age > 30 | where Occupation == 'Professional'; - -print '-- Complex query with unknown function --'; -hits | where CounterID == 62 and EventDate >= '2013-07-14' and EventDate <= '2013-07-15' and IsRefresh == 0 and DontCountHits == 0 | summarize count() by d=bin(poopoo(EventTime), 1m) | order by d | limit 10; -- { clientError UNKNOWN_FUNCTION } - -print '-- Missing column in front of startsWith --'; -StormEvents | where startswith "W" | summarize Count=count() by State; -- { clientError SYNTAX_ERROR } From f57edeb7dd9b2c7b6b98ee919decf1f7b8cef97c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 28 Jan 2024 18:32:02 +0100 Subject: [PATCH 19/21] impl --- .../00165_jit_aggregate_functions.reference | 70 ------------ .../00165_jit_aggregate_functions.sql | 102 ------------------ 2 files changed, 172 deletions(-) diff --git a/tests/queries/1_stateful/00165_jit_aggregate_functions.reference b/tests/queries/1_stateful/00165_jit_aggregate_functions.reference index fa084170f53..62baba2af8b 100644 --- a/tests/queries/1_stateful/00165_jit_aggregate_functions.reference +++ b/tests/queries/1_stateful/00165_jit_aggregate_functions.reference @@ -68,73 +68,3 @@ Simple functions with non compilable function without key 4611686725751467379 9223371678237104442 3626326766789368100 61384643584599682996279588 408650940859.2896 104735.01095549858 8873898 9223372036854775807 4611686018427387904 3818489297630359920 Simple functions if combinator without key 4611687533683519016 9223371678237104442 4124667747700004330 930178817930.5122 321189.2280948817 4434274 9223372036854775806 4611686018427387904 2265422677606390266 -Aggregation without JIT compilation -Simple functions -1704509 4611700827100483880 9223360787015464643 10441337359398154812 19954243669348.844 9648741.579254271 523264 9223372036854775807 4611686018427387904 4544239379628300646 -732797 4611701940806302259 9223355550934604746 977192643464016658 2054229034942.3723 51998323.94457991 475698 9223372036854775807 4611686018427387904 4091184823334377716 -598875 4611701407242345792 9223362250391155632 9312163881623734456 27615161624211.875 12261797.824844675 337212 9223372036854775807 4611686018427387904 3725992504798702670 -792887 4611699550286611812 9223290551912005343 6930300520201292824 27479710385933.586 53095331.60360441 252197 9223372036854775807 4611686018427387904 6536441508464694614 -3807842 4611710821592843606 9223326163906184987 16710274896338005145 85240848090850.69 22373416.533275086 196036 9223372036854775807 4611686018427387904 1797862753609257231 -25703952 4611709443519524003 9223353913449113943 9946868158853570839 67568783303242.086 3154349.826950714 147211 9223372036854775807 4611686018427387904 8737124378202300429 -716829 4611852156092872082 9223361623076951140 15381015774917924786 170693446547158.72 201431892.4773785 90109 9223372036854775807 4611686018427387904 8209915323001116338 -59183 4611730685242027332 9223354909338698162 8078812522502896568 94622946187035.42 1425270865.0901496 85379 9223372036854775807 4611686018427387904 8909082036598843562 -33010362 4611704682869732882 9223268545373999677 2064452191838585926 26532987929602.555 3695122.4062526934 77807 9223372036854775807 4611686018427387904 5411365383789552292 -800784 4611752907938305166 9223340418389788041 18082918611792817587 233352070043266.62 36535786.81446395 77492 9223372036854775807 4611686018427387904 2059255810151375435 -20810645 4611712185532639162 9223218900001937412 4996531385439292694 68246505203164.63 6316535.831023813 73213 9223372036854775807 4611686018427387904 8852740550386113674 -25843850 4611690025407720929 9223346023778617822 12755881190906812868 185015319325648.16 9962165.34831339 68945 9223372036854775807 4611686018427387904 7849665866595760148 -23447120 4611796031755620254 9223329309291309758 17231649548755339966 255019232629204.38 7937191.271698021 67570 9223372036854775807 4611686018427387904 3435410911925610424 -14739804 4611692230555590277 9223313509005166531 2458378896777063244 38308020331864.36 14590240.469105456 64174 9223372036854775807 4611686018427387904 511910855240035342 -32077710 4611884228437061959 9223352444952988904 12965822147651192908 214467085941034.7 7257521.096258734 60456 9223372036854775807 4611686018427387904 2256071920672551964 -22446879 4611846229717089436 9223124373140579096 13530160492087688838 231724477077663.4 4737362.521046629 58389 9223372036854775807 4611686018427387904 6236276364886386410 -170282 4611833225706935900 9223371583739401906 8076893424988479310 141657635880324.8 1613795518.1065989 57017 9223372036854775807 4611686018427387904 4755775861151848768 -11482817 4611708000353743073 9223337838355779113 14841435427430843458 283531099960470.8 9938452.835998287 52345 9223372036854775807 4611686018427387904 5371586112642152558 -63469 4611695097019173921 9223353530156141191 6296784708578574520 120762239817777.88 579655378.4603049 52142 9223372036854775807 4611686018427387904 4150567963952988110 -29103473 4611744585914335132 9223333530281362537 5908285283932344933 123712996438970.34 867841.595541967 47758 9223372036854775807 4611686018427387904 3238284030821087319 -Simple functions with non compilable function -1704509 4611700827100483880 9223360787015464643 10441337359398154812 3620921835565807284859452 19954243669348.844 9648741.579254271 523264 9223372036854775807 4611686018427387904 4544239379628300646 -732797 4611701940806302259 9223355550934604746 977192643464016658 3289442827160604417733394 2054229034942.3723 51998323.94457991 475698 9223372036854775807 4611686018427387904 4091184823334377716 -598875 4611701407242345792 9223362250391155632 9312163881623734456 2330921446573746856380600 27615161624211.875 12261797.824844675 337212 9223372036854775807 4611686018427387904 3725992504798702670 -792887 4611699550286611812 9223290551912005343 6930300520201292824 1745179600137886041476120 27479710385933.586 53095331.60360441 252197 9223372036854775807 4611686018427387904 6536441508464694614 -3807842 4611710821592843606 9223326163906184987 16710274896338005145 1356295121550317411019929 85240848090850.69 22373416.533275086 196036 9223372036854775807 4611686018427387904 1797862753609257231 -25703952 4611709443519524003 9223353913449113943 9946868158853570839 1018731388338768841564439 67568783303242.086 3154349.826950714 147211 9223372036854775807 4611686018427387904 8737124378202300429 -716829 4611852156092872082 9223361623076951140 15381015774917924786 623810478612337115371442 170693446547158.72 201431892.4773785 90109 9223372036854775807 4611686018427387904 8209915323001116338 -59183 4611730685242027332 9223354909338698162 8078812522502896568 589916507545680254024632 94622946187035.42 1425270865.0901496 85379 9223372036854775807 4611686018427387904 8909082036598843562 -33010362 4611704682869732882 9223268545373999677 2064452191838585926 538517864195994778911814 26532987929602.555 3695122.4062526934 77807 9223372036854775807 4611686018427387904 5411365383789552292 -800784 4611752907938305166 9223340418389788041 18082918611792817587 535545510122473785781683 233352070043266.62 36535786.81446395 77492 9223372036854775807 4611686018427387904 2059255810151375435 -20810645 4611712185532639162 9223218900001937412 4996531385439292694 506405014842860050255126 68246505203164.63 6316535.831023813 73213 9223372036854775807 4611686018427387904 8852740550386113674 -25843850 4611690025407720929 9223346023778617822 12755881190906812868 476547495537329753708996 185015319325648.16 9962165.34831339 68945 9223372036854775807 4611686018427387904 7849665866595760148 -23447120 4611796031755620254 9223329309291309758 17231649548755339966 467236365548464278670014 255019232629204.38 7937191.271698021 67570 9223372036854775807 4611686018427387904 3435410911925610424 -14739804 4611692230555590277 9223313509005166531 2458378896777063244 444126268697527941770060 38308020331864.36 14590240.469105456 64174 9223372036854775807 4611686018427387904 511910855240035342 -32077710 4611884228437061959 9223352444952988904 12965822147651192908 417407443977973675608140 214467085941034.7 7257521.096258734 60456 9223372036854775807 4611686018427387904 2256071920672551964 -22446879 4611846229717089436 9223124373140579096 13530160492087688838 403462269796593691082374 231724477077663.4 4737362.521046629 58389 9223372036854775807 4611686018427387904 6236276364886386410 -170282 4611833225706935900 9223371583739401906 8076893424988479310 394417911933408911581006 141657635880324.8 1613795518.1065989 57017 9223372036854775807 4611686018427387904 4755775861151848768 -11482817 4611708000353743073 9223337838355779113 14841435427430843458 361995300393829962204226 283531099960470.8 9938452.835998287 52345 9223372036854775807 4611686018427387904 5371586112642152558 -63469 4611695097019173921 9223353530156141191 6296784708578574520 360843057610541117735096 120762239817777.88 579655378.4603049 52142 9223372036854775807 4611686018427387904 4150567963952988110 -29103473 4611744585914335132 9223333530281362537 5908285283932344933 330534668598011678200421 123712996438970.34 867841.595541967 47758 9223372036854775807 4611686018427387904 3238284030821087319 -Simple functions if combinator -1704509 4611700827100483880 9223310246721229500 16398241567152875142 62618822667209.71 2224726.7626273884 261874 9223372036854775806 4611686018427387904 4518874482384062894 -732797 4611721382223060002 9223355550934604746 16281585268876620522 68472164943295.68 5898616.931652982 237784 9223372036854775806 4611686018427387904 3641900047478154650 -598875 4611701407242345792 9223362250391155632 3577699408183553052 21300140553347.42 53771550.26565126 167966 9223372036854775806 4611686018427387904 1688477495230210408 -792887 4611699550286611812 9223164887726235740 7088177025760385824 56461952267903.89 92835869.96920013 125539 9223372036854775806 4611686018427387904 4850868151095058072 -3807842 4611710821592843606 9223283397553859544 5756765290752687660 58835559208469.4 39794091.419183925 97845 9223372036854775806 4611686018427387904 6845214684357194564 -25703952 4611784761593342388 9223241341744449690 4782279928971192568 65182094768443.91 9276773.708181158 73368 9223372036854775806 4611686018427387904 1384302533387727316 -716829 4611852156092872082 9223361623076951140 8613712481895484190 191445613359755.62 291083243.75407773 44993 9223372036854775806 4611686018427387904 6344483471397203854 -59183 4611730685242027332 9223354909338698162 18369075291092794110 429013599530392 5925109959.715378 42817 9223372036854775806 4611686018427387904 5909305558020042898 -33010362 4611704682869732882 9223092117352620518 9991152681891671022 257099731913529.5 12412830.045471078 38861 9223372036854775806 4611686018427387904 4672855013852508626 -800784 4611752907938305166 9223309994342931384 5251877538869750510 135472890315726.03 53535427.52018088 38767 9223372036854775806 4611686018427387904 7801864489649220514 -20810645 4611712185532639162 9223218900001937412 11803718472901310700 323593455407553 10496765.20741332 36477 9223372036854775806 4611686018427387904 5941995311893397960 -25843850 4611744529689964352 9223346023778617822 127137885677350808 3700925266420.715 18966925.191309396 34353 9223372036854775806 4611686018427387904 6700111718676827412 -23447120 4611796031755620254 9223329309291309758 1841522159325376278 54534534450526.42 6271211.193812284 33768 9223372036854775806 4611686018427387904 2325654077031843898 -14739804 4611762063154116632 9223007205463222212 16302703534054321116 506987919332451.8 6885575.861759452 32156 9223372036854775806 4611686018427387904 2114922310535979832 -32077710 4612033458080771112 9223352444952988904 421072759851674408 13955745719596.793 12220152.393889504 30172 9223372036854775806 4611686018427387904 4399934528735249092 -22446879 4611846229717089436 9223124373140579096 6577134317587565298 224866980668999.47 2482202.163802278 29249 9223372036854775806 4611686018427387904 8763910740678180498 -170282 4611833225706935900 9223371583739401906 15764226366913732386 551447384017691 2515144222.953728 28587 9223372036854775806 4611686018427387904 8217388408377809010 -11482817 4611990575414646848 9223302669582414438 9828522700609834800 378121905921203.2 34845264.2080656 25993 9223372036854775806 4611686018427387904 4689180182672571856 -63469 4612175339998036670 9222961628400798084 17239621485933250238 663164390134376.5 7825349797.6059 25996 9223372036854775806 4611686018427387904 2067736879306995526 -29103473 4611744585914335132 9223035551850347954 12590190375872647672 525927999326314.7 26049107.15514301 23939 9223372036854775806 4611686018427387904 8318055464870862444 -Simple functions without key -4611686725751467379 9223371678237104442 3626326766789368100 408650940859.2896 104735.01095549858 8873898 9223372036854775807 4611686018427387904 3818489297630359920 -Simple functions with non compilable function without key -4611686725751467379 9223371678237104442 3626326766789368100 61384643584599682996279588 408650940859.2896 104735.01095549858 8873898 9223372036854775807 4611686018427387904 3818489297630359920 -Simple functions if combinator without key -4611687533683519016 9223371678237104442 4124667747700004330 930178817930.5122 321189.2280948817 4434274 9223372036854775806 4611686018427387904 2265422677606390266 diff --git a/tests/queries/1_stateful/00165_jit_aggregate_functions.sql b/tests/queries/1_stateful/00165_jit_aggregate_functions.sql index 157d5892ad8..7fd0e1822a5 100644 --- a/tests/queries/1_stateful/00165_jit_aggregate_functions.sql +++ b/tests/queries/1_stateful/00165_jit_aggregate_functions.sql @@ -1,4 +1,3 @@ -SET compile_aggregate_expressions = 1; SET min_count_to_compile_aggregate_expression = 0; SET max_bytes_before_external_group_by='200M'; -- might be randomized to 1 leading to timeout @@ -101,104 +100,3 @@ SELECT FROM test.hits ORDER BY min_watch_id DESC LIMIT 20; - -SET compile_aggregate_expressions = 0; - -SELECT 'Aggregation without JIT compilation'; - -SELECT 'Simple functions'; - -SELECT - CounterID, - min(WatchID), - max(WatchID), - sum(WatchID), - avg(WatchID), - avgWeighted(WatchID, CounterID), - count(WatchID), - groupBitOr(WatchID), - groupBitAnd(WatchID), - groupBitXor(WatchID) -FROM test.hits -GROUP BY CounterID ORDER BY count() DESC LIMIT 20; - -SELECT 'Simple functions with non compilable function'; -SELECT - CounterID, - min(WatchID), - max(WatchID), - sum(WatchID), - sum(toUInt128(WatchID)), - avg(WatchID), - avgWeighted(WatchID, CounterID), - count(WatchID), - groupBitOr(WatchID), - groupBitAnd(WatchID), - groupBitXor(WatchID) -FROM test.hits -GROUP BY CounterID ORDER BY count() DESC LIMIT 20; - -SELECT 'Simple functions if combinator'; - -WITH (WatchID % 2 == 0) AS predicate -SELECT - CounterID, - minIf(WatchID,predicate), - maxIf(WatchID, predicate), - sumIf(WatchID, predicate), - avgIf(WatchID, predicate), - avgWeightedIf(WatchID, CounterID, predicate), - countIf(WatchID, predicate), - groupBitOrIf(WatchID, predicate), - groupBitAndIf(WatchID, predicate), - groupBitXorIf(WatchID, predicate) -FROM test.hits -GROUP BY CounterID ORDER BY count() DESC LIMIT 20; - -SELECT 'Simple functions without key'; - -SELECT - min(WatchID) AS min_watch_id, - max(WatchID), - sum(WatchID), - avg(WatchID), - avgWeighted(WatchID, CounterID), - count(WatchID), - groupBitOr(WatchID), - groupBitAnd(WatchID), - groupBitXor(WatchID) -FROM test.hits -ORDER BY min_watch_id DESC LIMIT 20; - -SELECT 'Simple functions with non compilable function without key'; - -SELECT - min(WatchID) AS min_watch_id, - max(WatchID), - sum(WatchID), - sum(toUInt128(WatchID)), - avg(WatchID), - avgWeighted(WatchID, CounterID), - count(WatchID), - groupBitOr(WatchID), - groupBitAnd(WatchID), - groupBitXor(WatchID) -FROM test.hits -ORDER BY min_watch_id DESC LIMIT 20; - -SELECT 'Simple functions if combinator without key'; - -WITH (WatchID % 2 == 0) AS predicate -SELECT - minIf(WatchID, predicate) as min_watch_id, - maxIf(WatchID, predicate), - sumIf(WatchID, predicate), - avgIf(WatchID, predicate), - avgWeightedIf(WatchID, CounterID, predicate), - countIf(WatchID, predicate), - groupBitOrIf(WatchID, predicate), - groupBitAndIf(WatchID, predicate), - groupBitXorIf(WatchID, predicate) -FROM test.hits -ORDER BY min_watch_id -DESC LIMIT 20; From aec3f28ccb92e18595e87bce8e51e831b8c17b92 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 23:06:50 +0100 Subject: [PATCH 20/21] Support backups for compressed in-memory tables --- programs/server/config.xml | 4 +++ src/Formats/NativeWriter.cpp | 3 +- src/Storages/StorageMemory.cpp | 22 ++++++++++--- ...3_backup_of_in_memory_compressed.reference | 2 ++ .../02973_backup_of_in_memory_compressed.sh | 31 +++++++++++++++++++ 5 files changed, 56 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02973_backup_of_in_memory_compressed.reference create mode 100755 tests/queries/0_stateless/02973_backup_of_in_memory_compressed.sh diff --git a/programs/server/config.xml b/programs/server/config.xml index 0dc271692b8..6a40818332b 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1562,6 +1562,10 @@ 30000000 + + backups + +