diff --git a/contrib/googletest-cmake/CMakeLists.txt b/contrib/googletest-cmake/CMakeLists.txt index 3905df03155..5b00096615a 100644 --- a/contrib/googletest-cmake/CMakeLists.txt +++ b/contrib/googletest-cmake/CMakeLists.txt @@ -14,12 +14,12 @@ add_library(_gtest_all INTERFACE) target_link_libraries(_gtest_all INTERFACE _gtest _gtest_main) add_library(ch_contrib::gtest_all ALIAS _gtest_all) - add_library(_gmock "${SRC_DIR}/googlemock/src/gmock-all.cc") set_target_properties(_gmock PROPERTIES VERSION "1.0.0") target_compile_definitions (_gmock PUBLIC GTEST_HAS_POSIX_RE=0) target_include_directories(_gmock SYSTEM PUBLIC "${SRC_DIR}/googlemock/include" "${SRC_DIR}/googletest/include") target_include_directories(_gmock PRIVATE "${SRC_DIR}/googlemock") +target_link_libraries(_gmock PUBLIC _gtest) add_library(_gmock_main "${SRC_DIR}/googlemock/src/gmock_main.cc") set_target_properties(_gmock_main PROPERTIES VERSION "1.0.0") diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index e98f19b2a65..58c6280da35 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1924,6 +1924,14 @@ SELECT * FROM test.hits format Protobuf SETTINGS format_protobuf_use_autogenerat In this case autogenerated Protobuf schema will be saved in file `path/to/schema/schema.capnp`. +### Drop Protobuf cache + +To reload Protobuf schema loaded from [format_schema_path](../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-format_schema_path) use [SYSTEM DROP ... FORMAT CACHE](../sql-reference/statements/system.md/#system-drop-schema-format) statement. + +```sql +SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf +``` + ## ProtobufSingle {#protobufsingle} Same as [Protobuf](#protobuf) but for storing/parsing single Protobuf message without length delimiters. diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 4e1e7aa6a49..1558e64f99b 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -119,6 +119,18 @@ The compiled expression cache is enabled/disabled with the query/user/profile-le Clears the [query cache](../../operations/query-cache.md). +## DROP FORMAT SCHEMA CACHE {#system-drop-schema-format} + +Clears cache for schemas loaded from [format_schema_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path). + +Supported formats: + +- Protobuf + +```sql +SYSTEM DROP FORMAT SCHEMA CACHE [FOR Protobuf] +``` + ## FLUSH LOGS Flushes buffered log messages to system tables, e.g. system.query_log. Mainly useful for debugging since most system tables have a default flush interval of 7.5 seconds. diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index ac5c9f99e0e..7d70b81a178 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -706,6 +706,17 @@ bool Client::processWithFuzzing(const String & full_query) return true; } + // Kusto is not a subject for fuzzing (yet) + if (global_context->getSettingsRef().dialect == DB::Dialect::kusto) + { + return true; + } + if (auto *q = orig_ast->as()) + { + if (auto *setDialect = q->changes.tryGet("dialect"); setDialect && setDialect->safeGet() == "kusto") + return true; + } + // Don't repeat: // - INSERT -- Because the tables may grow too big. // - CREATE -- Because first we run the unmodified query, it will succeed, diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 6720022b3e1..7962d686ed0 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -155,6 +155,7 @@ enum class AccessType M(SYSTEM_DROP_FILESYSTEM_CACHE, "SYSTEM DROP FILESYSTEM CACHE, DROP FILESYSTEM CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_SYNC_FILESYSTEM_CACHE, "SYSTEM REPAIR FILESYSTEM CACHE, REPAIR FILESYSTEM CACHE, SYNC FILESYSTEM CACHE", GLOBAL, SYSTEM) \ M(SYSTEM_DROP_SCHEMA_CACHE, "SYSTEM DROP SCHEMA CACHE, DROP SCHEMA CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ + M(SYSTEM_DROP_FORMAT_SCHEMA_CACHE, "SYSTEM DROP FORMAT SCHEMA CACHE, DROP FORMAT SCHEMA CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_S3_CLIENT_CACHE, "SYSTEM DROP S3 CLIENT, DROP S3 CLIENT CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_CACHE, "DROP CACHE", GROUP, SYSTEM) \ M(SYSTEM_RELOAD_CONFIG, "RELOAD CONFIG", GLOBAL, SYSTEM_RELOAD) \ diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 1aa28d6c8e1..1c93bc5d35e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -591,8 +591,8 @@ if (ENABLE_TESTS) ) target_link_libraries(unit_tests_dbms PRIVATE - ch_contrib::gtest_all ch_contrib::gmock_all + ch_contrib::gtest_all clickhouse_functions clickhouse_aggregate_functions clickhouse_parsers diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 30ffffdeeb1..9ca104ff942 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -47,6 +47,7 @@ #include #include #include +#include #include #include @@ -349,7 +350,10 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu if (is_interactive || ignore_error) { String message; - res = tryParseQuery(*parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth); + if (dialect == Dialect::kusto) + res = tryParseKQLQuery(*parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth); + else + res = tryParseQuery(*parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth); if (!res) { @@ -359,7 +363,10 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu } else { - res = parseQueryAndMovePosition(*parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth); + if (dialect == Dialect::kusto) + res = parseKQLQueryAndMovePosition(*parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth); + else + res = parseQueryAndMovePosition(*parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth); } if (is_interactive) diff --git a/src/Core/MySQL/PacketsProtocolBinary.cpp b/src/Core/MySQL/PacketsProtocolBinary.cpp index 9f799d89835..c875fceab1d 100644 --- a/src/Core/MySQL/PacketsProtocolBinary.cpp +++ b/src/Core/MySQL/PacketsProtocolBinary.cpp @@ -50,10 +50,7 @@ ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTyp payload_size += 1; break; case TypeIndex::UInt8: - if (data_type->getName() == "Bool") - payload_size += 2; // BIT MySQL type is string in binary - else - payload_size += 1; + payload_size += 1; break; case TypeIndex::Int16: case TypeIndex::UInt16: @@ -168,8 +165,6 @@ void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const } case TypeIndex::UInt8: { UInt8 value = assert_cast &>(*col).getData()[row_num]; - if (data_type->getName() == "Bool") - buffer.write(static_cast(1)); buffer.write(reinterpret_cast(&value), 1); break; } diff --git a/src/Core/MySQL/PacketsProtocolText.cpp b/src/Core/MySQL/PacketsProtocolText.cpp index 3be8acad3ea..9aff3de457c 100644 --- a/src/Core/MySQL/PacketsProtocolText.cpp +++ b/src/Core/MySQL/PacketsProtocolText.cpp @@ -20,6 +20,7 @@ namespace ProtocolText ResultSetRow::ResultSetRow(const Serializations & serializations, const Columns & columns_, int row_num_) : columns(columns_), row_num(row_num_) { + static FormatSettings format_settings = {.bool_true_representation = "1", .bool_false_representation = "0"}; for (size_t i = 0; i < columns.size(); ++i) { if (columns[i]->isNullAt(row_num)) @@ -30,7 +31,7 @@ ResultSetRow::ResultSetRow(const Serializations & serializations, const Columns else { WriteBufferFromOwnString ostr; - serializations[i]->serializeText(*columns[i], row_num, ostr, FormatSettings()); + serializations[i]->serializeText(*columns[i], row_num, ostr, format_settings); payload_size += getLengthEncodedStringSize(ostr.str()); serialized.push_back(std::move(ostr.str())); } @@ -45,12 +46,10 @@ size_t ResultSetRow::getPayloadSize() const void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const { for (size_t i = 0; i < columns.size(); ++i) - { if (columns[i]->isNullAt(row_num)) buffer.write(serialized[i].data(), 1); else writeLengthEncodedString(serialized[i], buffer); - } } void ComFieldList::readPayloadImpl(ReadBuffer & payload) @@ -142,19 +141,13 @@ ColumnDefinition getColumnDefinition(const String & column_name, const DataTypeP CharacterSet charset = CharacterSet::binary; int flags = 0; uint8_t decimals = 0; - TypeIndex type_index = removeLowCardinality(removeNullable(data_type))->getTypeId(); + DataTypePtr normalized_data_type = removeLowCardinality(removeNullable(data_type)); + TypeIndex type_index = normalized_data_type->getTypeId(); switch (type_index) { case TypeIndex::UInt8: - if (data_type->getName() == "Bool") - { - column_type = ColumnType::MYSQL_TYPE_BIT; - } - else - { - column_type = ColumnType::MYSQL_TYPE_TINY; - flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; - } + column_type = ColumnType::MYSQL_TYPE_TINY; + flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; break; case TypeIndex::UInt16: column_type = ColumnType::MYSQL_TYPE_SHORT; @@ -213,7 +206,7 @@ ColumnDefinition getColumnDefinition(const String & column_name, const DataTypeP // MySQL Decimal has max 65 precision and 30 scale // Decimal256 (min scale is 39) is higher than the MySQL supported range and handled in the default case // See https://dev.mysql.com/doc/refman/8.0/en/precision-math-decimal-characteristics.html - const auto & type = assert_cast(*data_type); + const auto & type = assert_cast(*normalized_data_type); if (type.getPrecision() > 65 || type.getScale() > 30) { column_type = ColumnType::MYSQL_TYPE_STRING; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0d5e5627b22..513d5d762ef 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -680,7 +680,6 @@ class IColumn; M(Bool, query_plan_aggregation_in_order, true, "Use query plan for aggregation-in-order optimisation", 0) \ M(Bool, query_plan_remove_redundant_sorting, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries", 0) \ M(Bool, query_plan_remove_redundant_distinct, true, "Remove redundant Distinct step in query plan", 0) \ - M(Bool, query_plan_optimize_projection, true, "Use query plan for aggregation-in-order optimisation", 0) \ M(UInt64, regexp_max_matches_per_row, 1000, "Max matches of any single regexp per row, used to safeguard 'extractAllGroupsHorizontal' against consuming too much memory with greedy RE.", 0) \ \ M(UInt64, limit, 0, "Limit on read rows from the most 'end' result for select query, default 0 means no limit length", 0) \ @@ -867,6 +866,7 @@ class IColumn; MAKE_OBSOLETE(M, UInt64, restore_threads, 16) \ MAKE_OBSOLETE(M, Bool, optimize_duplicate_order_by_and_distinct, false) \ MAKE_OBSOLETE(M, UInt64, parallel_replicas_min_number_of_granules_to_enable, 0) \ + MAKE_OBSOLETE(M, Bool, query_plan_optimize_projection, true) \ /** The section above is for obsolete settings. Do not add anything there. */ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index a30d8040f47..96e80c103e2 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -140,7 +140,7 @@ IMPLEMENT_SETTING_ENUM(Dialect, ErrorCodes::BAD_ARGUMENTS, {{"clickhouse", Dialect::clickhouse}, {"kusto", Dialect::kusto}, {"prql", Dialect::prql}}) - // FIXME: do not add 'kusto_auto' to the list. Maybe remove it from code completely? + IMPLEMENT_SETTING_ENUM(ParallelReplicasCustomKeyFilterType, ErrorCodes::BAD_ARGUMENTS, {{"default", ParallelReplicasCustomKeyFilterType::DEFAULT}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 034e4c8c887..006221e4593 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -207,7 +207,6 @@ enum class Dialect { clickhouse, kusto, - kusto_auto, prql, }; diff --git a/src/Disks/IO/ReadBufferFromWebServer.h b/src/Disks/IO/ReadBufferFromWebServer.h index fa899cf2c5e..b4edf16b095 100644 --- a/src/Disks/IO/ReadBufferFromWebServer.h +++ b/src/Disks/IO/ReadBufferFromWebServer.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -33,7 +34,7 @@ public: void setReadUntilPosition(size_t position) override; - size_t getFileOffsetOfBufferEnd() const override { return offset; } + size_t getFileOffsetOfBufferEnd() const override { return offset.load(std::memory_order_relaxed); } bool supportsRightBoundedReads() const override { return true; } @@ -54,7 +55,10 @@ private: bool use_external_buffer; - off_t offset = 0; + /// atomic is required for CachedOnDiskReadBufferFromFile, which can access + /// to this variable via getFileOffsetOfBufferEnd()/seek() from multiple + /// threads. + std::atomic offset = 0; off_t read_until_position = 0; }; diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index ea05012fb61..1db5024124f 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -162,7 +162,22 @@ bool WebObjectStorage::exists(const std::string & path) const if (startsWith(it->first, path) || (it != files.begin() && startsWith(std::prev(it)->first, path))) + { + shared_lock.unlock(); + std::unique_lock unique_lock(metadata_mutex); + + /// The code relies on invariant that if this function returned true + /// the file exists in files. + /// In this case we have a directory which doesn't explicitly exists (like store/xxx/yyy) + /// ^^^^^ + /// Adding it to the files + files.emplace(std::make_pair(path, FileData({.type = FileType::Directory}))); + + unique_lock.unlock(); + shared_lock.lock(); + return true; + } return false; } diff --git a/src/Formats/ProtobufSchemas.cpp b/src/Formats/ProtobufSchemas.cpp index 86c81e1a3c3..5d1144e76ea 100644 --- a/src/Formats/ProtobufSchemas.cpp +++ b/src/Formats/ProtobufSchemas.cpp @@ -21,6 +21,12 @@ ProtobufSchemas & ProtobufSchemas::instance() return instance; } +void ProtobufSchemas::clear() +{ + std::lock_guard lock(mutex); + importers.clear(); +} + class ProtobufSchemas::ImporterWithSourceTree : public google::protobuf::compiler::MultiFileErrorCollector { public: diff --git a/src/Formats/ProtobufSchemas.h b/src/Formats/ProtobufSchemas.h index 6f868cd6803..966dffbd6b5 100644 --- a/src/Formats/ProtobufSchemas.h +++ b/src/Formats/ProtobufSchemas.h @@ -54,6 +54,8 @@ public: }; static ProtobufSchemas & instance(); + // Clear cached protobuf schemas + void clear(); /// Parses the format schema, then parses the corresponding proto file, and returns the descriptor of the message type. /// The function never returns nullptr, it throws an exception if it cannot load or parse the file. diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 48008827f48..31f7f24eb13 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -106,6 +106,9 @@ list (APPEND OBJECT_LIBS $) add_subdirectory(array) list (APPEND OBJECT_LIBS $) +add_subdirectory(Kusto) +list (APPEND OBJECT_LIBS $) + if (TARGET ch_contrib::datasketches) add_subdirectory(UniqTheta) list (APPEND OBJECT_LIBS $) diff --git a/src/Functions/Kusto/CMakeLists.txt b/src/Functions/Kusto/CMakeLists.txt new file mode 100644 index 00000000000..3c534905d22 --- /dev/null +++ b/src/Functions/Kusto/CMakeLists.txt @@ -0,0 +1,8 @@ +include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") +add_headers_and_sources(clickhouse_functions_kusto .) +add_library(clickhouse_functions_kusto OBJECT ${clickhouse_functions_kusto_sources} ${clickhouse_functions_kusto_headers}) +target_link_libraries(clickhouse_functions_kusto PRIVATE dbms clickhouse_functions_gatherutils) + +if (OMIT_HEAVY_DEBUG_SYMBOLS) + target_compile_options(clickhouse_functions_kusto PRIVATE "-g0") +endif() diff --git a/src/Functions/Kusto/KqlArraySort.cpp b/src/Functions/Kusto/KqlArraySort.cpp new file mode 100644 index 00000000000..5be36328cc3 --- /dev/null +++ b/src/Functions/Kusto/KqlArraySort.cpp @@ -0,0 +1,264 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; +} + +template +class FunctionKqlArraySort : public KqlFunctionBase +{ +public: + static constexpr auto name = Name::name; + explicit FunctionKqlArraySort(ContextPtr context_) : context(context_) { } + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} needs at least one argument; passed {}.", + getName(), + arguments.size()); + + auto array_count = arguments.size(); + + if (!isArray(arguments.at(array_count - 1).type)) + --array_count; + + DataTypes nested_types; + for (size_t index = 0; index < array_count; ++index) + { + const DataTypeArray * array_type = checkAndGetDataType(arguments[index].type.get()); + if (!array_type) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument {} of function {} must be array. Found {} instead.", + index + 1, + getName(), + arguments[0].type->getName()); + + nested_types.emplace_back(array_type->getNestedType()); + } + + DataTypes data_types(array_count); + + for (size_t i = 0; i < array_count; ++i) + data_types[i] = std::make_shared(makeNullable(nested_types[i])); + + return std::make_shared(data_types); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + size_t array_count = arguments.size(); + const auto & last_arg = arguments[array_count - 1]; + + size_t input_rows_count_local = input_rows_count; + + bool null_last = true; + if (!isArray(last_arg.type)) + { + --array_count; + null_last = check_condition(last_arg, context, input_rows_count_local); + } + + ColumnsWithTypeAndName new_args; + ColumnPtr first_array_column; + std::unordered_set null_indices; + DataTypes nested_types; + + String sort_function = is_desc ? "arrayReverseSort" : "arraySort"; + + for (size_t i = 0; i < array_count; ++i) + { + ColumnPtr holder = arguments[i].column->convertToFullColumnIfConst(); + + const ColumnArray * column_array = checkAndGetColumn(holder.get()); + const DataTypeArray * array_type = checkAndGetDataType(arguments[i].type.get()); + + if (!column_array) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Argument {} of function {} must be array. Found column {} instead.", + i + 1, + getName(), + holder->getName()); + + nested_types.emplace_back(makeNullable(array_type->getNestedType())); + if (i == 0) + { + first_array_column = holder; + new_args.push_back(arguments[i]); + } + else if (!column_array->hasEqualOffsets(static_cast(*first_array_column))) + { + null_indices.insert(i); + } + else + new_args.push_back(arguments[i]); + } + + auto zipped + = FunctionFactory::instance().get("arrayZip", context)->build(new_args)->execute(new_args, result_type, input_rows_count_local); + + ColumnsWithTypeAndName sort_arg({{zipped, std::make_shared(result_type), "zipped"}}); + auto sorted_tuple + = FunctionFactory::instance().get(sort_function, context)->build(sort_arg)->execute(sort_arg, result_type, input_rows_count_local); + + auto null_type = std::make_shared(std::make_shared()); + + Columns tuple_columns(array_count); + size_t sorted_index = 0; + for (size_t i = 0; i < array_count; ++i) + { + if (null_indices.contains(i)) + { + auto fun_array = FunctionFactory::instance().get("array", context); + + DataTypePtr arg_type + = std::make_shared(makeNullable(nested_types[i])); + + ColumnsWithTypeAndName null_array_arg({ + {null_type->createColumnConstWithDefaultValue(input_rows_count_local), null_type, "NULL"}, + }); + + tuple_columns[i] = fun_array->build(null_array_arg)->execute(null_array_arg, arg_type, input_rows_count_local); + tuple_columns[i] = tuple_columns[i]->convertToFullColumnIfConst(); + } + else + { + ColumnsWithTypeAndName untuple_args( + {{ColumnWithTypeAndName(sorted_tuple, std::make_shared(result_type), "sorted")}, + {DataTypeUInt8().createColumnConst(1, toField(UInt8(sorted_index + 1))), std::make_shared(), ""}}); + auto tuple_coulmn = FunctionFactory::instance() + .get("tupleElement", context) + ->build(untuple_args) + ->execute(untuple_args, result_type, input_rows_count_local); + + auto out_tmp = ColumnArray::create(nested_types[i]->createColumn()); + + size_t array_size = tuple_coulmn->size(); + const auto * arr = checkAndGetColumn(tuple_coulmn.get()); + + for (size_t j = 0; j < array_size; ++j) + { + Field arr_field; + arr->get(j, arr_field); + out_tmp->insert(arr_field); + } + + tuple_columns[i] = std::move(out_tmp); + + ++sorted_index; + } + } + + if (!null_last) + { + Columns adjusted_columns(array_count); + + ColumnWithTypeAndName arg_of_index{nullptr, std::make_shared(nested_types[0]), "array"}; + arg_of_index.column = tuple_columns[0]; + + auto inside_null_type = nested_types[0]; + ColumnsWithTypeAndName indexof_args({ + arg_of_index, + {inside_null_type->createColumnConstWithDefaultValue(input_rows_count_local), inside_null_type, "NULL"}, + }); + + auto null_index_datetype = std::make_shared(); + + ColumnWithTypeAndName slice_index{nullptr, null_index_datetype, ""}; + slice_index.column = FunctionFactory::instance() + .get("indexOf", context) + ->build(indexof_args) + ->execute(indexof_args, result_type, input_rows_count_local); + + auto null_index_in_array = slice_index.column->get64(0); + if (null_index_in_array > 0) + { + ColumnWithTypeAndName slice_index_len{nullptr, null_index_datetype, ""}; + slice_index_len.column = DataTypeUInt64().createColumnConst(1, toField(UInt64(null_index_in_array - 1))); + + auto fun_slice = FunctionFactory::instance().get("arraySlice", context); + + for (size_t i = 0; i < array_count; ++i) + { + if (null_indices.contains(i)) + { + adjusted_columns[i] = std::move(tuple_columns[i]); + } + else + { + DataTypePtr arg_type = std::make_shared(nested_types[i]); + + ColumnsWithTypeAndName slice_args_left( + {{ColumnWithTypeAndName(tuple_columns[i], arg_type, "array")}, + {DataTypeUInt8().createColumnConst(1, toField(UInt8(1))), std::make_shared(), ""}, + slice_index_len}); + + ColumnsWithTypeAndName slice_args_right( + {{ColumnWithTypeAndName(tuple_columns[i], arg_type, "array")}, slice_index}); + ColumnWithTypeAndName arr_left{ + fun_slice->build(slice_args_left)->execute(slice_args_left, arg_type, input_rows_count_local), arg_type, ""}; + ColumnWithTypeAndName arr_right{ + fun_slice->build(slice_args_right)->execute(slice_args_right, arg_type, input_rows_count_local), arg_type, ""}; + + ColumnsWithTypeAndName arr_cancat({arr_right, arr_left}); + auto out_tmp = FunctionFactory::instance() + .get("arrayConcat", context) + ->build(arr_cancat) + ->execute(arr_cancat, arg_type, input_rows_count_local); + adjusted_columns[i] = std::move(out_tmp); + } + } + return ColumnTuple::create(adjusted_columns); + } + } + return ColumnTuple::create(tuple_columns); + } + +private: + ContextPtr context; +}; + +struct NameKqlArraySortAsc +{ + static constexpr auto name = "kql_array_sort_asc"; +}; + +struct NameKqlArraySortDesc +{ + static constexpr auto name = "kql_array_sort_desc"; +}; + +using FunctionKqlArraySortAsc = FunctionKqlArraySort; +using FunctionKqlArraySortDesc = FunctionKqlArraySort; + +REGISTER_FUNCTION(KqlArraySort) +{ + factory.registerFunction(); + factory.registerFunction(); +} + +} diff --git a/src/Functions/Kusto/KqlFunctionBase.h b/src/Functions/Kusto/KqlFunctionBase.h new file mode 100644 index 00000000000..efdf8982f4b --- /dev/null +++ b/src/Functions/Kusto/KqlFunctionBase.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include "Functions/array/FunctionArrayMapped.h" + +namespace DB +{ + +class KqlFunctionBase : public IFunction +{ +public: + static bool check_condition (const ColumnWithTypeAndName & condition, ContextPtr context, size_t input_rows_count) + { + ColumnsWithTypeAndName if_columns( + { + condition, + {DataTypeUInt8().createColumnConst(1, toField(UInt8(1))), std::make_shared(), ""}, + {DataTypeUInt8().createColumnConst(1, toField(UInt8(2))), std::make_shared(), ""} + }); + auto if_res = FunctionFactory::instance().get("if", context)->build(if_columns)->execute(if_columns, std::make_shared(), input_rows_count); + auto result = if_res->getUInt(0); + return (result == 1); + } +}; + +} diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index e21513e0ea2..49c82a1fc0e 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -44,6 +44,10 @@ public: virtual String getInfoForLog() { return ""; } + /// NOTE: This method should be thread-safe against seek(), since it can be + /// used in CachedOnDiskReadBufferFromFile from multiple threads (because + /// it first releases the buffer, and then do logging, and so other thread + /// can already call seek() which will lead to data-race). virtual size_t getFileOffsetOfBufferEnd() const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getFileOffsetOfBufferEnd() not implemented"); } /// If true, setReadUntilPosition() guarantees that eof will be reported at the given position. diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 337d948471f..0c9b9180074 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -863,20 +863,6 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map & key_names_pos_map) -{ - ::sort(outputs.begin(), outputs.end(), [&key_names_pos_map](const Node * lhs, const Node * rhs) - { - return key_names_pos_map.find(lhs->result_name)->second < key_names_pos_map.find(rhs->result_name)->second; - }); -} - -void ActionsDAG::addAggregatesViaProjection(const Block & aggregates) -{ - for (const auto & aggregate : aggregates) - outputs.push_back(&addInput(aggregate)); -} - void ActionsDAG::addAliases(const NamesWithAliases & aliases) { std::unordered_map names_map; diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 3431daf99f2..96dd95c4e2f 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -245,12 +245,6 @@ public: const std::unordered_map & new_inputs, const NodeRawConstPtrs & required_outputs); - /// Reorder the output nodes using given position mapping. - void reorderAggregationKeysForProjection(const std::unordered_map & key_names_pos_map); - - /// Add aggregate columns to output nodes from projection - void addAggregatesViaProjection(const Block & aggregates); - bool hasArrayJoin() const; bool hasStatefulFunctions() const; bool trivial() const; /// If actions has no functions or array join. diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 1e0fdca7e07..07a1ae7d170 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -64,6 +64,10 @@ #include #include +#if USE_PROTOBUF +#include +#endif + #if USE_AWS_S3 #include #endif @@ -462,6 +466,20 @@ BlockIO InterpreterSystemQuery::execute() #if USE_AZURE_BLOB_STORAGE if (caches_to_drop.contains("AZURE")) StorageAzureBlob::getSchemaCache(getContext()).clear(); +#endif + break; + } + case Type::DROP_FORMAT_SCHEMA_CACHE: + { + getContext()->checkAccess(AccessType::SYSTEM_DROP_FORMAT_SCHEMA_CACHE); + std::unordered_set caches_to_drop; + if (query.schema_cache_format.empty()) + caches_to_drop = {"Protobuf"}; + else + caches_to_drop = {query.schema_cache_format}; +#if USE_PROTOBUF + if (caches_to_drop.contains("Protobuf")) + ProtobufSchemas::instance().clear(); #endif break; } @@ -1082,6 +1100,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() case Type::DROP_FILESYSTEM_CACHE: case Type::SYNC_FILESYSTEM_CACHE: case Type::DROP_SCHEMA_CACHE: + case Type::DROP_FORMAT_SCHEMA_CACHE: #if USE_AWS_S3 case Type::DROP_S3_CLIENT_CACHE: #endif diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index eb9c7e344a6..b96e05c1f9a 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -77,6 +77,7 @@ #include #include +#include namespace ProfileEvents { @@ -708,7 +709,7 @@ static std::tuple executeQueryImpl( ParserKQLStatement parser(end, settings.allow_settings_after_format_in_insert); /// TODO: parser should fail early when max_query_size limit is reached. - ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); + ast = parseKQLQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); } else if (settings.dialect == Dialect::prql && !internal) { diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index ecd7ecccb2e..267148ee62b 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -20,6 +20,8 @@ #include #include +#include + using namespace std::literals; @@ -80,6 +82,8 @@ namespace void markSecretArgument(size_t index, bool argument_is_named = false) { + if (index >= arguments->size()) + return; if (!result.count) { result.start = index; @@ -100,7 +104,8 @@ namespace /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) findMySQLFunctionSecretArguments(); } - else if ((function.name == "s3") || (function.name == "cosn") || (function.name == "oss")) + else if ((function.name == "s3") || (function.name == "cosn") || (function.name == "oss") || + (function.name == "deltaLake") || (function.name == "hudi") || (function.name == "iceberg")) { /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) findS3FunctionSecretArguments(/* is_cluster_function= */ false); @@ -150,41 +155,26 @@ namespace return; } - /// We're going to replace 'aws_secret_access_key' with '[HIDDEN'] for the following signatures: + /// We should check other arguments first because we don't need to do any replacement in case of + /// s3('url', NOSIGN, 'format' [, 'compression']) + /// s3('url', 'format', 'structure' [, 'compression']) + if ((url_arg_idx + 3 <= arguments->size()) && (arguments->size() <= url_arg_idx + 4)) + { + String second_arg; + if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) + { + if (boost::iequals(second_arg, "NOSIGN")) + return; /// The argument after 'url' is "NOSIGN". + + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: s3('url', 'format', ...) + } + } + + /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - - /// But we should check the number of arguments first because we don't need to do any replacements in case of - /// s3('url' [, 'format']) or s3Cluster('cluster_name', 'url' [, 'format']) - if (arguments->size() < url_arg_idx + 3) - return; - - if (arguments->size() >= url_arg_idx + 5) - { - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'structure', ...) - markSecretArgument(url_arg_idx + 2); - } - else - { - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - /// We need to distinguish that from s3('url', 'format', 'structure' [, 'compression_method']). - /// So we will check whether the argument after 'url' is a format. - String format; - if (!tryGetStringFromArgument(url_arg_idx + 1, &format, /* allow_identifier= */ false)) - { - /// We couldn't evaluate the argument after 'url' so we don't know whether it is a format or `aws_access_key_id`. - /// So it's safer to wipe the next argument just in case. - markSecretArgument(url_arg_idx + 2); /// Wipe either `aws_secret_access_key` or `structure`. - return; - } - - if (KnownFormatNames::instance().exists(format)) - return; /// The argument after 'url' is a format: s3('url', 'format', ...) - - /// The argument after 'url' is not a format so we do our replacement: - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) -> s3('url', 'aws_access_key_id', '[HIDDEN]', ...) - markSecretArgument(url_arg_idx + 2); - } + markSecretArgument(url_arg_idx + 2); } bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const @@ -351,7 +341,8 @@ namespace /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) findMySQLFunctionSecretArguments(); } - else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS")) + else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS") || + (engine_name == "DeltaLake") || (engine_name == "Hudi") || (engine_name == "Iceberg")) { /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) findS3TableEngineSecretArguments(); @@ -381,15 +372,29 @@ namespace return; } + /// We should check other arguments first because we don't need to do any replacement in case of + /// S3('url', NOSIGN, 'format' [, 'compression']) + /// S3('url', 'format', 'compression') + if ((3 <= arguments->size()) && (arguments->size() <= 4)) + { + String second_arg; + if (tryGetStringFromArgument(1, &second_arg)) + { + if (boost::iequals(second_arg, "NOSIGN")) + return; /// The argument after 'url' is "NOSIGN". + + if (arguments->size() == 3) + { + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: S3('url', 'format', ...) + } + } + } + /// We replace 'aws_secret_access_key' with '[HIDDEN'] for the following signatures: + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - - /// But we should check the number of arguments first because we don't need to do that replacements in case of - /// S3('url' [, 'format' [, 'compression']]) - if (arguments->size() < 4) - return; - markSecretArgument(2); } @@ -404,6 +409,11 @@ namespace /// PostgreSQL('host:port', 'database', 'user', 'password') findMySQLDatabaseSecretArguments(); } + else if (engine_name == "S3") + { + /// S3('url', 'access_key_id', 'secret_access_key') + findS3DatabaseSecretArguments(); + } } void findMySQLDatabaseSecretArguments() @@ -420,6 +430,20 @@ namespace } } + void findS3DatabaseSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// S3(named_collection, ..., secret_access_key = 'password', ...) + findSecretNamedArgument("secret_access_key", 1); + } + else + { + /// S3('url', 'access_key_id', 'secret_access_key') + markSecretArgument(2); + } + } + void findBackupNameSecretArguments() { const String & engine_name = function.name; diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index 76b409ba973..8e10fe0e6a9 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -211,6 +211,11 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, << (settings.hilite ? hilite_keyword : "") << " SECOND" << (settings.hilite ? hilite_none : ""); } + else if (type == Type::DROP_FORMAT_SCHEMA_CACHE) + { + if (!schema_cache_format.empty()) + settings.ostr << (settings.hilite ? hilite_none : "") << " FOR " << backQuoteIfNeed(schema_cache_format); + } else if (type == Type::DROP_FILESYSTEM_CACHE) { if (!filesystem_cache_name.empty()) diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index aafb68af6f3..cc06e0fdcb5 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -33,6 +33,7 @@ public: #endif DROP_FILESYSTEM_CACHE, DROP_SCHEMA_CACHE, + DROP_FORMAT_SCHEMA_CACHE, #if USE_AWS_S3 DROP_S3_CLIENT_CACHE, #endif @@ -120,6 +121,8 @@ public: String schema_cache_storage; + String schema_cache_format; + String fail_point_name; SyncReplicaMode sync_replica_mode = SyncReplicaMode::DEFAULT; diff --git a/src/Parsers/CMakeLists.txt b/src/Parsers/CMakeLists.txt index d74137f8a91..3bc1b3a981f 100644 --- a/src/Parsers/CMakeLists.txt +++ b/src/Parsers/CMakeLists.txt @@ -5,6 +5,7 @@ add_headers_and_sources(clickhouse_parsers ./Access) add_headers_and_sources(clickhouse_parsers ./MySQL) add_headers_and_sources(clickhouse_parsers ./Kusto) add_headers_and_sources(clickhouse_parsers ./PRQL) +add_headers_and_sources(clickhouse_parsers ./Kusto/KustoFunctions) add_library(clickhouse_parsers ${clickhouse_parsers_headers} ${clickhouse_parsers_sources}) target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io clickhouse_common_access string_utils) if (TARGET ch_rust::prql) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index ce3aa8abf38..2fcc6bff02b 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -43,7 +43,6 @@ #include - namespace DB { diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 3ebf5571eae..584d7c8a0ab 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -27,6 +27,7 @@ #include #include #include +#include using namespace std::literals; @@ -662,6 +663,26 @@ public: } else { + /// enable using subscript operator for kql_array_sort + if (cur_op.function_name == "arrayElement" && !operands.empty()) + { + auto* first_arg_as_node = operands.front()->as(); + if (first_arg_as_node) + { + if (first_arg_as_node->name == "kql_array_sort_asc" || first_arg_as_node->name == "kql_array_sort_desc") + { + cur_op.function_name = "tupleElement"; + cur_op.type = OperatorType::TupleElement; + } + else if (first_arg_as_node->name == "arrayElement" && !first_arg_as_node->arguments->children.empty()) + { + auto *arg_inside = first_arg_as_node->arguments->children[0]->as(); + if (arg_inside && (arg_inside->name == "kql_array_sort_asc" || arg_inside->name == "kql_array_sort_desc")) + first_arg_as_node->name = "tupleElement"; + } + } + } + function = makeASTFunction(cur_op); if (!popLastNOperands(function->children[0]->children, cur_op.arity)) @@ -2163,6 +2184,56 @@ private: bool if_permitted; }; +/// Layer for table function 'kql' +class KustoLayer : public Layer +{ +public: + + KustoLayer() : Layer(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true) {} + + bool parse(IParser::Pos & pos, Expected & expected, Action & /*action*/) override + { + /// kql(table|project ...) + /// 0. Parse the kql query + /// 1. Parse closing token + if (state == 0) + { + ASTPtr query; + --pos; + if (!ParserKQLTableFunction().parse(pos, query, expected)) + return false; + --pos; + pushResult(query); + + if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + return false; + + finished = true; + state = 1; + return true; + } + + if (state == 1) + { + if (ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + { + if (!mergeElement()) + return false; + + finished = true; + } + } + + return true; + } + +protected: + bool getResultImpl(ASTPtr & node) override + { + node = makeASTFunction("view", std::move(elements)); // reuse view function for kql + return true; + } +}; std::unique_ptr getFunctionLayer(ASTPtr identifier, bool is_table_function, bool allow_function_parameters_ = true) { @@ -2199,6 +2270,8 @@ std::unique_ptr getFunctionLayer(ASTPtr identifier, bool is_table_functio return std::make_unique(false); else if (function_name_lowercase == "viewifpermitted") return std::make_unique(true); + else if (function_name_lowercase == "kql") + return std::make_unique(); } if (function_name == "tuple") @@ -2454,7 +2527,7 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos if (layers.front()->is_table_function) { - if (typeid_cast(layers.back().get())) + if (typeid_cast(layers.back().get()) || typeid_cast(layers.back().get())) { if (identifier_parser.parse(pos, tmp, expected) && ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)) @@ -2592,6 +2665,7 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos } else if (pos->type == TokenType::OpeningRoundBracket) { + if (subquery_parser.parse(pos, tmp, expected)) { layers.back()->pushOperand(std::move(tmp)); diff --git a/src/Parsers/Kusto/KQL_ReleaseNote.md b/src/Parsers/Kusto/KQL_ReleaseNote.md new file mode 100644 index 00000000000..bea1a627129 --- /dev/null +++ b/src/Parsers/Kusto/KQL_ReleaseNote.md @@ -0,0 +1,996 @@ +## KQL implemented features + +# October 9, 2022 + +## operator +- [distinct](https://learn.microsoft.com/en-us/azure/data-explorer/kusto/query/distinctoperator) + `Customers | distinct *` + `Customers | distinct Occupation` + `Customers | distinct Occupation, Education` + `Customers | where Age <30 | distinct Occupation, Education` + `Customers | where Age <30 | order by Age| distinct Occupation, Education` + +## String functions +- [reverse](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/reversefunction) + `print reverse(123)` + `print reverse(123.34)` + `print reverse('clickhouse')` + `print reverse(3h)` + `print reverse(datetime(2017-1-1 12:23:34))` + +- [parse_command_line](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parse-command-line) + `print parse_command_line('echo \"hello world!\" print$?', \"Windows\")` + +- [parse_csv](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parsecsvfunction) + `print result=parse_csv('aa,b,cc')` + `print result_multi_record=parse_csv('record1,a,b,c\nrecord2,x,y,z')` + +- [parse_json](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parsejsonfunction) + `print parse_json( dynamic([1, 2, 3]))` + `print parse_json('{"a":123.5, "b":"{\\"c\\":456}"}')` + +- [extract_json](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractjsonfunction) + `print extract_json( "$.a" , '{"a":123, "b":"{\\"c\\":456}"}' , typeof(int))` + +- [parse_version](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parse-versionfunction) + `print parse_version('1')` + `print parse_version('1.2.3.40')` + +## Bug fixed +- [correct array index in expression](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1474) + array index should start with 0 +- [Summarize should generate alias or use correct columns](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1303) + - if bin is used , the column should be in select list if no alias include + - if no column included in aggregate functions, ( like count() ), should has alias with fun name + '_',e.g count_ + - if column name included in aggregate functions, should have fun name + "_" + column name , like count(Age) -> count_Age + - if argument of an aggregate functions is an exprision, Columns1 ... Columnsn should be used as alias + ``` + Customers | summarize count() by bin(Age, 10) + ┌─Age─┬─count_─┐ + │ 40 │ 2 │ + │ 20 │ 6 │ + │ 30 │ 4 │ + └─────┴────────┘ + Customers | summarize count(Age) by bin(Age, 10) + ┌─Age─┬─count_Age─┐ + │ 40 │ 2 │ + │ 20 │ 6 │ + │ 30 │ 4 │ + └─────┴───────────┘ + Customers | summarize count(Age+1) by bin(Age+1, 10) + ┌─Columns1─┬─count_─┐ + │ 40 │ 2 │ + │ 20 │ 6 │ + │ 30 │ 4 │ + └──────────┴────────┘ + ``` +- [extend doesn't replace existing columns](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1246) + +- [throw exception if use quoted string as alias](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1470) + +- [repeat() doesn't work with count argument as negative value](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1368) + +- [substring() doesn't work right with negative offsets](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1336) +- [endofmonth() doesn't return correct result](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1370) + +- [split() outputs array instead of string](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1343) + +- [split() returns empty string when arg goes out of bound](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1328) + +- [split() doesn't work with negative index](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1325) + + +# September 26, 2022 +## Bug fixed : +["select * from kql" results in syntax error](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1119) +[Parsing ipv4 with arrayStringConcat throws exception](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1259) +[CH Client crashes on invalid function name](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1266) +[extract() doesn't work right with 4th argument i.e typeof()](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1327) +[parse_ipv6_mask return incorrect results](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1050) +[timespan returns wrong output in seconds](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1275) +[timespan doesn't work for nanoseconds and tick](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1298) +[totimespan() doesn't work for nanoseconds and tick timespan unit](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1301) +[data types should throw exception in certain cases](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1112) +[decimal does not support scientific notation](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1197) +[extend statement causes client core dumping](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1260) +[extend crashes with array sorting](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1247) +[Core dump happens when WHERE keyword doesn't follow field name](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1335) +[Null values are missing in the result of `make_list_with_nulls'](https://github.ibm.com/ClickHouse/issue-repo/issues/1009) +[trim functions use non-unique aliases](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1111) +[format_ipv4_mask returns incorrect mask value](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1039) + +# September 12, 2022 +## Extend operator +https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extendoperator +`T | extend T | extend duration = endTime - startTime` +`T | project endTime, startTime | extend duration = endTime - startTime` +## Array functions +- [array_reverse](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/array-reverse-function) + `print array_reverse(dynamic(["this", "is", "an", "example"])) == dynamic(["example","an","is","this"])` + +- [array_rotate_left](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/array_rotate_leftfunction) + `print array_rotate_left(dynamic([1,2,3,4,5]), 2) == dynamic([3,4,5,1,2])` + `print array_rotate_left(dynamic([1,2,3,4,5]), -2) == dynamic([4,5,1,2,3])` + +- [array_rotate_right](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/array_rotate_rightfunction) + `print array_rotate_right(dynamic([1,2,3,4,5]), -2) == dynamic([3,4,5,1,2])` + `print array_rotate_right(dynamic([1,2,3,4,5]), 2) == dynamic([4,5,1,2,3])` + +- [array_shift_left](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/array_shift_leftfunction) + `print array_shift_left(dynamic([1,2,3,4,5]), 2) == dynamic([3,4,5,null,null])` + `print array_shift_left(dynamic([1,2,3,4,5]), -2) == dynamic([null,null,1,2,3])` + `print array_shift_left(dynamic([1,2,3,4,5]), 2, -1) == dynamic([3,4,5,-1,-1])` + `print array_shift_left(dynamic(['a', 'b', 'c']), 2) == dynamic(['c','',''])` + +- [array_shift_right](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/array_shift_rightfunction) + `print array_shift_right(dynamic([1,2,3,4,5]), -2) == dynamic([3,4,5,null,null])` + `print array_shift_right(dynamic([1,2,3,4,5]), 2) == dynamic([null,null,1,2,3])` + `print array_shift_right(dynamic([1,2,3,4,5]), -2, -1) == dynamic([3,4,5,-1,-1])` + `print array_shift_right(dynamic(['a', 'b', 'c']), -2) == dynamic(['c','',''])` + +- [pack_array](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/packarrayfunction) + `print x = 1, y = x * 2, z = y * 2, pack_array(x,y,z)` + + Please note that only arrays of elements of the same type may be created at this time. The underlying reasons are explained under the release note section of the `dynamic` data type. + +- [repeat](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/repeatfunction) + `print repeat(1, 0) == dynamic([])` + `print repeat(1, 3) == dynamic([1, 1, 1])` + `print repeat("asd", 3) == dynamic(['asd', 'asd', 'asd'])` + `print repeat(timespan(1d), 3) == dynamic([86400, 86400, 86400])` + `print repeat(true, 3) == dynamic([true, true, true])` + +- [zip](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/zipfunction) + `print zip(dynamic([1,3,5]), dynamic([2,4,6]))` + + Please note that only arrays of the same type are supported in our current implementation. The underlying reasons are explained under the release note section of the `dynamic` data type. + +## Data types + - [dynamic](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/dynamic) + `print isnull(dynamic(null))` + `print dynamic(1) == 1` + `print dynamic(timespan(1d)) == 86400` + `print dynamic([1, 2, 3])` + `print dynamic([[1], [2], [3]])` + `print dynamic(['a', "b", 'c'])` + + According to the KQL specifications `dynamic` is a literal, which means that no function calls are permitted. Expressions producing literals such as `datetime` and `timespan` and their aliases (ie. `date` and `time`, respectively) along with nested `dynamic` literals are allowed. + + Please note that our current implementation supports only scalars and arrays made up of elements of the same type. Support for mixed types and property bags is deferred for now, based on our understanding of the required effort and discussion with representatives of the QRadar team. + +## Mathematical functions + - [isnan](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/isnanfunction) + `print isnan(double(nan)) == true` + `print isnan(4.2) == false` + `print isnan(4) == false` + `print isnan(real(+inf)) == false` + +## Set functions +Please note that functions returning arrays with set semantics may return them in any particular order, which may be subject to change in the future. + + - [jaccard_index](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/jaccard-index-function) + `print jaccard_index(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3, 4, 4, 4])) == 0.75` + `print jaccard_index(dynamic([1, 2, 3]), dynamic([])) == 0` + `print jaccard_index(dynamic([]), dynamic([1, 2, 3, 4])) == 0` + `print isnan(jaccard_index(dynamic([]), dynamic([])))` + `print jaccard_index(dynamic([1, 2, 3]), dynamic([4, 5, 6, 7])) == 0` + `print jaccard_index(dynamic(['a', 's', 'd']), dynamic(['f', 'd', 's', 'a'])) == 0.75` + `print jaccard_index(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])) == 0.25` + + - [set_difference](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/setdifferencefunction) + `print set_difference(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])) == dynamic([])` + `print array_sort_asc(set_difference(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[1] == dynamic([4, 5, 6])` + `print set_difference(dynamic([4]), dynamic([1, 2, 3])) == dynamic([4])` + `print array_sort_asc(set_difference(dynamic([1, 2, 3, 4, 5]), dynamic([5]), dynamic([2, 4])))[1] == dynamic([1, 3])` + `print array_sort_asc(set_difference(dynamic([1, 2, 3]), dynamic([])))[1] == dynamic([1, 2, 3])` + `print array_sort_asc(set_difference(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])))[1] == dynamic(['d', 's'])` + `print array_sort_asc(set_difference(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])))[1] == dynamic(['Chewbacca', 'Han Solo'])` + + - [set_has_element](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/sethaselementfunction) + `print set_has_element(dynamic(["this", "is", "an", "example"]), "example") == true` + `print set_has_element(dynamic(["this", "is", "an", "example"]), "test") == false` + `print set_has_element(dynamic([1, 2, 3]), 2) == true` + `print set_has_element(dynamic([1, 2, 3, 4.2]), 4) == false` + + - [set_intersect](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/setintersectfunction) + `print array_sort_asc(set_intersect(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])))[1] == dynamic([1, 2, 3])` + `print array_sort_asc(set_intersect(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[1] == dynamic([1, 2, 3])` + `print set_intersect(dynamic([4]), dynamic([1, 2, 3])) == dynamic([])` + `print set_intersect(dynamic([1, 2, 3, 4, 5]), dynamic([1, 3, 5]), dynamic([2, 5])) == dynamic([5])` + `print set_intersect(dynamic([1, 2, 3]), dynamic([])) == dynamic([])` + `print set_intersect(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])) == dynamic(['a'])` + `print set_intersect(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])) == dynamic(['Darth Vader'])` + + - [set_union](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/setunionfunction) + `print array_sort_asc(set_union(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])))[1] == dynamic([1, 2, 3])` + `print array_sort_asc(set_union(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[1] == dynamic([1, 2, 3, 4, 5, 6])` + `print array_sort_asc(set_union(dynamic([4]), dynamic([1, 2, 3])))[1] == dynamic([1, 2, 3, 4])` + `print array_sort_asc(set_union(dynamic([1, 3, 4]), dynamic([5]), dynamic([2, 4])))[1] == dynamic([1, 2, 3, 4, 5])` + `print array_sort_asc(set_union(dynamic([1, 2, 3]), dynamic([])))[1] == dynamic([1, 2, 3])` + `print array_sort_asc(set_union(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])))[1] == dynamic(['a', 'd', 'f', 's'])` + `print array_sort_asc(set_union(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])))[1] == dynamic(['Chewbacca', 'Darth Sidious', 'Darth Vader', 'Han Solo'])` + +# August 29, 2022 + +## **mv-expand operator** +https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/mvexpandoperator +Note: **expand on array columns only** +- test cases + ``` + CREATE TABLE T + ( + a UInt8, + b Array(String), + c Array(Int8), + d Array(Int8) + ) ENGINE = Memory; + + INSERT INTO T VALUES (1, ['Salmon', 'Steak','Chicken'],[1,2,3,4],[5,6,7,8]) + + T | mv-expand c + T | mv-expand c, d + T | mv-expand b | mv-expand c + T | mv-expand c to typeof(bool) + T | mv-expand with_itemindex=index b, c, d + T | mv-expand array_concat(c,d) + T | mv-expand x = c, y = d + T | mv-expand xy = array_concat(c, d) + T | mv-expand with_itemindex=index c,d to typeof(bool) + ``` + +## **make-series operator** +https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/make-seriesoperator + +- test case make-series on datetime column + ``` + CREATE TABLE T + ( + Supplier Nullable(String), + Fruit String , + Price Float64, + Purchase Date + ) ENGINE = Memory; + + INSERT INTO T VALUES ('Aldi','Apple',4,'2016-09-10'); + INSERT INTO T VALUES ('Costco','Apple',2,'2016-09-11'); + INSERT INTO T VALUES ('Aldi','Apple',6,'2016-09-10'); + INSERT INTO T VALUES ('Costco','Snargaluff',100,'2016-09-12'); + INSERT INTO T VALUES ('Aldi','Apple',7,'2016-09-12'); + INSERT INTO T VALUES ('Aldi','Snargaluff',400,'2016-09-11'); + INSERT INTO T VALUES ('Costco','Snargaluff',104,'2016-09-12'); + INSERT INTO T VALUES ('Aldi','Apple',5,'2016-09-12'); + INSERT INTO T VALUES ('Aldi','Snargaluff',600,'2016-09-11'); + INSERT INTO T VALUES ('Costco','Snargaluff',200,'2016-09-10'); + ``` + Have from and to + ``` + T | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) to datetime(2016-09-13) step 1d by Supplier, Fruit + ``` + Has from , without to + ``` + T | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) step 1d by Supplier, Fruit + ``` + Without from , has to + ``` + T | make-series PriceAvg = avg(Price) default=0 on Purchase to datetime(2016-09-13) step 1d by Supplier, Fruit + ``` + Without from , without to + ``` + T | make-series PriceAvg = avg(Price) default=0 on Purchase step 1d by Supplier, Fruit + ``` + Without by clause + ``` + T | make-series PriceAvg = avg(Price) default=0 on Purchase step 1d + ``` + Without aggregation alias + ``` + T | make-series avg(Price) default=0 on Purchase step 1d by Supplier, Fruit + ``` + Has group expression alias + ``` + T | make-series avg(Price) default=0 on Purchase step 1d by Supplier_Name = Supplier, Fruit + ``` + Use different step value + ``` + T | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) to datetime(2016-09-13) step 3d by Supplier, Fruit + ``` +- test case make-series on numeric column + ``` + CREATE TABLE T2 + ( + Supplier Nullable(String), + Fruit String , + Price Int32, + Purchase Int32 + ) ENGINE = Memory; + + INSERT INTO T2 VALUES ('Aldi','Apple',4,10); + INSERT INTO T2 VALUES ('Costco','Apple',2,11); + INSERT INTO T2 VALUES ('Aldi','Apple',6,10); + INSERT INTO T2 VALUES ('Costco','Snargaluff',100,12); + INSERT INTO T2 VALUES ('Aldi','Apple',7,12); + INSERT INTO T2 VALUES ('Aldi','Snargaluff',400,11); + INSERT INTO T2 VALUES ('Costco','Snargaluff',104,12); + INSERT INTO T2 VALUES ('Aldi','Apple',5,12); + INSERT INTO T2 VALUES ('Aldi','Snargaluff',600,11); + INSERT INTO T2 VALUES ('Costco','Snargaluff',200,10); + ``` + Have from and to + ``` + T2 | make-series PriceAvg=avg(Price) default=0 on Purchase from 10 to 15 step 1.0 by Supplier, Fruit; + ``` + Has from , without to + ``` + T2 | make-series PriceAvg=avg(Price) default=0 on Purchase from 10 step 1.0 by Supplier, Fruit; + ``` + Without from , has to + ``` + T2 | make-series PriceAvg=avg(Price) default=0 on Purchase to 18 step 4.0 by Supplier, Fruit; + ``` + Without from , without to + ``` + T2 | make-series PriceAvg=avg(Price) default=0 on Purchase step 2.0 by Supplier, Fruit; + ``` + Without by clause + ``` + T2 | make-series PriceAvg=avg(Price) default=0 on Purchase step 2.0; + ``` + +## Aggregate Functions +- [bin](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binfunction) + `print bin(4.5, 1)` + `print bin(time(16d), 7d)` + `print bin(datetime(1970-05-11 13:45:07), 1d)` +- [stdev](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/stdev-aggfunction) + `Customers | summarize t = stdev(Age) by FirstName` + +- [stdevif](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/stdevif-aggfunction) + `Customers | summarize t = stdevif(Age, Age < 10) by FirstName` + +- [binary_all_and](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-all-and-aggfunction) + `Customers | summarize t = binary_all_and(Age) by FirstName` + +- [binary_all_or](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-all-or-aggfunction) + `Customers | summarize t = binary_all_or(Age) by FirstName` + +- [binary_all_xor](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-all-xor-aggfunction) + `Customers | summarize t = binary_all_xor(Age) by FirstName` + +- [percentiles](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/percentiles-aggfunction) + `Customers | summarize percentiles(Age, 30, 40, 50, 60, 70) by FirstName` + +- [percentilesw](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/percentiles-aggfunction) + `DataTable | summarize t = percentilesw(Bucket, Frequency, 50, 75, 99.9)` + +- [percentile](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/percentiles-aggfunction) + `Customers | summarize t = percentile(Age, 50) by FirstName` + +- [percentilew](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/percentiles-aggfunction) + `DataTable | summarize t = percentilew(Bucket, Frequency, 50)` + +## Dynamic functions +- [array_sort_asc](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arraysortascfunction) + **Only support the constant dynamic array.** + **Returns an array. So, each element of the input has to be of same datatype.** + `print t = array_sort_asc(dynamic([null, 'd', 'a', 'c', 'c']))` + `print t = array_sort_asc(dynamic([4, 1, 3, 2]))` + `print t = array_sort_asc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))` + `print t = array_sort_asc(dynamic(['q', 'p', 'r']), dynamic(['clickhouse','hello', 'world']))` + `print t = array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , false)` + `print t = array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2)` + `print t = array_sort_asc( dynamic([null, 'd', null, null, 'a', 'c', 'c', null, null, null]) , false)` + `print t = array_sort_asc( dynamic([null, null, null]) , false)` + `print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]), 1 > 2)` + `print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30, 50, 3]), 1 > 2)` + +- [array_sort_desc](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arraysortdescfunction) **(only support the constant dynamic array)** + + `print t = array_sort_desc(dynamic([null, 'd', 'a', 'c', 'c']))` + `print t = array_sort_desc(dynamic([4, 1, 3, 2]))` + `print t = array_sort_desc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))` + `print t = array_sort_desc(dynamic(['q', 'p', 'r']), dynamic(['clickhouse','hello', 'world']))` + `print t = array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , false)` + `print t = array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2)` + `print t = array_sort_desc( dynamic([null, 'd', null, null, 'a', 'c', 'c', null, null, null]) , false)` + `print t = array_sort_desc( dynamic([null, null, null]) , false)` + `print t = array_sort_desc(dynamic([2, 1, null, 3]), dynamic([20, 10, 40, 30]), 1 > 2)` + `print t = array_sort_desc(dynamic([2, 1, null,3, null]), dynamic([20, 10, 40, 30, 50, 3]), 1 > 2)` + +- [array_concat](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arrayconcatfunction) + `print array_concat(dynamic([1, 2, 3]), dynamic([4, 5]), dynamic([6, 7, 8, 9])) == dynamic([1, 2, 3, 4, 5, 6, 7, 8, 9])` + +- [array_iff / array_iif](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arrayifffunction) + `print array_iif(dynamic([true, false, true]), dynamic([1, 2, 3]), dynamic([4, 5, 6])) == dynamic([1, 5, 3])` + `print array_iif(dynamic([true, false, true]), dynamic([1, 2, 3, 4]), dynamic([4, 5, 6])) == dynamic([1, 5, 3])` + `print array_iif(dynamic([true, false, true, false]), dynamic([1, 2, 3, 4]), dynamic([4, 5, 6])) == dynamic([1, 5, 3, null])` + `print array_iif(dynamic([1, 0, -1, 44, 0]), dynamic([1, 2, 3, 4]), dynamic([4, 5, 6])) == dynamic([1, 5, 3, 4, null])` + +- [array_slice](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arrayslicefunction) + `print array_slice(dynamic([1,2,3]), 1, 2) == dynamic([2, 3])` + `print array_slice(dynamic([1,2,3,4,5]), 2, -1) == dynamic([3, 4, 5])` + `print array_slice(dynamic([1,2,3,4,5]), -3, -2) == dynamic([3, 4])` + +- [array_split](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arraysplitfunction) + `print array_split(dynamic([1,2,3,4,5]), 2) == dynamic([[1,2],[3,4,5]])` + `print array_split(dynamic([1,2,3,4,5]), dynamic([1,3])) == dynamic([[1],[2,3],[4,5]])` + +## DateTimeFunctions + +- [ago](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/agofunction) + `print ago(2h)` + +- [endofday](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/endofdayfunction) + `print endofday(datetime(2017-01-01 10:10:17), -1)` + `print endofday(datetime(2017-01-01 10:10:17), 1)` + `print endofday(datetime(2017-01-01 10:10:17))` + +- [endofmonth](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/endofmonthfunction) + `print endofmonth(datetime(2017-01-01 10:10:17), -1)` + `print endofmonth(datetime(2017-01-01 10:10:17), 1)` + `print endofmonth(datetime(2017-01-01 10:10:17))` + +- [endofweek](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/endofweekfunction) + `print endofweek(datetime(2017-01-01 10:10:17), 1)` + `print endofweek(datetime(2017-01-01 10:10:17), -1)` + `print endofweek(datetime(2017-01-01 10:10:17))` + +- [endofyear](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/endofyearfunction) + `print endofyear(datetime(2017-01-01 10:10:17), -1)` + `print endofyear(datetime(2017-01-01 10:10:17), 1)` + `print endofyear(datetime(2017-01-01 10:10:17))` + +- [make_datetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/make-datetimefunction) + `print make_datetime(2017,10,01)` + `print make_datetime(2017,10,01,12,10)` + `print make_datetime(2017,10,01,12,11,0.1234567)` + +- [datetime_diff](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/datetime-difffunction) + `print datetime_diff('year',datetime(2017-01-01),datetime(2000-12-31))` + `print datetime_diff('quarter',datetime(2017-07-01),datetime(2017-03-30))` + `print datetime_diff('minute',datetime(2017-10-30 23:05:01),datetime(2017-10-30 23:00:59))` + +- [unixtime_microseconds_todatetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/unixtime-microseconds-todatetimefunction) + `print unixtime_microseconds_todatetime(1546300800000000)` + +- [unixtime_milliseconds_todatetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/unixtime-milliseconds-todatetimefunction) + `print unixtime_milliseconds_todatetime(1546300800000)` + +- [unixtime_nanoseconds_todatetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/unixtime-nanoseconds-todatetimefunction) + `print unixtime_nanoseconds_todatetime(1546300800000000000)` + +- [datetime_part](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/datetime-partfunction) + `print datetime_part('day', datetime(2017-10-30 01:02:03.7654321))` + +- [datetime_add](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/datetime-addfunction) + `print datetime_add('day',1,datetime(2017-10-30 01:02:03.7654321))` + +- [format_timespan](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/format-timespanfunction) + `print format_timespan(time(1d), 'd-[hh:mm:ss]')` + `print format_timespan(time('12:30:55.123'), 'ddddd-[hh:mm:ss.ffff]')` + +- [format_datetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/format-datetimefunction) + `print format_datetime(todatetime('2009-06-15T13:45:30.6175425'), 'yy-M-dd [H:mm:ss.fff]')` + `print format_datetime(datetime(2015-12-14 02:03:04.12345), 'y-M-d h:m:s tt')` + +- [todatetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/todatetimefunction) + `print todatetime('2014-05-25T08:20:03.123456Z')` + `print todatetime('2014-05-25 20:03.123')` + +- [totimespan] (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/totimespanfunction) + ` print totimespan('0.01:34:23')` + `print totimespan(1d)` + +# August 15, 2022 + **double quote support** + ``print res = strcat("double ","quote")`` +## Aggregate functions + - [bin_at](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binatfunction) + `print res = bin_at(6.5, 2.5, 7)` + `print res = bin_at(1h, 1d, 12h)` + `print res = bin_at(datetime(2017-05-15 10:20:00.0), 1d, datetime(1970-01-01 12:00:00.0))` + `print res = bin_at(datetime(2017-05-17 10:20:00.0), 7d, datetime(2017-06-04 00:00:00.0))` + + - [array_index_of](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arrayindexoffunction) + *Supports only basic lookup. Do not support start_index, length and occurrence* + `print output = array_index_of(dynamic(['John', 'Denver', 'Bob', 'Marley']), 'Marley')` + `print output = array_index_of(dynamic([1, 2, 3]), 2)` + - [array_sum](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/array-sum-function) + `print output = array_sum(dynamic([2, 5, 3]))` + `print output = array_sum(dynamic([2.5, 5.5, 3]))` + - [array_length](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arraylengthfunction) + `print output = array_length(dynamic(['John', 'Denver', 'Bob', 'Marley']))` + `print output = array_length(dynamic([1, 2, 3]))` + +## Conversion +- [tobool / toboolean](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/toboolfunction) + `print tobool(true) == true` + `print toboolean(false) == false` + `print tobool(0) == false` + `print toboolean(19819823) == true` + `print tobool(-2) == true` + `print isnull(toboolean('a'))` + `print tobool('true') == true` + `print toboolean('false') == false` + +- [todouble / toreal](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/todoublefunction) + `print todouble(4) == 4` + `print toreal(4.2) == 4.2` + `print isnull(todouble('a'))` + `print toreal('-0.3') == -0.3` + +- [toint](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/tointfunction) + `print isnull(toint('a'))` + `print toint(4) == 4` + `print toint('4') == 4` + `print isnull(toint(4.2))` + +- [tostring](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/tostringfunction) + `print tostring(123) == '123'` + `print tostring('asd') == 'asd'` + +## Data Types + - [dynamic](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/dynamic) + *Supports only 1D array* + `print output = dynamic(['a', 'b', 'c'])` + `print output = dynamic([1, 2, 3])` + +- [bool,boolean](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/bool) + `print bool(1)` + `print boolean(0)` + +- [datetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/datetime) + `print datetime(2015-12-31 23:59:59.9)` + `print datetime('2015-12-31 23:59:59.9')` + `print datetime("2015-12-31:)` + +- [guid](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/guid) + `print guid(74be27de-1e4e-49d9-b579-fe0b331d3642)` + `print guid('74be27de-1e4e-49d9-b579-fe0b331d3642')` + `print guid('74be27de1e4e49d9b579fe0b331d3642')` + +- [int](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/int) + `print int(1)` + +- [long](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/long) + `print long(16)` + +- [real](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/real) + `print real(1)` + +- [timespan ,time](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/timespan) + **Note** the timespan is used for calculating datatime, so the output is in seconds. e.g. time(1h) = 3600 + `print 1d` + `print 30m` + `print time('0.12:34:56.7')` + `print time(2h)` + `print timespan(2h)` + + +## StringFunctions + +- [base64_encode_fromguid](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/base64-encode-fromguid-function) +`print Quine = base64_encode_fromguid('ae3133f2-6e22-49ae-b06a-16e6a9b212eb')` +- [base64_decode_toarray](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/base64_decode_toarrayfunction) +`print base64_decode_toarray('S3VzdG8=')` +- [base64_decode_toguid](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/base64-decode-toguid-function) +`print base64_decode_toguid('YWUzMTMzZjItNmUyMi00OWFlLWIwNmEtMTZlNmE5YjIxMmVi')` +- [replace_regex](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/replace-regex-function) +`print replace_regex('Hello, World!', '.', '\\0\\0')` +- [has_any_index](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-any-index-function) +`print idx = has_any_index('this is an example', dynamic(['this', 'example']))` +- [translate](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/translatefunction) +`print translate('krasp', 'otsku', 'spark')` +- [trim](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/trimfunction) +`print trim('--', '--https://bing.com--')` +- [trim_end](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/trimendfunction) +`print trim_end('.com', 'bing.com')` +- [trim_start](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/trimstartfunction) +`print trim_start('[^\\w]+', strcat('- ','Te st1','// $'))` + +## DateTimeFunctions +- [startofyear](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/startofyearfunction) + `print startofyear(datetime(2017-01-01 10:10:17), -1)` + `print startofyear(datetime(2017-01-01 10:10:17), 0)` + `print startofyear(datetime(2017-01-01 10:10:17), 1)` +- [weekofyear](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/weekofyearfunction) + `print week_of_year(datetime(2020-12-31))` + `print week_of_year(datetime(2020-06-15))` + `print week_of_year(datetime(1970-01-01))` + `print week_of_year(datetime(2000-01-01))` + +- [startofweek](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/startofweekfunction) + `print startofweek(datetime(2017-01-01 10:10:17), -1)` + `print startofweek(datetime(2017-01-01 10:10:17), 0)` + `print startofweek(datetime(2017-01-01 10:10:17), 1)` + +- [startofmonth](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/startofmonthfunction) + `print startofmonth(datetime(2017-01-01 10:10:17), -1)` + `print startofmonth(datetime(2017-01-01 10:10:17), 0)` + `print startofmonth(datetime(2017-01-01 10:10:17), 1)` + +- [startofday](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/startofdayfunction) + `print startofday(datetime(2017-01-01 10:10:17), -1)` + `print startofday(datetime(2017-01-01 10:10:17), 0)` + `print startofday(datetime(2017-01-01 10:10:17), 1)` + +- [monthofyear](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/monthofyearfunction) + `print monthofyear(datetime("2015-12-14"))` + +- [hourofday](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/hourofdayfunction) + `print hourofday(datetime(2015-12-14 18:54:00))` + +- [getyear](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/getyearfunction) + `print getyear(datetime(2015-10-12))` + +- [getmonth](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/getmonthfunction) + `print getmonth(datetime(2015-10-12))` + +- [dayofyear](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/dayofyearfunction) + `print dayofyear(datetime(2015-12-14))` + +- [dayofmonth](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/dayofmonthfunction) + `print (datetime(2015-12-14))` + +- [unixtime_seconds_todatetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/unixtime-seconds-todatetimefunction) + `print unixtime_seconds_todatetime(1546300800)` + +- [dayofweek](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/dayofweekfunction) + `print dayofweek(datetime(2015-12-20))` + +- [now](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/nowfunction) + `print now()` + `print now(2d)` + `print now(-2h)` + `print now(5microseconds)` + `print now(5seconds)` + `print now(6minutes)` + `print now(-2d) ` + `print now(time(1d))` + + +## Binary functions +- [binary_and](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-andfunction) + `print binary_and(15, 3) == 3` + `print binary_and(1, 2) == 0` +- [binary_not](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-notfunction) + `print binary_not(1) == -2` +- [binary_or](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-orfunction) + `print binary_or(3, 8) == 11` + `print binary_or(1, 2) == 3` +- [binary_shift_left](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-shift-leftfunction) + `print binary_shift_left(1, 1) == 2` + `print binary_shift_left(1, 64) == 1` +- [binary_shift_right](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-shift-rightfunction) + `print binary_shift_right(1, 1) == 0` + `print binary_shift_right(1, 64) == 1` +- [binary_xor](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-xorfunction) + `print binary_xor(1, 3) == 2` +- [bitset_count_ones](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/bitset-count-onesfunction) + `print bitset_count_ones(42) == 3` + +## IP functions +- [format_ipv4](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/format-ipv4-function) + `print format_ipv4('192.168.1.255', 24) == '192.168.1.0'` + `print format_ipv4(3232236031, 24) == '192.168.1.0'` +- [format_ipv4_mask](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/format-ipv4-mask-function) + `print format_ipv4_mask('192.168.1.255', 24) == '192.168.1.0/24'` + `print format_ipv4_mask(3232236031, 24) == '192.168.1.0/24'` +- [ipv4_compare](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-comparefunction) + `print ipv4_compare('127.0.0.1', '127.0.0.1') == 0` + `print ipv4_compare('192.168.1.1', '192.168.1.255') < 0` + `print ipv4_compare('192.168.1.1/24', '192.168.1.255/24') == 0` + `print ipv4_compare('192.168.1.1', '192.168.1.255', 24) == 0` +- [ipv4_is_match](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-is-matchfunction) + `print ipv4_is_match('127.0.0.1', '127.0.0.1') == true` + `print ipv4_is_match('192.168.1.1', '192.168.1.255') == false` + `print ipv4_is_match('192.168.1.1/24', '192.168.1.255/24') == true` + `print ipv4_is_match('192.168.1.1', '192.168.1.255', 24) == true` +- [ipv6_compare](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv6-comparefunction) + `print ipv6_compare('::ffff:7f00:1', '127.0.0.1') == 0` + `print ipv6_compare('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995') < 0` + `print ipv6_compare('192.168.1.1/24', '192.168.1.255/24') == 0` + `print ipv6_compare('fe80::85d:e82c:9446:7994/127', 'fe80::85d:e82c:9446:7995/127') == 0` + `print ipv6_compare('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995', 127) == 0` +- [ipv6_is_match](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv6-is-matchfunction) + `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` +- [parse_ipv4_mask](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parse-ipv4-maskfunction) + `print parse_ipv4_mask('127.0.0.1', 24) == 2130706432` + `print parse_ipv4_mask('192.1.168.2', 31) == 3221334018` + `print parse_ipv4_mask('192.1.168.3', 31) == 3221334018` + `print parse_ipv4_mask('127.2.3.4', 32) == 2130838276` +- [parse_ipv6_mask](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parse-ipv6-maskfunction) + `print parse_ipv6_mask('127.0.0.1', 24) == '0000:0000:0000:0000:0000:ffff:7f00:0000'` + `print parse_ipv6_mask('fe80::85d:e82c:9446:7994', 120) == 'fe80:0000:0000:0000:085d:e82c:9446:7900'` + +# August 1, 2022 + +**The config setting to allow modify dialect setting**. + - Set dialect setting in server configuration XML at user level(` users.xml `). This sets the ` dialect ` at server startup and CH will do query parsing for all users with ` default ` profile according to dialect value. + + For example: + ` + + + random + kusto + ` + + - Query can be executed with HTTP client as below once dialect is set in users.xml + ` echo "KQL query" | curl -sS "http://localhost:8123/?" --data-binary @- ` + + - To execute the query using clickhouse-client , Update clickhouse-client.xml as below and connect clickhouse-client with --config-file option (` clickhouse-client --config-file= `) + + ` + kusto + ` + + OR + pass dialect setting with '--'. For example : + ` clickhouse-client --dialect='kusto' -q "KQL query" ` + +- **strcmp** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/strcmpfunction) + `print strcmp('abc','ABC')` + +- **parse_url** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parseurlfunction) + `print Result = parse_url('scheme://username:password@www.google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment')` + +- **parse_urlquery** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parseurlqueryfunction) + `print Result = parse_urlquery('k1=v1&k2=v2&k3=v3')` + +- **print operator** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/printoperator) + `print x=1, s=strcat('Hello', ', ', 'World!')` + +- **Aggregate Functions:** + - [make_list()](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/makelist-aggfunction) + `Customers | summarize t = make_list(FirstName) by FirstName` + `Customers | summarize t = make_list(FirstName, 10) by FirstName` + - [make_list_if()](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/makelistif-aggfunction) + `Customers | summarize t = make_list_if(FirstName, Age > 10) by FirstName` + `Customers | summarize t = make_list_if(FirstName, Age > 10, 10) by FirstName` + - [make_list_with_nulls()](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/make-list-with-nulls-aggfunction) + `Customers | summarize t = make_list_with_nulls(Age) by FirstName` + - [make_set()](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/makeset-aggfunction) + `Customers | summarize t = make_set(FirstName) by FirstName` + `Customers | summarize t = make_set(FirstName, 10) by FirstName` + - [make_set_if()](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/makesetif-aggfunction) + `Customers | summarize t = make_set_if(FirstName, Age > 10) by FirstName` + `Customers | summarize t = make_set_if(FirstName, Age > 10, 10) by FirstName` + +## IP functions + +- **The following functions now support arbitrary expressions as their argument:** + - [ipv4_is_private](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-is-privatefunction) + - [ipv4_is_in_range](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-is-in-range-function) + - [ipv4_netmask_suffix](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-netmask-suffix-function) + +# July 17, 2022 + +## Renamed dialect from sql_dialect to dialect + +`set dialect='clickhouse'` +`set dialect='kusto'` + +## IP functions +- [parse_ipv4](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parse-ipv4function) + `"Customers | project parse_ipv4('127.0.0.1')"` +- [parse_ipv6](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parse-ipv6function) + `"Customers | project parse_ipv6('127.0.0.1')"` + +Please note that the functions listed below only take constant parameters for now. Further improvement is to be expected to support expressions. + +- [ipv4_is_private](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-is-privatefunction) + `"Customers | project ipv4_is_private('192.168.1.6/24')"` + `"Customers | project ipv4_is_private('192.168.1.6')"` +- [ipv4_is_in_range](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-is-in-range-function) + `"Customers | project ipv4_is_in_range('127.0.0.1', '127.0.0.1')"` + `"Customers | project ipv4_is_in_range('192.168.1.6', '192.168.1.1/24')"` +- [ipv4_netmask_suffix](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-netmask-suffix-function) + `"Customers | project ipv4_netmask_suffix('192.168.1.1/24')"` + `"Customers | project ipv4_netmask_suffix('192.168.1.1')"` + +## string functions +- **support subquery for `in` orerator** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/in-cs-operator) + (subquery need to be wrapped with bracket inside bracket) + + `Customers | where Age in ((Customers|project Age|where Age < 30))` + Note: case-insensitive not supported yet +- **has_all** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-all-operator) + `Customers|where Occupation has_any ('Skilled','abcd')` + note : subquery not supported yet +- **has _any** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-anyoperator) + `Customers|where Occupation has_all ('Skilled','abcd')` + note : subquery not supported yet +- **countof** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/countoffunction) + `Customers | project countof('The cat sat on the mat', 'at')` + `Customers | project countof('The cat sat on the mat', 'at', 'normal')` + `Customers | project countof('The cat sat on the mat', 'at', 'regex')` +- **extract** ( https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractfunction) +`Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 0, 'The price of PINEAPPLE ice cream is 20')` +`Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 1, 'The price of PINEAPPLE ice cream is 20')` +`Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 20')` +`Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 3, 'The price of PINEAPPLE ice cream is 20')` +`Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 20', typeof(real))` + +- **extract_all** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractallfunction) + + `Customers | project extract_all('(\\w)(\\w+)(\\w)','The price of PINEAPPLE ice cream is 20')` + note: captureGroups not supported yet + +- **split** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/splitfunction) + `Customers | project split('aa_bb', '_')` + `Customers | project split('aaa_bbb_ccc', '_', 1)` + `Customers | project split('', '_')` + `Customers | project split('a__b', '_')` + `Customers | project split('aabbcc', 'bb')` + +- **strcat_delim** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/strcat-delimfunction) + `Customers | project strcat_delim('-', '1', '2', 'A') , 1s)` + `Customers | project strcat_delim('-', '1', '2', strcat('A','b'))` + note: only support string now. + +- **indexof** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/indexoffunction) + `Customers | project indexof('abcdefg','cde')` + `Customers | project indexof('abcdefg','cde',2)` + `Customers | project indexof('abcdefg','cde',6)` + note: length and occurrence not supported yet + + + + +# July 4, 2022 + +## sql_dialect + +- default is `clickhouse` + `set sql_dialect='clickhouse'` +- only process kql + `set sql_dialect='kusto'` + +## KQL() function + + - create table + `CREATE TABLE kql_table4 ENGINE = Memory AS select *, now() as new_column From kql(Customers | project LastName,Age);` + verify the content of `kql_table` + `select * from kql_table` + + - insert into table + create a tmp table: + ``` + CREATE TABLE temp + ( + FirstName Nullable(String), + LastName String, + Age Nullable(UInt8) + ) ENGINE = Memory; + ``` + `INSERT INTO temp select * from kql(Customers|project FirstName,LastName,Age);` + verify the content of `temp` + `select * from temp` + + - Select from kql() + `Select * from kql(Customers|project FirstName)` + +## KQL operators: + - Tabular expression statements + `Customers` + - Select Column + `Customers | project FirstName,LastName,Occupation` + - Limit returned results + `Customers | project FirstName,LastName,Occupation | take 1 | take 3` + - sort, order + `Customers | order by Age desc , FirstName asc` + - Filter + `Customers | where Occupation == 'Skilled Manual'` + - summarize + `Customers |summarize max(Age) by Occupation` + +## KQL string operators and functions + - contains + `Customers |where Education contains 'degree'` + - !contains + `Customers |where Education !contains 'degree'` + - contains_cs + `Customers |where Education contains 'Degree'` + - !contains_cs + `Customers |where Education !contains 'Degree'` + - endswith + `Customers | where FirstName endswith 'RE'` + - !endswith + `Customers | where !FirstName endswith 'RE'` + - endswith_cs + `Customers | where FirstName endswith_cs 're'` + - !endswith_cs + `Customers | where FirstName !endswith_cs 're'` + - == + `Customers | where Occupation == 'Skilled Manual'` + - != + `Customers | where Occupation != 'Skilled Manual'` + - has + `Customers | where Occupation has 'skilled'` + - !has + `Customers | where Occupation !has 'skilled'` + - has_cs + `Customers | where Occupation has 'Skilled'` + - !has_cs + `Customers | where Occupation !has 'Skilled'` + - hasprefix + `Customers | where Occupation hasprefix_cs 'Ab'` + - !hasprefix + `Customers | where Occupation !hasprefix_cs 'Ab'` + - hasprefix_cs + `Customers | where Occupation hasprefix_cs 'ab'` + - !hasprefix_cs + `Customers | where Occupation! hasprefix_cs 'ab'` + - hassuffix + `Customers | where Occupation hassuffix 'Ent'` + - !hassuffix + `Customers | where Occupation !hassuffix 'Ent'` + - hassuffix_cs + `Customers | where Occupation hassuffix 'ent'` + - !hassuffix_cs + `Customers | where Occupation hassuffix 'ent'` + - in + `Customers |where Education in ('Bachelors','High School')` + - !in + `Customers | where Education !in ('Bachelors','High School')` + - matches regex + `Customers | where FirstName matches regex 'P.*r'` + - startswith + `Customers | where FirstName startswith 'pet'` + - !startswith + `Customers | where FirstName !startswith 'pet'` + - startswith_cs + `Customers | where FirstName startswith_cs 'pet'` + - !startswith_cs + `Customers | where FirstName !startswith_cs 'pet'` + + - base64_encode_tostring() + `Customers | project base64_encode_tostring('Kusto1') | take 1` + - base64_decode_tostring() + `Customers | project base64_decode_tostring('S3VzdG8x') | take 1` + - isempty() + `Customers | where isempty(LastName)` + - isnotempty() + `Customers | where isnotempty(LastName)` + - isnotnull() + `Customers | where isnotnull(FirstName)` + - isnull() + `Customers | where isnull(FirstName)` + - url_decode() + `Customers | project url_decode('https%3A%2F%2Fwww.test.com%2Fhello%20word') | take 1` + - url_encode() + `Customers | project url_encode('https://www.test.com/hello word') | take 1` + - substring() + `Customers | project name_abbr = strcat(substring(FirstName,0,3), ' ', substring(LastName,2))` + - strcat() + `Customers | project name = strcat(FirstName, ' ', LastName)` + - strlen() + `Customers | project FirstName, strlen(FirstName)` + - strrep() + `Customers | project strrep(FirstName,2,'_')` + - toupper() + `Customers | project toupper(FirstName)` + - tolower() + `Customers | project tolower(FirstName)` + + ## Aggregate Functions + - arg_max() + - arg_min() + - avg() + - avgif() + - count() + - countif() + - max() + - maxif() + - min() + - minif() + - sum() + - sumif() + - dcount() + - dcountif() + - bin + \ No newline at end of file diff --git a/src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp b/src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp new file mode 100644 index 00000000000..44b08dd0b86 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp @@ -0,0 +1,382 @@ +#include "KQLFunctionFactory.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB::ErrorCodes +{ +extern const int NOT_IMPLEMENTED; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int SYNTAX_ERROR; +extern const int UNKNOWN_FUNCTION; +} + +namespace +{ +constexpr DB::TokenType determineClosingPair(const DB::TokenType token_type) +{ + if (token_type == DB::TokenType::OpeningCurlyBrace) + return DB::TokenType::ClosingCurlyBrace; + else if (token_type == DB::TokenType::OpeningRoundBracket) + return DB::TokenType::ClosingRoundBracket; + else if (token_type == DB::TokenType::OpeningSquareBracket) + return DB::TokenType::ClosingSquareBracket; + + throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unhandled token: {}", magic_enum::enum_name(token_type)); +} + +constexpr bool isClosingBracket(const DB::TokenType token_type) +{ + return token_type == DB::TokenType::ClosingCurlyBrace || token_type == DB::TokenType::ClosingRoundBracket + || token_type == DB::TokenType::ClosingSquareBracket; +} + +constexpr bool isOpeningBracket(const DB::TokenType token_type) +{ + return token_type == DB::TokenType::OpeningCurlyBrace || token_type == DB::TokenType::OpeningRoundBracket + || token_type == DB::TokenType::OpeningSquareBracket; +} +} + +namespace DB +{ +bool IParserKQLFunction::convert(String & out, IParser::Pos & pos) +{ + return wrapConvertImpl( + pos, + IncreaseDepthTag{}, + [&] + { + bool res = convertImpl(out, pos); + if (!res) + out = ""; + return res; + }); +} + +bool IParserKQLFunction::directMapping( + String & out, IParser::Pos & pos, const std::string_view ch_fn, const Interval & argument_count_interval) +{ + const auto fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + out.append(ch_fn.data(), ch_fn.length()); + out.push_back('('); + + int argument_count = 0; + const auto begin = pos; + while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon) + { + if (pos != begin) + out.append(", "); + + if (const auto argument = getOptionalArgument(fn_name, pos)) + { + ++argument_count; + out.append(*argument); + } + + if (pos->type == TokenType::ClosingRoundBracket) + { + if (!argument_count_interval.IsWithinBounds(argument_count)) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "{}: between {} and {} arguments are expected, but {} were provided", + fn_name, + argument_count_interval.Min(), + argument_count_interval.Max(), + argument_count); + + out.push_back(')'); + return true; + } + } + + out.clear(); + pos = begin; + return false; +} + +String IParserKQLFunction::generateUniqueIdentifier() +{ + // This particular random generator hits each number exactly once before looping over. + // Because of this, it's sufficient for queries consisting of up to 2^16 (= 65536) distinct function calls. + // Reference: https://www.pcg-random.org/using-pcg-cpp.html#insecure-generators + static pcg32_once_insecure random_generator; + return std::to_string(random_generator()); +} + +String IParserKQLFunction::getArgument(const String & function_name, DB::IParser::Pos & pos, const ArgumentState argument_state) +{ + if (auto optional_argument = getOptionalArgument(function_name, pos, argument_state)) + return std::move(*optional_argument); + + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Required argument was not provided in {}", function_name); +} + +std::vector IParserKQLFunction::getArguments( + const String & function_name, DB::IParser::Pos & pos, const ArgumentState argument_state, const Interval & argument_count_interval) +{ + std::vector arguments; + while (auto argument = getOptionalArgument(function_name, pos, argument_state)) + { + arguments.push_back(std::move(*argument)); + } + if (!argument_count_interval.IsWithinBounds(static_cast(arguments.size()))) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "{}: between {} and {} arguments are expected, but {} were provided", + function_name, + argument_count_interval.Min(), + argument_count_interval.Max(), + arguments.size()); + + return arguments; +} + +String IParserKQLFunction::getConvertedArgument(const String & fn_name, IParser::Pos & pos) +{ + int32_t round_bracket_count = 0, square_bracket_count = 0; + if (pos->type == TokenType::ClosingRoundBracket || pos->type == TokenType::ClosingSquareBracket) + return {}; + + if (pos->isEnd() || pos->type == TokenType::PipeMark || pos->type == TokenType::Semicolon) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Need more argument(s) in function: {}", fn_name); + + std::vector tokens; + while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon) + { + if (pos->type == TokenType::OpeningRoundBracket) + ++round_bracket_count; + if (pos->type == TokenType::ClosingRoundBracket) + --round_bracket_count; + + if (pos->type == TokenType::OpeningSquareBracket) + ++square_bracket_count; + if (pos->type == TokenType::ClosingSquareBracket) + --square_bracket_count; + + if (!KQLOperators::convert(tokens, pos)) + { + if (pos->type == TokenType::BareWord) + { + tokens.push_back(IParserKQLFunction::getExpression(pos)); + } + else if ( + pos->type == TokenType::Comma || pos->type == TokenType::ClosingRoundBracket + || pos->type == TokenType::ClosingSquareBracket) + { + if (pos->type == TokenType::Comma) + break; + if (pos->type == TokenType::ClosingRoundBracket && round_bracket_count == -1) + break; + if (pos->type == TokenType::ClosingSquareBracket && square_bracket_count == 0) + break; + tokens.push_back(String(pos->begin, pos->end)); + } + else + { + String token; + if (pos->type == TokenType::QuotedIdentifier) + token = "'" + escapeSingleQuotes(String(pos->begin + 1, pos->end - 1)) + "'"; + else if (pos->type == TokenType::OpeningSquareBracket) + { + ++pos; + String array_index; + while (!pos->isEnd() && pos->type != TokenType::ClosingSquareBracket) + { + array_index += getExpression(pos); + ++pos; + } + token = std::format("[ {0} >=0 ? {0} + 1 : {0}]", array_index); + } + else + token = String(pos->begin, pos->end); + + tokens.push_back(token); + } + } + + ++pos; + if (pos->type == TokenType::Comma || pos->type == TokenType::ClosingRoundBracket || pos->type == TokenType::ClosingSquareBracket) + { + if (pos->type == TokenType::Comma) + break; + if (pos->type == TokenType::ClosingRoundBracket && round_bracket_count == -1) + break; + if (pos->type == TokenType::ClosingSquareBracket && square_bracket_count == 0) + break; + } + } + + String converted_arg; + for (const auto & token : tokens) + converted_arg.append((converted_arg.empty() ? "" : " ") + token); + + return converted_arg; +} + +std::optional +IParserKQLFunction::getOptionalArgument(const String & function_name, DB::IParser::Pos & pos, const ArgumentState argument_state) +{ + if (const auto type = pos->type; type != DB::TokenType::Comma && type != DB::TokenType::OpeningRoundBracket) + return {}; + + ++pos; + if (const auto type = pos->type; type == DB::TokenType::ClosingRoundBracket || type == DB::TokenType::ClosingSquareBracket) + return {}; + + if (argument_state == ArgumentState::Parsed) + return getConvertedArgument(function_name, pos); + + if (argument_state != ArgumentState::Raw) + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Argument extraction is not implemented for {}::{}", + magic_enum::enum_type_name(), + magic_enum::enum_name(argument_state)); + + const auto * begin = pos->begin; + std::stack scopes; + while (!pos->isEnd() && (!scopes.empty() || (pos->type != DB::TokenType::Comma && pos->type != DB::TokenType::ClosingRoundBracket))) + { + const auto token_type = pos->type; + if (isOpeningBracket(token_type)) + scopes.push(token_type); + else if (isClosingBracket(token_type)) + { + if (scopes.empty() || determineClosingPair(scopes.top()) != token_type) + throw Exception( + DB::ErrorCodes::SYNTAX_ERROR, "Unmatched token: {} when parsing {}", magic_enum::enum_name(token_type), function_name); + + scopes.pop(); + } + + ++pos; + } + + return std::string(begin, pos->begin); +} + +String IParserKQLFunction::getKQLFunctionName(IParser::Pos & pos) +{ + String fn_name(pos->begin, pos->end); + ++pos; + if (pos->type != TokenType::OpeningRoundBracket) + { + --pos; + return ""; + } + return fn_name; +} + +String IParserKQLFunction::kqlCallToExpression( + const std::string_view function_name, const std::initializer_list params, const uint32_t max_depth) +{ + return kqlCallToExpression(function_name, std::span(params), max_depth); +} + +String IParserKQLFunction::kqlCallToExpression( + const std::string_view function_name, const std::span params, const uint32_t max_depth) +{ + const auto params_str = std::accumulate( + std::cbegin(params), + std::cend(params), + String(), + [](String acc, const std::string_view param) + { + if (!acc.empty()) + acc.append(", "); + + acc.append(param.data(), param.length()); + return acc; + }); + + const auto kql_call = std::format("{}({})", function_name, params_str); + DB::Tokens call_tokens(kql_call.c_str(), kql_call.c_str() + kql_call.length()); + DB::IParser::Pos tokens_pos(call_tokens, max_depth); + return DB::IParserKQLFunction::getExpression(tokens_pos); +} + +void IParserKQLFunction::validateEndOfFunction(const String & fn_name, IParser::Pos & pos) +{ + if (pos->type != TokenType::ClosingRoundBracket) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Too many arguments in function: {}", fn_name); +} + +String IParserKQLFunction::getExpression(IParser::Pos & pos) +{ + String arg(pos->begin, pos->end); + auto parseConstTimespan = [&]() + { + ParserKQLDateTypeTimespan time_span; + ASTPtr node; + Expected expected; + + if (time_span.parse(pos, node, expected)) + arg = boost::lexical_cast(time_span.toSeconds()); + }; + + if (pos->type == TokenType::BareWord) + { + const auto fun = KQLFunctionFactory::get(arg); + if (String new_arg; fun && fun->convert(new_arg, pos)) + { + validateEndOfFunction(arg, pos); + arg = std::move(new_arg); + } + else + { + if (!fun) + { + ++pos; + if (pos->type == TokenType::OpeningRoundBracket) + { + if (Poco::toLower(arg) != "and" && Poco::toLower(arg) != "or") + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "{} is not a supported kusto function", arg); + } + --pos; + } + + parseConstTimespan(); + } + } + else if (pos->type == TokenType::ErrorWrongNumber) + parseConstTimespan(); + else if (pos->type == TokenType::QuotedIdentifier) + arg = "'" + escapeSingleQuotes(String(pos->begin + 1, pos->end - 1)) + "'"; + else if (pos->type == TokenType::OpeningSquareBracket) + { + ++pos; + String array_index; + while (!pos->isEnd() && pos->type != TokenType::ClosingSquareBracket) + { + array_index += getExpression(pos); + ++pos; + } + arg = std::format("[ {0} >=0 ? {0} + 1 : {0}]", array_index); + } + + return arg; +} + +String IParserKQLFunction::escapeSingleQuotes(const String & input) +{ + String output; + for (const auto & ch : input) + { + if (ch == '\'') + output += ch; + output += ch; + } + return output; +} +} diff --git a/src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.h b/src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.h new file mode 100644 index 00000000000..147436551f9 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.h @@ -0,0 +1,91 @@ +#pragma once + +#include + +#include + +namespace DB +{ +class Interval +{ +public: + using Representation = int; + + Interval(const Representation min_, const Representation max_) : max(max_), min(min_) { } + + Representation Max() const { return max; } + Representation Min() const { return min; } + bool IsWithinBounds(const Representation value) const { return min <= value && value <= max; } + + static constexpr auto max_bound = std::numeric_limits::max(); + static constexpr auto min_bound = std::numeric_limits::min(); + +private: + Representation max = max_bound; + Representation min = min_bound; +}; + +class IParserKQLFunction +{ +public: + enum class ArgumentState + { + Parsed, + Raw + }; + + template + ALWAYS_INLINE static bool wrapConvertImpl(IParser::Pos & pos, const F & func) + { + IParser::Pos begin = pos; + bool res = func(); + if (!res) + pos = begin; + return res; + } + + struct IncreaseDepthTag + { + }; + + template + ALWAYS_INLINE static bool wrapConvertImpl(IParser::Pos & pos, IncreaseDepthTag, const F & func) + { + IParser::Pos begin = pos; + pos.increaseDepth(); + bool res = func(); + pos.decreaseDepth(); + if (!res) + pos = begin; + return res; + } + + bool convert(String & out, IParser::Pos & pos); + virtual const char * getName() const = 0; + virtual ~IParserKQLFunction() = default; + + static String generateUniqueIdentifier(); + static String getArgument(const String & function_name, DB::IParser::Pos & pos, ArgumentState argument_state = ArgumentState::Parsed); + static std::vector getArguments( + const String & function_name, + DB::IParser::Pos & pos, + ArgumentState argument_state = ArgumentState::Parsed, + const Interval & argument_count_interval = {0, Interval::max_bound}); + static String getConvertedArgument(const String & fn_name, IParser::Pos & pos); + static String getExpression(IParser::Pos & pos); + static String getKQLFunctionName(IParser::Pos & pos); + static std::optional + getOptionalArgument(const String & function_name, DB::IParser::Pos & pos, ArgumentState argument_state = ArgumentState::Parsed); + static String + kqlCallToExpression(std::string_view function_name, std::initializer_list params, uint32_t max_depth); + static String kqlCallToExpression(std::string_view function_name, std::span params, uint32_t max_depth); + static String escapeSingleQuotes(const String & input); + +protected: + virtual bool convertImpl(String & out, IParser::Pos & pos) = 0; + + static bool directMapping( + String & out, IParser::Pos & pos, std::string_view ch_fn, const Interval & argument_count_interval = {0, Interval::max_bound}); + static void validateEndOfFunction(const String & fn_name, IParser::Pos & pos); +}; +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.cpp b/src/Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.cpp new file mode 100644 index 00000000000..16436d38d32 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.cpp @@ -0,0 +1,488 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +bool ArgMax::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "argMax"); +} + +bool ArgMin::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "argMin"); +} + +bool Avg::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "avg"); +} + +bool AvgIf::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "avgIf"); +} + +bool BinaryAllAnd::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "groupBitAnd"); +} + +bool BinaryAllOr::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "groupBitOr"); +} + +bool BinaryAllXor::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "groupBitXor"); +} + +bool BuildSchema::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool Count::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "count"); +} + +bool CountIf::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "countIf"); +} + +bool DCount::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + ++pos; + String value = getConvertedArgument(fn_name, pos); + + out = "count(DISTINCT " + value + ")"; + return true; +} + +bool DCountIf::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + ++pos; + String value = getConvertedArgument(fn_name, pos); + ++pos; + String condition = getConvertedArgument(fn_name, pos); + out = "countIf (DISTINCT " + value + ", " + condition + ")"; + return true; +} + +bool MakeBag::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool MakeBagIf::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool MakeList::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + ++pos; + const auto expr = getConvertedArgument(fn_name, pos); + if (pos->type == TokenType::Comma) + { + ++pos; + const auto max_size = getConvertedArgument(fn_name, pos); + out = "groupArrayIf(" + max_size + ")(" + expr + " , " + expr + " IS NOT NULL)"; + } + else + out = "groupArrayIf(" + expr + " , " + expr + " IS NOT NULL)"; + return true; +} + +bool MakeListIf::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + ++pos; + const auto expr = getConvertedArgument(fn_name, pos); + ++pos; + const auto predicate = getConvertedArgument(fn_name, pos); + if (pos->type == TokenType::Comma) + { + ++pos; + const auto max_size = getConvertedArgument(fn_name, pos); + out = "groupArrayIf(" + max_size + ")(" + expr + " , " + predicate + " )"; + } + else + out = "groupArrayIf(" + expr + " , " + predicate + " )"; + return true; +} + +bool MakeListWithNulls::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + ++pos; + const auto column_name = getConvertedArgument(fn_name, pos); + out = "arrayConcat(groupArray(" + column_name + "), arrayMap(x -> null, range(0, toUInt32(count(*)-length( groupArray(" + column_name + + "))),1)))"; + return true; +} + +bool MakeSet::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + ++pos; + const auto expr = getConvertedArgument(fn_name, pos); + if (pos->type == TokenType::Comma) + { + ++pos; + const auto max_size = getConvertedArgument(fn_name, pos); + out = "groupUniqArray(" + max_size + ")(" + expr + ")"; + } + else + out = "groupUniqArray(" + expr + ")"; + return true; +} + +bool MakeSetIf::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + ++pos; + const auto expr = getConvertedArgument(fn_name, pos); + ++pos; + const auto predicate = getConvertedArgument(fn_name, pos); + if (pos->type == TokenType::Comma) + { + ++pos; + const auto max_size = getConvertedArgument(fn_name, pos); + out = "groupUniqArrayIf(" + max_size + ")(" + expr + " , " + predicate + " )"; + } + else + out = "groupUniqArrayIf(" + expr + " , " + predicate + " )"; + return true; +} + +bool Max::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "max"); +} + +bool MaxIf::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "maxIf"); +} + +bool Min::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "min"); +} + +bool MinIf::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "minIf"); +} + +bool Percentile::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + + ++pos; + String column_name = getConvertedArgument(fn_name, pos); + trim(column_name); + + ++pos; + String value = getConvertedArgument(fn_name, pos); + trim(value); + + out = "quantile(" + value + "/100)(" + column_name + ")"; + return true; +} + +bool Percentilew::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + + ++pos; + String bucket_column = getConvertedArgument(fn_name, pos); + trim(bucket_column); + + ++pos; + String frequency_column = getConvertedArgument(fn_name, pos); + trim(frequency_column); + + ++pos; + String value = getConvertedArgument(fn_name, pos); + trim(value); + + out = "quantileExactWeighted(" + value + "/100)(" + bucket_column + "," + frequency_column + ")"; + return true; +} + +bool Percentiles::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + + ++pos; + String column_name = getConvertedArgument(fn_name, pos); + trim(column_name); + String expr = "quantiles("; + String value; + while (pos->type != TokenType::ClosingRoundBracket) + { + if (pos->type != TokenType::Comma) + { + value = String(pos->begin, pos->end); + expr = expr + value + "/100"; + ++pos; + if (pos->type != TokenType::ClosingRoundBracket) + expr += ", "; + } + else + ++pos; + } + out = expr + ")(" + column_name + ")"; + return true; +} + +bool PercentilesArray::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + + ++pos; + String column_name = getConvertedArgument(fn_name, pos); + trim(column_name); + String expr = "quantiles("; + String value; + while (pos->type != TokenType::ClosingRoundBracket) + { + if (pos->type != TokenType::Comma && String(pos->begin, pos->end) != "dynamic" && pos->type != TokenType::OpeningRoundBracket + && pos->type != TokenType::OpeningSquareBracket && pos->type != TokenType::ClosingSquareBracket) + { + value = String(pos->begin, pos->end); + expr = expr + value + "/100"; + + if (pos->type != TokenType::Comma && pos->type != TokenType::OpeningRoundBracket && pos->type != TokenType::OpeningSquareBracket + && pos->type != TokenType::ClosingSquareBracket) + expr += ", "; + ++pos; + } + else + { + ++pos; + } + } + ++pos; + if (pos->type != TokenType::ClosingRoundBracket) + --pos; + + expr.pop_back(); + expr.pop_back(); + expr = expr + ")(" + column_name + ")"; + out = expr; + return true; +} + +bool Percentilesw::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + + ++pos; + String bucket_column = getConvertedArgument(fn_name, pos); + trim(bucket_column); + + ++pos; + String frequency_column = getConvertedArgument(fn_name, pos); + trim(frequency_column); + + String expr = "quantilesExactWeighted("; + String value; + + while (pos->type != TokenType::ClosingRoundBracket) + { + if (pos->type != TokenType::Comma) + { + value = String(pos->begin, pos->end); + expr = expr + value + "/100"; + ++pos; + if (pos->type != TokenType::ClosingRoundBracket) + expr += ", "; + } + else + ++pos; + } + expr = expr + ")(" + bucket_column + "," + frequency_column + ")"; + out = expr; + return true; +} + +bool PercentileswArray::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + + ++pos; + String bucket_column = getConvertedArgument(fn_name, pos); + trim(bucket_column); + + ++pos; + String frequency_column = getConvertedArgument(fn_name, pos); + trim(frequency_column); + + String expr = "quantilesExactWeighted("; + String value; + while (pos->type != TokenType::ClosingRoundBracket) + { + if (pos->type != TokenType::Comma && String(pos->begin, pos->end) != "dynamic" && pos->type != TokenType::OpeningRoundBracket + && pos->type != TokenType::OpeningSquareBracket && pos->type != TokenType::ClosingSquareBracket) + { + value = String(pos->begin, pos->end); + expr = expr + value + "/100"; + + if (pos->type != TokenType::Comma && pos->type != TokenType::OpeningRoundBracket && pos->type != TokenType::OpeningSquareBracket + && pos->type != TokenType::ClosingSquareBracket) + expr += ", "; + ++pos; + } + else + { + ++pos; + } + } + ++pos; + if (pos->type != TokenType::ClosingRoundBracket) + --pos; + + expr.pop_back(); + expr.pop_back(); + expr = expr + ")(" + bucket_column + "," + frequency_column + ")"; + out = expr; + return true; +} + +bool Stdev::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + ++pos; + const auto expr = getConvertedArgument(fn_name, pos); + out = "sqrt(varSamp(" + expr + "))"; + return true; +} + +bool StdevIf::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + ++pos; + const auto expr = getConvertedArgument(fn_name, pos); + if (pos->type != TokenType::Comma) + return false; + + ++pos; + const auto predicate = getConvertedArgument(fn_name, pos); + out = "sqrt(varSampIf(" + expr + ", " + predicate + "))"; + return true; +} + +bool Sum::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "sum"); +} + +bool SumIf::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "sumIf"); +} + +bool TakeAny::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool TakeAnyIf::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool Variance::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool VarianceIf::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.h b/src/Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.h new file mode 100644 index 00000000000..5002590ae43 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.h @@ -0,0 +1,267 @@ +#pragma once + +#include +#include +namespace DB +{ +class ArgMax : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "arg_max()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ArgMin : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "arg_min()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Avg : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "avg()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class AvgIf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "avgif()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class BinaryAllAnd : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "binary_all_and()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class BinaryAllOr : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "binary_all_or()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class BinaryAllXor : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "binary_all_xor()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class BuildSchema : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "buildschema()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Count : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "count()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class CountIf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "countif()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DCount : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "dcount()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DCountIf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "dcountif()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class MakeBag : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "make_bag()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class MakeBagIf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "make_bag_if()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class MakeList : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "make_list()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class MakeListIf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "make_list_if()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class MakeListWithNulls : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "make_list_with_nulls()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class MakeSet : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "make_set()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class MakeSetIf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "make_set_if()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Max : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "max()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class MaxIf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "maxif()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Min : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "min()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class MinIf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "minif()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Percentile : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "percentile()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Percentilew : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "percentilew()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Percentiles : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "percentiles()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class PercentilesArray : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "percentiles_array()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Percentilesw : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "percentilesw()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class PercentileswArray : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "percentilesw_array()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Stdev : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "stdev()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class StdevIf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "stdevif()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Sum : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "sum()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SumIf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "sumif()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class TakeAny : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "take_any()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class TakeAnyIf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "take_anyif()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Variance : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "variance()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class VarianceIf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "varianceif()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.cpp b/src/Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.cpp new file mode 100644 index 00000000000..f8765b116d4 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.cpp @@ -0,0 +1,99 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +bool BinaryAnd::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto lhs = getArgument(function_name, pos); + const auto rhs = getArgument(function_name, pos); + out = std::format("bitAnd(cast({0}, 'Int64'), cast({1}, 'Int64'))", lhs, rhs); + return true; +} + +bool BinaryNot::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto value = getArgument(function_name, pos); + out = std::format("bitNot(cast({0}, 'Int64'))", value); + return true; +} + +bool BinaryOr::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto lhs = getArgument(function_name, pos); + const auto rhs = getArgument(function_name, pos); + out = std::format("bitOr(cast({0}, 'Int64'), cast({1}, 'Int64'))", lhs, rhs); + return true; +} + +bool BinaryShiftLeft::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto value = getArgument(function_name, pos); + const auto count = getArgument(function_name, pos); + out = std::format("if({1} < 0, null, bitShiftLeft(cast({0}, 'Int64'), {1}))", value, count); + return true; +} + +bool BinaryShiftRight::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto value = getArgument(function_name, pos); + const auto count = getArgument(function_name, pos); + out = std::format("if({1} < 0, null, bitShiftRight(cast({0}, 'Int64'), {1}))", value, count); + return true; +} + +bool BinaryXor::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto lhs = getArgument(function_name, pos); + const auto rhs = getArgument(function_name, pos); + out = std::format("bitXor(cast({0}, 'Int64'), cast({1}, 'Int64'))", lhs, rhs); + return true; +} + +bool BitsetCountOnes::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "bitCount"); +} + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.h b/src/Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.h new file mode 100644 index 00000000000..f24b73a0e78 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.h @@ -0,0 +1,56 @@ +#pragma once + +#include +#include +namespace DB +{ +class BinaryAnd : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "binary_and()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class BinaryNot : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "binary_not()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class BinaryOr : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "binary_or()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class BinaryShiftLeft : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "binary_shift_left()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class BinaryShiftRight : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "binary_shift_right()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class BinaryXor : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "binary_xor()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class BitsetCountOnes : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "bitset_count_ones()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLCastingFunctions.cpp b/src/Parsers/Kusto/KustoFunctions/KQLCastingFunctions.cpp new file mode 100644 index 00000000000..a6a7b2ee320 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLCastingFunctions.cpp @@ -0,0 +1,170 @@ +#include +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ +bool ToBool::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto param = getArgument(function_name, pos); + out = std::format( + "multiIf(toString({0}) = 'true', true, " + "toString({0}) = 'false', false, toInt64OrNull(toString({0})) != 0)", + param, + generateUniqueIdentifier()); + return true; +} + +bool ToDateTime::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto param = getArgument(function_name, pos); + + out = std::format("parseDateTime64BestEffortOrNull(toString({0}),9,'UTC')", param); + return true; +} + +bool ToDouble::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto param = getArgument(function_name, pos); + out = std::format("toFloat64OrNull(toString({0}))", param); + return true; +} + +bool ToInt::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto param = getArgument(function_name, pos); + out = std::format("toInt32OrNull(toString({0}))", param); + return true; +} + +bool ToLong::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto param = getArgument(function_name, pos); + out = std::format("toInt64OrNull(toString({0}))", param); + return true; +} + +bool ToString::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto param = getArgument(function_name, pos); + out = std::format("ifNull(toString({0}), '')", param); + return true; +} +bool ToTimeSpan::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + ++pos; + String arg; + if (pos->type == TokenType::QuotedIdentifier) + arg = String(pos->begin + 1, pos->end - 1); + else if (pos->type == TokenType::StringLiteral) + arg = String(pos->begin, pos->end); + else + arg = getConvertedArgument(function_name, pos); + + if (pos->type == TokenType::StringLiteral || pos->type == TokenType::QuotedIdentifier) + { + ++pos; + try + { + auto result = kqlCallToExpression("time", {arg}, pos.max_depth); + out = std::format("{}", result); + } + catch (...) + { + out = "NULL"; + } + } + else + out = std::format("{}", arg); + + return true; +} + +bool ToDecimal::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + String res; + int scale = 0; + int precision; + + if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral) + { + res = String(pos->begin + 1, pos->end - 1); + ++pos; + precision = 34; + } + else + { + res = getConvertedArgument(fn_name, pos); + precision = 17; + } + static const std::regex expr{"^[0-9]+e[+-]?[0-9]+"}; + bool is_string = std::any_of(res.begin(), res.end(), ::isalpha) && !(std::regex_match(res, expr)); + + if (is_string) + out = "NULL"; + else if (std::regex_match(res, expr)) + { + auto exponential_pos = res.find('e'); + if (res[exponential_pos + 1] == '+' || res[exponential_pos + 1] == '-') + scale = std::stoi(res.substr(exponential_pos + 2, res.length())); + else + scale = std::stoi(res.substr(exponential_pos + 1, res.length())); + + out = std::format("toDecimal128({}::String,{})", res, scale); + } + else + { + if (const auto dot_pos = res.find('.'); dot_pos != String::npos) + { + const auto tmp = res.substr(0, dot_pos - 1); + const auto tmp_length = static_cast(std::ssize(tmp)); + scale = std::max(precision - tmp_length, 0); + } + if (scale < 0) + out = "NULL"; + else + out = std::format("toDecimal128({}::String,{})", res, scale); + } + + return true; +} + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h b/src/Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h new file mode 100644 index 00000000000..45ce2e44d56 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h @@ -0,0 +1,63 @@ +#pragma once + +#include +#include + +namespace DB +{ +class ToBool : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "tobool()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ToDateTime : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "todatetime()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ToDouble : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "todouble()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ToInt : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "toint()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ToLong : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "tolong()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ToString : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "tostring()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ToTimeSpan : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "totimespan()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ToDecimal : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "todecimal()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLDataTypeFunctions.cpp b/src/Parsers/Kusto/KustoFunctions/KQLDataTypeFunctions.cpp new file mode 100644 index 00000000000..9ddc009307a --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLDataTypeFunctions.cpp @@ -0,0 +1,261 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "Poco/String.h" +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int SYNTAX_ERROR; +} + +bool DatatypeBool::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "toBool"); +} + +bool DatatypeDatetime::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + String datetime_str; + + ++pos; + if (pos->type == TokenType::QuotedIdentifier) + datetime_str = std::format("'{}'", String(pos->begin + 1, pos->end - 1)); + else if (pos->type == TokenType::StringLiteral) + datetime_str = String(pos->begin, pos->end); + else if (pos->type == TokenType::BareWord) + { + datetime_str = getConvertedArgument(fn_name, pos); + if (Poco::toUpper(datetime_str) == "NULL") + out = "NULL"; + else + out = std::format( + "if(toTypeName({0}) = 'Int64' OR toTypeName({0}) = 'Int32'OR toTypeName({0}) = 'Float64' OR toTypeName({0}) = 'UInt32' OR " + " toTypeName({0}) = 'UInt64', toDateTime64({0},9,'UTC'), parseDateTime64BestEffortOrNull({0}::String,9,'UTC'))", + datetime_str); + return true; + } + else + { + auto start = pos; + while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon) + { + ++pos; + if (pos->type == TokenType::ClosingRoundBracket) + break; + } + --pos; + datetime_str = std::format("'{}'", String(start->begin, pos->end)); + } + out = std::format("parseDateTime64BestEffortOrNull({},9,'UTC')", datetime_str); + ++pos; + return true; +} + +bool DatatypeDynamic::convertImpl(String & out, IParser::Pos & pos) +{ + static const std::unordered_set ALLOWED_FUNCTIONS{"date", "datetime", "dynamic", "time", "timespan"}; + + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + ++pos; + if (pos->type == TokenType::OpeningCurlyBrace) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Property bags are not supported for now in {}", function_name); + + while (!pos->isEnd() && pos->type != TokenType::ClosingRoundBracket) + { + if (const auto token_type = pos->type; token_type == TokenType::BareWord || token_type == TokenType::Number + || token_type == TokenType::QuotedIdentifier || token_type == TokenType::StringLiteral) + { + if (const std::string_view token(pos->begin, pos->end); token_type == TokenType::BareWord && !ALLOWED_FUNCTIONS.contains(token)) + { + ++pos; + if (pos->type != TokenType::ClosingRoundBracket && pos->type != TokenType::ClosingSquareBracket + && pos->type != TokenType::Comma) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Expression {} is not supported inside {}", token, function_name); + + --pos; + } + + out.append(getConvertedArgument(function_name, pos)); + } + else + { + out.append(pos->begin, pos->end); + ++pos; + } + } + + return true; +} + +bool DatatypeGuid::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + String guid_str; + + ++pos; + if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral) + guid_str = String(pos->begin + 1, pos->end - 1); + else + { + auto start = pos; + while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon) + { + ++pos; + if (pos->type == TokenType::ClosingRoundBracket) + break; + } + --pos; + guid_str = String(start->begin, pos->end); + } + out = std::format("toUUIDOrNull('{}')", guid_str); + ++pos; + return true; +} + +bool DatatypeInt::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + String guid_str; + + ++pos; + if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "String is not parsed as int literal."); + else + { + auto arg = getConvertedArgument(fn_name, pos); + out = std::format("toInt32({})", arg); + } + return true; +} + +bool DatatypeLong::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "toInt64"); +} + +bool DatatypeReal::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "String is not parsed as double literal."); + else + { + auto arg = getConvertedArgument(fn_name, pos); + out = std::format("toFloat64({})", arg); + } + return true; +} + +bool DatatypeString::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool DatatypeTimespan::convertImpl(String & out, IParser::Pos & pos) +{ + ParserKQLDateTypeTimespan time_span; + ASTPtr node; + Expected expected; + bool sign = false; + + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + ++pos; + if (pos->type == TokenType::Minus) + { + sign = true; + ++pos; + } + if (time_span.parse(pos, node, expected)) + { + if (sign) + out = std::format("-{}::Float64", time_span.toSeconds()); + else + out = std::format("{}::Float64", time_span.toSeconds()); + ++pos; + } + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not a correct timespan expression: {}", fn_name); + return true; +} + +bool DatatypeDecimal::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + String arg; + int scale = 0; + int precision = 34; + + if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse String as decimal Literal: {}", fn_name); + + --pos; + arg = getArgument(fn_name, pos); + + //NULL expr returns NULL not exception + static const std::regex expr{"^[0-9]+e[+-]?[0-9]+"}; + bool is_string = std::any_of(arg.begin(), arg.end(), ::isalpha) && Poco::toUpper(arg) != "NULL" && !(std::regex_match(arg, expr)); + if (is_string) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse String as decimal Literal: {}", fn_name); + + if (std::regex_match(arg, expr)) + { + auto exponential_pos = arg.find('e'); + if (arg[exponential_pos + 1] == '+' || arg[exponential_pos + 1] == '-') + scale = std::stoi(arg.substr(exponential_pos + 2, arg.length())); + else + scale = std::stoi(arg.substr(exponential_pos + 1, arg.length())); + + out = std::format("toDecimal128({}::String,{})", arg, scale); + return true; + } + + if (const auto dot_pos = arg.find('.'); dot_pos != String::npos) + { + const auto length = static_cast(std::ssize(arg.substr(0, dot_pos - 1))); + scale = std::max(precision - length, 0); + } + if (is_string) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse String as decimal Literal: {}", fn_name); + + if (scale < 0 || Poco::toUpper(arg) == "NULL") + out = "NULL"; + else + out = std::format("toDecimal128({}::String,{})", arg, scale); + + return true; +} + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLDataTypeFunctions.h b/src/Parsers/Kusto/KustoFunctions/KQLDataTypeFunctions.h new file mode 100644 index 00000000000..e2463cc7d04 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLDataTypeFunctions.h @@ -0,0 +1,77 @@ +#pragma once + +#include +#include +namespace DB +{ +class DatatypeBool : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "bool(),boolean()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DatatypeDatetime : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "datetime(),date()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DatatypeDynamic : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "dynamic()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DatatypeGuid : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "guid()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DatatypeInt : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "int()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DatatypeLong : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "long()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DatatypeReal : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "real(),double()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DatatypeString : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "string()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DatatypeTimespan : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "timespan(), time()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DatatypeDecimal : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "decimal()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.cpp b/src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.cpp new file mode 100644 index 00000000000..18ac5164df9 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.cpp @@ -0,0 +1,789 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "Poco/String.h" +namespace DB::ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int SYNTAX_ERROR; +} +namespace DB +{ + +bool TimeSpan::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool Ago::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + if (pos->type != TokenType::ClosingRoundBracket) + { + const auto offset = getConvertedArgument(fn_name, pos); + out = std::format("now64(9,'UTC') - {}", offset); + } + else + out = "now64(9,'UTC')"; + return true; +} + +bool DatetimeAdd::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + String period = getConvertedArgument(fn_name, pos); + //remove quotes from period. + trim(period); + if (period.front() == '\"' || period.front() == '\'') + { + //period.remove + period.erase(0, 1); // erase the first quote + period.erase(period.size() - 1); // erase the last quote + } + ++pos; + const String offset = getConvertedArgument(fn_name, pos); + ++pos; + const String datetime = getConvertedArgument(fn_name, pos); + + out = std::format("date_add({}, {}, {})", period, offset, datetime); + + return true; +}; + +bool DatetimePart::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + String part = Poco::toUpper(getConvertedArgument(fn_name, pos)); + trim(part); + if (part.front() == '\"' || part.front() == '\'') + { + //period.remove + part.erase(0, 1); // erase the first quote + part.erase(part.size() - 1); // erase the last quote + } + String date; + if (pos->type == TokenType::Comma) + { + ++pos; + date = getConvertedArgument(fn_name, pos); + } + String format; + + if (part == "YEAR") + format = "%G"; + else if (part == "QUARTER") + format = "%Q"; + else if (part == "MONTH") + format = "%m"; + else if (part == "WEEK_OF_YEAR") + format = "%V"; + else if (part == "DAY") + format = "%e"; + else if (part == "DAYOFYEAR") + format = "%j"; + else if (part == "HOUR") + format = "%H"; + else if (part == "MINUTE") + format = "%i"; + else if (part == "SECOND") + format = "%S"; + else + throw Exception(ErrorCodes::SYNTAX_ERROR, "Unexpected argument {} for {}", part, fn_name); + + out = std::format("formatDateTime({}, '{}')", date, format); + return true; +} + +bool DatetimeDiff::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + ++pos; + String arguments; + + arguments = arguments + getConvertedArgument(fn_name, pos) + ","; + ++pos; + arguments = arguments + getConvertedArgument(fn_name, pos) + ","; + ++pos; + arguments = arguments + getConvertedArgument(fn_name, pos); + + out = std::format("DateDiff({}) * -1", arguments); + return true; +} + +bool DayOfMonth::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "toDayOfMonth"); +} + +bool DayOfWeek::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + ++pos; + const String datetime_str = getConvertedArgument(fn_name, pos); + + out = std::format("concat((toDayOfWeek({})%7)::String, '.00:00:00')", datetime_str); + return true; +} + +bool DayOfYear::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "toDayOfYear"); +} + +bool EndOfMonth::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String datetime_str = getConvertedArgument(fn_name, pos); + String offset = "0"; + + if (pos->type == TokenType::Comma) + { + ++pos; + offset = getConvertedArgument(fn_name, pos); + if (offset.empty()) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Number of arguments do not match in function: {}", fn_name); + } + out = std::format( + "toDateTime(toLastDayOfMonth(toDateTime({}, 9, 'UTC') + toIntervalMonth({})), 9, 'UTC') + toIntervalHour(23) + " + "toIntervalMinute(59) + toIntervalSecond(60) - toIntervalMicrosecond(1)", + datetime_str, + toString(offset)); + + return true; +} + +bool EndOfDay::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String datetime_str = getConvertedArgument(fn_name, pos); + String offset = "0"; + + if (pos->type == TokenType::Comma) + { + ++pos; + offset = getConvertedArgument(fn_name, pos); + } + out = std::format( + "toDateTime(toStartOfDay({}),9,'UTC') + (INTERVAL {} +1 DAY) - (INTERVAL 1 microsecond)", datetime_str, toString(offset)); + + return true; +} + +bool EndOfWeek::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String datetime_str = getConvertedArgument(fn_name, pos); + String offset = "0"; + + if (pos->type == TokenType::Comma) + { + ++pos; + offset = getConvertedArgument(fn_name, pos); + } + out = std::format( + "toDateTime(toStartOfDay({}),9,'UTC') + (INTERVAL {} +1 WEEK) - (INTERVAL 1 microsecond)", datetime_str, toString(offset)); + + return true; +} + +bool EndOfYear::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String datetime_str = getConvertedArgument(fn_name, pos); + + if (datetime_str.empty()) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Number of arguments do not match in function: {}", fn_name); + + String offset = "0"; + if (pos->type == TokenType::Comma) + { + ++pos; + offset = getConvertedArgument(fn_name, pos); + if (offset.empty()) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Number of arguments do not match in function: {}", fn_name); + offset.erase(remove(offset.begin(), offset.end(), ' '), offset.end()); + } + + out = std::format( + "(((((toDateTime(toString(toLastDayOfMonth(toDateTime({0}, 9, 'UTC') + toIntervalYear({1}) + toIntervalMonth(12 - " + "toInt8(substring(toString(toDateTime({0}, 9, 'UTC')), 6, 2))))), 9, 'UTC') + toIntervalHour(23)) + toIntervalMinute(59)) + " + "toIntervalSecond(60)) - toIntervalMicrosecond(1)))", + datetime_str, + toString(offset)); + + return true; +} + +bool FormatDateTime::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + String formatspecifier; + ++pos; + const auto datetime = getConvertedArgument(fn_name, pos); + ++pos; + auto format = getConvertedArgument(fn_name, pos); + trim(format); + + //remove quotes and end space from format argument. + if (format.front() == '\"' || format.front() == '\'') + { + format.erase(0, 1); // erase the first quote + format.erase(format.size() - 1); // erase the last quote + } + + std::vector res; + getTokens(format, res); + std::string::size_type i = 0; + size_t decimal = 0; + while (i < format.size()) + { + char c = format[i]; + if (!isalpha(c)) + { + //delimiter + if (c == ' ' || c == '-' || c == '_' || c == '[' || c == ']' || c == '/' || c == ',' || c == '.' || c == ':') + formatspecifier = formatspecifier + c; + else + throw Exception(ErrorCodes::SYNTAX_ERROR, "Invalid format delimiter in function: {}", fn_name); + ++i; + } + else + { + //format specifier + String arg = res.back(); + + if (arg == "y" || arg == "yy") + formatspecifier = formatspecifier + "%y"; + else if (arg == "yyyy") + formatspecifier = formatspecifier + "%Y"; + else if (arg == "M" || arg == "MM") + formatspecifier = formatspecifier + "%m"; + else if (arg == "s" || arg == "ss") + formatspecifier = formatspecifier + "%S"; + else if (arg == "m" || arg == "mm") + formatspecifier = formatspecifier + "%i"; + else if (arg == "h" || arg == "hh") + formatspecifier = formatspecifier + "%h"; + else if (arg == "H" || arg == "HH") + formatspecifier = formatspecifier + "%H"; + else if (arg == "d") + formatspecifier = formatspecifier + "%e"; + else if (arg == "dd") + formatspecifier = formatspecifier + "%d"; + else if (arg == "tt") + formatspecifier = formatspecifier + "%p"; + else if (arg.starts_with('f') || arg.starts_with('F')) + decimal = arg.size(); + else + throw Exception(ErrorCodes::SYNTAX_ERROR, "Format specifier {} in function: {} is not supported", arg, fn_name); + res.pop_back(); + i = i + arg.size(); + } + } + if (decimal > 0 && formatspecifier.find('.') != String::npos) + { + out = std::format( + "concat(" + "substring(toString(formatDateTime({0} , '{1}')),1, position(toString(formatDateTime({0},'{1}')),'.')) ," + "substring(substring(toString({0}), position(toString({0}),'.')+1),1,{2})," + "substring(toString(formatDateTime({0},'{1}')), position(toString(formatDateTime({0},'{1}')), '.')+1, length " + "(toString(formatDateTime({0},'{1}'))))) ", + datetime, + formatspecifier, + decimal); + } + else + out = std::format("formatDateTime({0},'{1}')", datetime, formatspecifier); + + return true; +} +bool FormatTimeSpan::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + String formatspecifier; + ++pos; + const auto datetime = getConvertedArgument(fn_name, pos); + ++pos; + auto format = getConvertedArgument(fn_name, pos); + size_t decimal = 0; + trim(format); + //remove quotes and end space from format argument. + if (format.front() == '\"' || format.front() == '\'') + { + format.erase(0, 1); // erase the first quote + format.erase(format.size() - 1); // erase the last quote + } + std::vector res; + getTokens(format, res); + size_t pad = 0; + std::string::size_type i = 0; + + + bool is_day_in_format = false; + String day_val = std::to_string(std::stoi(datetime) / 86400); + bool is_hour_zero = std::stoi(datetime) % 86400 > 3600 ? false : true; + + while (i < format.size()) + { + char c = format[i]; + if (!isalpha(c)) + { + //delimiter + if (c == ' ' || c == '-' || c == '_' || c == '[' || c == ']' || c == '/' || c == ',' || c == '.' || c == ':') + formatspecifier = formatspecifier + c; + else + throw Exception(ErrorCodes::SYNTAX_ERROR, "Invalid format delimiter in function: {}", fn_name); + ++i; + } + else + { + //format specifier + String arg = res.back(); + + if (arg == "s" || arg == "ss") + formatspecifier = formatspecifier + "%S"; + else if (arg == "m" || arg == "mm") + formatspecifier = formatspecifier + "%i"; + else if (arg == "h" || arg == "hh") + { + if (is_hour_zero) //To handle the CH limit for 12hr format(01-12). If not handled, 1.00:00:00 returned as 1.12:00:00(in 12 hr format) + formatspecifier = formatspecifier + "%h"; + else + formatspecifier = formatspecifier + "%H"; + } + else if (arg == "H" || arg == "HH") + formatspecifier = formatspecifier + "%H"; + else if (arg.starts_with('d')) //&& arg.size() >2) + { + pad = std::max(arg.size(), day_val.length()); + is_day_in_format = true; + } + else if (arg.starts_with('f') || arg.starts_with('F')) + decimal = arg.size(); + else + throw Exception(ErrorCodes::SYNTAX_ERROR, "Format specifier {} in function:{} is not supported", arg, fn_name); + res.pop_back(); + i = i + arg.size(); + } + } + auto last_delim = formatspecifier.substr(formatspecifier.length() - 1, formatspecifier.length()); + + if (!is_day_in_format) + { + if (decimal > 0) + { + if (format.substr(format.length() - decimal - 1, 1) == last_delim) + out = std::format( + "concat(substring(toString(formatDateTime(toDateTime64({0},9,'UTC') ,'{1}')),1, length(toString(formatDateTime(" + "toDateTime64({0},9,'UTC'),'{1}'))) - position(" + "reverse(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}'))),'{3}')+1),substring(SUBSTRING(toString(" + "toDateTime64({0},9,'UTC')),position(toString(toDateTime64({0},9,'UTC')),'.')+1),1,{2}))", + datetime, + formatspecifier, + decimal, + last_delim); + else + out = std::format( + "concat(substring(toString(formatDateTime(toDateTime64({0},9,'UTC') ,'{1}')),1, length(toString(formatDateTime(" + "toDateTime64({0},9,'UTC'),'{1}'))) - position(" + "reverse(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}'))),'{3}')),substring(SUBSTRING(toString(toDateTime64({" + "0},9,'UTC')),position(toString(toDateTime64({0},9,'UTC')),'.')+1),1,{2}))", + datetime, + formatspecifier, + decimal, + last_delim); + } + else + out = std::format("formatDateTime(toDateTime64({0},9,'UTC'),'{1}')", datetime, formatspecifier); + } + else + { + if (decimal > 0) + { + if (format.substr(format.length() - decimal - 1, 1) == last_delim) + out = std::format( + "concat(leftPad('{5}', {3},'0'),substring(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}')),1," + "length(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}'))) - position(" + "reverse(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}'))),'{4}') " + "+1),substring(SUBSTRING(toString(toDateTime64({0},9,'UTC')),position(toString(toDateTime64({0},9,'UTC')),'.')+1),1,{2}" + "))", + datetime, + formatspecifier, + decimal, + pad, + last_delim, + day_val); + else + out = std::format( + "concat(leftPad('{5}', {3}, '0'),substring(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}')),1," + "length(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}'))) - position(" + "reverse(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}'))),'{4}')),substring(SUBSTRING(toString(toDateTime64({" + "0},9,'UTC')),position(toString(toDateTime64({0},9,'UTC')),'.')+1),1,{2}),substring(toString(formatDateTime(" + "toDateTime64({0},9,'UTC'),'{1}')),position(toString(formatDateTime(" + "toDateTime64({0},9,'UTC'),'{1}')),'{4}'),length(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}')))))", + datetime, + formatspecifier, + decimal, + pad, + last_delim, + day_val); + } + else if (decimal == 0) + out = std::format( + "concat(leftPad('{3}',{2},'0'),toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}')))", + datetime, + formatspecifier, + pad, + day_val); + } + return true; +} + +bool GetMonth::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "toMonth"); +} + +bool GetYear::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "toYear"); +} + +bool HoursOfDay::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "toHour"); +} + +bool MakeTimeSpan::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + + ++pos; + String datetime_str; + String hour; + String day; + String minute; + String second; + int arg_count = 0; + std::vector args; + while (!pos->isEnd() && pos->type != TokenType::ClosingRoundBracket) + { + String arg = getConvertedArgument(fn_name, pos); + args.insert(args.begin(), arg); + if (pos->type == TokenType::Comma) + ++pos; + ++arg_count; + } + + if (arg_count < 2 || arg_count > 4) + throw Exception(ErrorCodes::SYNTAX_ERROR, "argument count out of bound in function: {}", fn_name); + + if (arg_count == 2) + { + hour = args.back(); + args.pop_back(); + minute = args.back(); + args.pop_back(); + datetime_str = hour + ":" + minute; + } + else if (arg_count == 3) + { + hour = args.back(); + args.pop_back(); + minute = args.back(); + args.pop_back(); + second = args.back(); + args.pop_back(); + + datetime_str = hour + ":" + minute + ":" + second; + } + else if (arg_count == 4) + { + day = args.back(); + args.pop_back(); + hour = args.back(); + args.pop_back(); + minute = args.back(); + args.pop_back(); + second = args.back(); + args.pop_back(); + + datetime_str = hour + ":" + minute + ":" + second; + day = day + "."; + } + else + throw Exception(ErrorCodes::SYNTAX_ERROR, "argument count out of bound in function: {}", fn_name); + + //Add dummy yyyy-mm-dd to parse datetime in CH + datetime_str = "0000-00-00 " + datetime_str; + + out = std::format( + "CONCAT('{}',toString(SUBSTRING(toString(toTime(parseDateTime64BestEffortOrNull('{}', 9,'UTC'))),12)))", day, datetime_str); + return true; +} + +bool MakeDateTime::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + + ++pos; + String arguments; + int arg_count = 0; + + while (!pos->isEnd() && pos->type != TokenType::ClosingRoundBracket) + { + String arg = getConvertedArgument(fn_name, pos); + if (pos->type == TokenType::Comma) + ++pos; + arguments = arguments + arg + ","; + ++arg_count; + } + + if (arg_count < 1 || arg_count > 7) + throw Exception(ErrorCodes::SYNTAX_ERROR, "argument count out of bound in function: {}", fn_name); + + if (arg_count < 7) + { + for (int i = arg_count; i < 7; ++i) + arguments = arguments + "0,"; + } + + arguments = arguments + "7,'UTC'"; + out = std::format("makeDateTime64({})", arguments); + + return true; +} + +bool Now::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + if (pos->type != TokenType::ClosingRoundBracket) + { + const auto offset = getConvertedArgument(fn_name, pos); + out = std::format("now64(9,'UTC') + {}", offset); + } + else + out = "now64(9,'UTC')"; + return true; +} + +bool StartOfDay::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + + ++pos; + const String datetime_str = getConvertedArgument(fn_name, pos); + String offset = "0"; + + if (pos->type == TokenType::Comma) + { + ++pos; + offset = getConvertedArgument(fn_name, pos); + } + out = std::format("date_add(DAY,{}, parseDateTime64BestEffortOrNull(toString((toStartOfDay({}))), 9, 'UTC')) ", offset, datetime_str); + return true; +} + +bool StartOfMonth::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String datetime_str = getConvertedArgument(fn_name, pos); + String offset = "0"; + + if (pos->type == TokenType::Comma) + { + ++pos; + offset = getConvertedArgument(fn_name, pos); + } + out = std::format( + "date_add(MONTH,{}, parseDateTime64BestEffortOrNull(toString((toStartOfMonth({}))), 9, 'UTC')) ", offset, datetime_str); + return true; +} + +bool StartOfWeek::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String datetime_str = getConvertedArgument(fn_name, pos); + String offset = "0"; + + if (pos->type == TokenType::Comma) + { + ++pos; + offset = getConvertedArgument(fn_name, pos); + } + out = std::format( + "date_add(Week,{}, parseDateTime64BestEffortOrNull(toString((toStartOfWeek({}))), 9, 'UTC')) ", offset, datetime_str); + return true; +} + +bool StartOfYear::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String datetime_str = getConvertedArgument(fn_name, pos); + String offset = "0"; + + if (pos->type == TokenType::Comma) + { + ++pos; + offset = getConvertedArgument(fn_name, pos); + } + out = std::format( + "date_add(YEAR,{}, parseDateTime64BestEffortOrNull(toString((toStartOfYear({}, 'UTC'))), 9, 'UTC'))", offset, datetime_str); + return true; +} + +bool UnixTimeMicrosecondsToDateTime::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String value = getConvertedArgument(fn_name, pos); + + out = std::format("fromUnixTimestamp64Micro({},'UTC')", value); + return true; +} + +bool UnixTimeMillisecondsToDateTime::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String value = getConvertedArgument(fn_name, pos); + + out = std::format("fromUnixTimestamp64Milli({},'UTC')", value); + return true; +} + +bool UnixTimeNanosecondsToDateTime::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String value = getConvertedArgument(fn_name, pos); + + out = std::format("fromUnixTimestamp64Nano({},'UTC')", value); + return true; +} + +bool UnixTimeSecondsToDateTime::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} accepts only long, int and double type of arguments", fn_name); + + String expression = getConvertedArgument(fn_name, pos); + out = std::format( + " if(toTypeName({0}) = 'Int64' OR toTypeName({0}) = 'Int32'OR toTypeName({0}) = 'Float64' OR toTypeName({0}) = 'UInt32' OR " + "toTypeName({0}) = 'UInt64', toDateTime64({0}, 9, 'UTC'), toDateTime64(throwIf(true, '{1} only accepts Int, Long and double type " + "of arguments'), 9, 'UTC'))", + expression, + fn_name); + + return true; +} + +bool WeekOfYear::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + ++pos; + const String time_str = getConvertedArgument(fn_name, pos); + out = std::format("toWeek({},3,'UTC')", time_str); + return true; +} + +bool MonthOfYear::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "toMonth"); +} + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.h b/src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.h new file mode 100644 index 00000000000..77ed110700b --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.h @@ -0,0 +1,250 @@ +#pragma once + +#include +#include +namespace DB +{ + +class TimeSpan : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "timespan()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; +/* +class DateTime : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "datetime()"; } + bool convertImpl(String &out,IParser::Pos &pos) override; +};*/ + + +class Ago : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "ago()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DatetimeAdd : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "datetime_add()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DatetimePart : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "datetime_part()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DatetimeDiff : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "datetime_diff()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DayOfMonth : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "dayofmonth()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DayOfWeek : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "dayofweek()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class DayOfYear : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "dayofyear()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class EndOfDay : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "endofday()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class EndOfMonth : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "endofmonth()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class EndOfWeek : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "endofweek()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class EndOfYear : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "endofyear()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class FormatDateTime : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "format_datetime()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class FormatTimeSpan : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "format_timespan()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class GetMonth : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "getmonth()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class GetYear : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "getyear()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class HoursOfDay : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "hourofday()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class MakeTimeSpan : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "make_timespan()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class MakeDateTime : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "make_datetime()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Now : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "now()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class StartOfDay : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "startofday()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class StartOfMonth : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "startofmonth()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class StartOfWeek : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "startofweek()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class StartOfYear : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "startofyear()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class UnixTimeMicrosecondsToDateTime : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "unixtime_microseconds_todatetime()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class UnixTimeMillisecondsToDateTime : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "unixtime_milliseconds_todatetime()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class UnixTimeNanosecondsToDateTime : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "unixtime_nanoseconds_todatetime()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class UnixTimeSecondsToDateTime : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "unixtime_seconds_todatetime()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class WeekOfYear : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "week_of_year()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class MonthOfYear : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "monthofyear()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +void inline getTokens(String format, std::vector & res) +{ + String str = format; + String token; + auto pos = str.find_first_not_of("abcdefghijklmnopqrstuvwxyzQWERTYUIOPASDFGHJKLZXCVBNM"); + while (pos != String::npos) + { + if (pos != 0) + { + // Found a token + token = str.substr(0, pos); + res.insert(res.begin(), token); + } + str.erase(0, pos + 1); // Always remove pos+1 to get rid of delimiter + pos = str.find_first_not_of("abcdefghijklmnopqrstuvwxyzQWERTYUIOPASDFGHJKLZXCVBNM"); + } + // Cover the last (or only) token + if (str.length() > 0) + { + token = str; + res.insert(res.begin(), token); + } +} + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.cpp b/src/Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.cpp new file mode 100644 index 00000000000..924ac9f6490 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.cpp @@ -0,0 +1,377 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int SYNTAX_ERROR; +} + +bool ArrayConcat::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "arrayConcat"); +} + +bool ArrayIif::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto conditions = getArgument(function_name, pos); + const auto if_true = getArgument(function_name, pos); + const auto if_false = getArgument(function_name, pos); + + out = std::format( + "arrayMap(x -> multiIf(toTypeName(x.1) = 'String', null, toInt64(x.1) != 0, x.2, x.3), " + "arrayZip({0}, arrayResize({1}, length({0}), null), arrayResize({2}, length({0}), null)))", + conditions, + if_true, + if_false); + + return true; +} + +bool ArrayIndexOf::convertImpl(String & out, IParser::Pos & pos) +{ + const auto fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + const auto array = getArgument(fn_name, pos); + const auto needle = getArgument(fn_name, pos); + out = "minus(indexOf(" + array + ", " + needle + "), 1)"; + + return true; +} + +bool ArrayLength::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "length"); +} + +bool ArrayReverse::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto array = getArgument(function_name, pos); + out = std::format("if(throwIf(not startsWith(toTypeName({0}), 'Array'), 'Only arrays are supported'), [], reverse({0}))", array); + + return true; +} + +bool ArrayRotateLeft::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto array = getArgument(function_name, pos); + const auto count = getArgument(function_name, pos); + out = std::format( + "arrayMap(x -> {0}[moduloOrZero(x + length({0}) + moduloOrZero({1}, toInt64(length({0}))), length({0})) + 1], " + "range(0, length({0})))", + array, + count); + + return true; +} + +bool ArrayRotateRight::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto array = getArgument(function_name, pos, ArgumentState::Raw); + const auto count = getArgument(function_name, pos, ArgumentState::Raw); + out = kqlCallToExpression("array_rotate_left", {array, "-1 * " + count}, pos.max_depth); + + return true; +} + +bool ArrayShiftLeft::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto array = getArgument(function_name, pos); + const auto count = getArgument(function_name, pos); + const auto fill = getOptionalArgument(function_name, pos); + out = std::format( + "arrayResize(if({1} > 0, arraySlice({0}, {1} + 1), arrayConcat(arrayWithConstant(abs({1}), fill_value_{3}), {0})), " + "length({0}), if(isNull({2}) and (extract(toTypeName({0}), 'Array\\((.*)\\)') as element_type_{3}) = 'String', " + "defaultValueOfTypeName(if(element_type_{3} = 'Nothing', 'Nullable(Nothing)', element_type_{3})), {2}) as fill_value_{3})", + array, + count, + fill ? *fill : "null", + generateUniqueIdentifier()); + + return true; +} + +bool ArrayShiftRight::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto array = getArgument(function_name, pos, ArgumentState::Raw); + const auto count = getArgument(function_name, pos, ArgumentState::Raw); + const auto fill = getOptionalArgument(function_name, pos, ArgumentState::Raw); + + const auto negated_count = "-1 * " + count; + out = kqlCallToExpression( + "array_shift_left", + fill ? std::initializer_list{array, negated_count, *fill} + : std::initializer_list{array, negated_count}, + pos.max_depth); + + return true; +} + +bool ArraySlice::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto array = getArgument(function_name, pos); + const auto start = getArgument(function_name, pos); + const auto end = getArgument(function_name, pos); + + out = std::format( + "arraySlice({0}, plus(1, if({1} >= 0, {1}, arrayMax([-length({0}), {1}]) + length({0}))) as offset_{3}, " + " plus(1, if({2} >= 0, {2}, arrayMax([-length({0}), {2}]) + length({0}))) - offset_{3} + 1)", + array, + start, + end, + generateUniqueIdentifier()); + + return true; +} + +bool ArraySortAsc::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "kql_array_sort_asc"); +} + +bool ArraySortDesc::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "kql_array_sort_desc"); +} + +bool ArraySplit::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto array = getArgument(function_name, pos); + const auto indices = getArgument(function_name, pos); + + out = std::format( + "if(empty(arrayMap(x -> if(x >= 0, x, arrayMax([0, x + length({0})::Int64])), flatten([{1}])) as indices_{2}), [{0}], " + "arrayConcat([arraySlice({0}, 1, indices_{2}[1])], arrayMap(i -> arraySlice({0}, indices_{2}[i] + 1, " + "if(i = length(indices_{2}), length({0})::Int64, indices_{2}[i + 1]::Int64) - indices_{2}[i]), " + "range(1, length(indices_{2}) + 1))))", + array, + indices, + generateUniqueIdentifier()); + + return true; +} + +bool ArraySum::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "arraySum"); +} + +bool BagKeys::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool BagMerge::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool BagRemoveKeys::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool JaccardIndex::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto lhs = getArgument(function_name, pos, ArgumentState::Raw); + const auto rhs = getArgument(function_name, pos, ArgumentState::Raw); + out = std::format( + "divide(length({0}), length({1}))", + kqlCallToExpression("set_intersect", {lhs, rhs}, pos.max_depth), + kqlCallToExpression("set_union", {lhs, rhs}, pos.max_depth)); + + return true; +} + +bool Pack::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool PackAll::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool PackArray::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "array", {1, Interval::max_bound}); +} + +bool Repeat::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + String value = getArgument(function_name, pos); + String count = getArgument(function_name, pos); + + value.erase(remove(value.begin(), value.end(), ' '), value.end()); + count.erase(remove(count.begin(), count.end(), ' '), count.end()); + + if (count.empty()) + throw Exception(ErrorCodes::SYNTAX_ERROR, "number of arguments do not match in function: {}", function_name); + else + out = "if(" + count + " < 0, [NULL], " + std::format("arrayWithConstant(abs({1}), {0}))", value, count); + + return true; +} + +bool SetDifference::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto lhs = getArgument(function_name, pos); + const auto rhs = std::invoke( + [&function_name, &pos] + { + std::vector arrays{getArgument(function_name, pos, ArgumentState::Raw)}; + while (auto next_array = getOptionalArgument(function_name, pos, ArgumentState::Raw)) + arrays.push_back(*next_array); + + return kqlCallToExpression("set_union", std::vector(arrays.cbegin(), arrays.cend()), pos.max_depth); + }); + + out = std::format("arrayFilter(x -> not has({1}, x), arrayDistinct({0}))", lhs, rhs); + + return true; +} + +bool SetHasElement::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "has"); +} + +bool SetIntersect::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "arrayIntersect"); +} + +bool SetUnion::convertImpl(String & out, IParser::Pos & pos) +{ + if (!directMapping(out, pos, "arrayConcat")) + return false; + + out = std::format("arrayDistinct({0})", out); + + return true; +} + +bool TreePath::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool Zip::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto arguments = std::invoke( + [&function_name, &pos] + { + std::vector result; + while (auto argument = getOptionalArgument(function_name, pos)) + result.push_back(std::move(*argument)); + + return result; + }); + + if (const auto size = arguments.size(); size < 2 || size > 16) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Between 2 and 16 arguments are expected, but {} were provided", size); + + const auto unique_identifier = generateUniqueIdentifier(); + const auto resized_arguments = std::invoke( + [&arguments, &unique_identifier] + { + String lengths; + for (int i = 0; i < std::ssize(arguments); ++i) + { + lengths.append(i > 0 ? ", " : ""); + lengths.append(std::format( + "length(if(match(toTypeName({0}), 'Array\\(Nullable\\(.*\\)\\)'), {0}, " + "cast({0}, concat('Array(Nullable(', extract(toTypeName({0}), 'Array\\((.*)\\)'), '))'))) as arg{1}_{2})", + arguments[i], + i, + unique_identifier)); + } + + auto result = std::format("arrayResize(arg0_{1}, arrayMax([{0}]) as max_length_{1}, null)", lengths, unique_identifier); + for (int i = 1; i < std::ssize(arguments); ++i) + result.append(std::format(", arrayResize(arg{0}_{1}, max_length_{1}, null)", i, unique_identifier)); + + return result; + }); + + out = std::format("arrayMap(t -> [untuple(t)], arrayZip({0}))", resized_arguments); + + return true; +} +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.h b/src/Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.h new file mode 100644 index 00000000000..52c4c637d8a --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.h @@ -0,0 +1,203 @@ +#pragma once + +#include +#include +namespace DB +{ +class ArrayConcat : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "array_concat()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ArrayIif : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "array_iif()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ArrayIndexOf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "array_index_of()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ArrayLength : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "array_length()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ArrayReverse : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "array_reverse()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ArrayRotateLeft : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "array_rotate_left()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ArrayRotateRight : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "array_rotate_right()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ArrayShiftLeft : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "array_shift_left()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ArrayShiftRight : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "array_shift_right()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ArraySlice : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "array_slice()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ArraySortAsc : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "array_sort_asc()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ArraySortDesc : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "array_sort_desc()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ArraySplit : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "array_split()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ArraySum : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "array_sum()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class BagKeys : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "bag_keys()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class BagMerge : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "bag_merge()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class BagRemoveKeys : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "bag_remove_keys()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class JaccardIndex : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "jaccard_index()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Pack : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "pack()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class PackAll : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "pack_all()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class PackArray : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "pack_array()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Repeat : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "repeat()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SetDifference : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "set_difference()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SetHasElement : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "set_has_element()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SetIntersect : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "set_intersect()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SetUnion : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "set_union()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class TreePath : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "treepath()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Zip : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "zip()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLFunctionFactory.cpp b/src/Parsers/Kusto/KustoFunctions/KQLFunctionFactory.cpp new file mode 100644 index 00000000000..adac892b49d --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLFunctionFactory.cpp @@ -0,0 +1,819 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +std::unordered_map KQLFunctionFactory::kql_functions + = {{"ago", KQLFunctionValue::ago}, + {"datetime_add", KQLFunctionValue::datetime_add}, + {"datetime_part", KQLFunctionValue::datetime_part}, + {"datetime_diff", KQLFunctionValue::datetime_diff}, + {"dayofmonth", KQLFunctionValue::dayofmonth}, + {"dayofweek", KQLFunctionValue::dayofweek}, + {"dayofyear", KQLFunctionValue::dayofyear}, + {"endofday", KQLFunctionValue::endofday}, + {"endofweek", KQLFunctionValue::endofweek}, + {"endofyear", KQLFunctionValue::endofyear}, + {"endofmonth", KQLFunctionValue::endofmonth}, + + {"format_datetime", KQLFunctionValue::format_datetime}, + {"format_timespan", KQLFunctionValue::format_timespan}, + {"getmonth", KQLFunctionValue::getmonth}, + {"getyear", KQLFunctionValue::getyear}, + {"hourofday", KQLFunctionValue::hourofday}, + {"make_timespan", KQLFunctionValue::make_timespan}, + {"make_datetime", KQLFunctionValue::make_datetime}, + {"now", KQLFunctionValue::now}, + {"startofday", KQLFunctionValue::startofday}, + {"startofmonth", KQLFunctionValue::startofmonth}, + {"startofweek", KQLFunctionValue::startofweek}, + {"startofyear", KQLFunctionValue::startofyear}, + {"todatetime", KQLFunctionValue::todatetime}, + {"totimespan", KQLFunctionValue::totimespan}, + {"unixtime_microseconds_todatetime", KQLFunctionValue::unixtime_microseconds_todatetime}, + {"unixtime_milliseconds_todatetime", KQLFunctionValue::unixtime_milliseconds_todatetime}, + {"unixtime_nanoseconds_todatetime", KQLFunctionValue::unixtime_nanoseconds_todatetime}, + {"unixtime_seconds_todatetime", KQLFunctionValue::unixtime_seconds_todatetime}, + {"week_of_year", KQLFunctionValue::week_of_year}, + {"monthofyear", KQLFunctionValue::monthofyear}, + {"base64_encode_tostring", KQLFunctionValue::base64_encode_tostring}, + {"base64_encode_fromguid", KQLFunctionValue::base64_encode_fromguid}, + {"base64_decode_tostring", KQLFunctionValue::base64_decode_tostring}, + {"base64_decode_toarray", KQLFunctionValue::base64_decode_toarray}, + {"base64_decode_toguid", KQLFunctionValue::base64_decode_toguid}, + {"countof", KQLFunctionValue::countof}, + {"extract", KQLFunctionValue::extract}, + {"extract_all", KQLFunctionValue::extract_all}, + {"extract_json", KQLFunctionValue::extract_json}, + {"extractjson", KQLFunctionValue::extract_json}, + {"has_any_index", KQLFunctionValue::has_any_index}, + {"indexof", KQLFunctionValue::indexof}, + {"isempty", KQLFunctionValue::isempty}, + {"isnan", KQLFunctionValue::isnan}, + {"isnotempty", KQLFunctionValue::isnotempty}, + {"notempty", KQLFunctionValue::isnotempty}, + {"isnotnull", KQLFunctionValue::isnotnull}, + {"notnull", KQLFunctionValue::isnotnull}, + {"isnull", KQLFunctionValue::isnull}, + {"parse_command_line", KQLFunctionValue::parse_command_line}, + {"parse_csv", KQLFunctionValue::parse_csv}, + {"parse_json", KQLFunctionValue::parse_json}, + {"parse_url", KQLFunctionValue::parse_url}, + {"parse_urlquery", KQLFunctionValue::parse_urlquery}, + {"parse_version", KQLFunctionValue::parse_version}, + {"replace_regex", KQLFunctionValue::replace_regex}, + {"reverse", KQLFunctionValue::reverse}, + {"split", KQLFunctionValue::split}, + {"strcat", KQLFunctionValue::strcat}, + {"strcat_delim", KQLFunctionValue::strcat_delim}, + {"strcmp", KQLFunctionValue::strcmp}, + {"strlen", KQLFunctionValue::strlen}, + {"strrep", KQLFunctionValue::strrep}, + {"substring", KQLFunctionValue::substring}, + {"tolower", KQLFunctionValue::tolower}, + {"toupper", KQLFunctionValue::toupper}, + {"translate", KQLFunctionValue::translate}, + {"trim", KQLFunctionValue::trim}, + {"trim_end", KQLFunctionValue::trim_end}, + {"trim_start", KQLFunctionValue::trim_start}, + {"url_decode", KQLFunctionValue::url_decode}, + {"url_encode", KQLFunctionValue::url_encode}, + + {"array_concat", KQLFunctionValue::array_concat}, + {"array_iff", KQLFunctionValue::array_iif}, + {"array_iif", KQLFunctionValue::array_iif}, + {"array_index_of", KQLFunctionValue::array_index_of}, + {"array_length", KQLFunctionValue::array_length}, + {"array_reverse", KQLFunctionValue::array_reverse}, + {"array_rotate_left", KQLFunctionValue::array_rotate_left}, + {"array_rotate_right", KQLFunctionValue::array_rotate_right}, + {"array_shift_left", KQLFunctionValue::array_shift_left}, + {"array_shift_right", KQLFunctionValue::array_shift_right}, + {"array_slice", KQLFunctionValue::array_slice}, + {"array_sort_asc", KQLFunctionValue::array_sort_asc}, + {"array_sort_desc", KQLFunctionValue::array_sort_desc}, + {"array_split", KQLFunctionValue::array_split}, + {"array_sum", KQLFunctionValue::array_sum}, + {"bag_keys", KQLFunctionValue::bag_keys}, + {"bag_merge", KQLFunctionValue::bag_merge}, + {"bag_remove_keys", KQLFunctionValue::bag_remove_keys}, + {"jaccard_index", KQLFunctionValue::jaccard_index}, + {"pack", KQLFunctionValue::pack}, + {"pack_all", KQLFunctionValue::pack_all}, + {"pack_array", KQLFunctionValue::pack_array}, + {"repeat", KQLFunctionValue::repeat}, + {"set_difference", KQLFunctionValue::set_difference}, + {"set_has_element", KQLFunctionValue::set_has_element}, + {"set_intersect", KQLFunctionValue::set_intersect}, + {"set_union", KQLFunctionValue::set_union}, + {"treepath", KQLFunctionValue::treepath}, + {"zip", KQLFunctionValue::zip}, + + {"tobool", KQLFunctionValue::tobool}, + {"toboolean", KQLFunctionValue::tobool}, + {"todouble", KQLFunctionValue::todouble}, + {"toint", KQLFunctionValue::toint}, + {"tolong", KQLFunctionValue::tolong}, + {"toreal", KQLFunctionValue::todouble}, + {"tostring", KQLFunctionValue::tostring}, + {"totimespan", KQLFunctionValue::totimespan}, + {"todecimal", KQLFunctionValue::todecimal}, + + {"arg_max", KQLFunctionValue::arg_max}, + {"arg_min", KQLFunctionValue::arg_min}, + {"avg", KQLFunctionValue::avg}, + {"avgif", KQLFunctionValue::avgif}, + {"binary_all_and", KQLFunctionValue::binary_all_and}, + {"binary_all_or", KQLFunctionValue::binary_all_or}, + {"binary_all_xor", KQLFunctionValue::binary_all_xor}, + {"buildschema", KQLFunctionValue::buildschema}, + {"count", KQLFunctionValue::count}, + {"countif", KQLFunctionValue::countif}, + {"dcount", KQLFunctionValue::dcount}, + {"dcountif", KQLFunctionValue::dcountif}, + {"make_bag", KQLFunctionValue::make_bag}, + {"make_bag_if", KQLFunctionValue::make_bag_if}, + {"make_list", KQLFunctionValue::make_list}, + {"make_list_if", KQLFunctionValue::make_list_if}, + {"make_list_with_nulls", KQLFunctionValue::make_list_with_nulls}, + {"make_set", KQLFunctionValue::make_set}, + {"make_set_if", KQLFunctionValue::make_set_if}, + {"max", KQLFunctionValue::max}, + {"maxif", KQLFunctionValue::maxif}, + {"min", KQLFunctionValue::min}, + {"minif", KQLFunctionValue::minif}, + {"percentile", KQLFunctionValue::percentile}, + {"percentilew", KQLFunctionValue::percentilew}, + {"percentiles", KQLFunctionValue::percentiles}, + {"percentiles_array", KQLFunctionValue::percentiles_array}, + {"percentilesw", KQLFunctionValue::percentilesw}, + {"percentilesw_array", KQLFunctionValue::percentilesw_array}, + {"stdev", KQLFunctionValue::stdev}, + {"stdevif", KQLFunctionValue::stdevif}, + {"sum", KQLFunctionValue::sum}, + {"sumif", KQLFunctionValue::sumif}, + {"take_any", KQLFunctionValue::take_any}, + {"take_anyif", KQLFunctionValue::take_anyif}, + {"variance", KQLFunctionValue::variance}, + {"varianceif", KQLFunctionValue::varianceif}, + + {"series_fir", KQLFunctionValue::series_fir}, + {"series_iir", KQLFunctionValue::series_iir}, + {"series_fit_line", KQLFunctionValue::series_fit_line}, + {"series_fit_line_dynamic", KQLFunctionValue::series_fit_line_dynamic}, + {"series_fit_2lines", KQLFunctionValue::series_fit_2lines}, + {"series_fit_2lines_dynamic", KQLFunctionValue::series_fit_2lines_dynamic}, + {"series_outliers", KQLFunctionValue::series_outliers}, + {"series_periods_detect", KQLFunctionValue::series_periods_detect}, + {"series_periods_validate", KQLFunctionValue::series_periods_validate}, + {"series_stats_dynamic", KQLFunctionValue::series_stats_dynamic}, + {"series_stats", KQLFunctionValue::series_stats}, + {"series_fill_backward", KQLFunctionValue::series_fill_backward}, + {"series_fill_const", KQLFunctionValue::series_fill_const}, + {"series_fill_forward", KQLFunctionValue::series_fill_forward}, + {"series_fill_linear", KQLFunctionValue::series_fill_linear}, + + {"ipv4_compare", KQLFunctionValue::ipv4_compare}, + {"ipv4_is_in_range", KQLFunctionValue::ipv4_is_in_range}, + {"ipv4_is_match", KQLFunctionValue::ipv4_is_match}, + {"ipv4_is_private", KQLFunctionValue::ipv4_is_private}, + {"ipv4_netmask_suffix", KQLFunctionValue::ipv4_netmask_suffix}, + {"parse_ipv4", KQLFunctionValue::parse_ipv4}, + {"parse_ipv4_mask", KQLFunctionValue::parse_ipv4_mask}, + {"ipv6_compare", KQLFunctionValue::ipv6_compare}, + {"ipv6_is_match", KQLFunctionValue::ipv6_is_match}, + {"parse_ipv6", KQLFunctionValue::parse_ipv6}, + {"parse_ipv6_mask", KQLFunctionValue::parse_ipv6_mask}, + {"format_ipv4", KQLFunctionValue::format_ipv4}, + {"format_ipv4_mask", KQLFunctionValue::format_ipv4_mask}, + + {"binary_and", KQLFunctionValue::binary_and}, + {"binary_not", KQLFunctionValue::binary_not}, + {"binary_or", KQLFunctionValue::binary_or}, + {"binary_shift_left", KQLFunctionValue::binary_shift_left}, + {"binary_shift_right", KQLFunctionValue::binary_shift_right}, + {"binary_xor", KQLFunctionValue::binary_xor}, + {"bitset_count_ones", KQLFunctionValue::bitset_count_ones}, + + {"bin", KQLFunctionValue::bin}, + {"bin_at", KQLFunctionValue::bin_at}, + + {"bool", KQLFunctionValue::datatype_bool}, + {"boolean", KQLFunctionValue::datatype_bool}, + {"datetime", KQLFunctionValue::datatype_datetime}, + {"date", KQLFunctionValue::datatype_datetime}, + {"dynamic", KQLFunctionValue::datatype_dynamic}, + {"guid", KQLFunctionValue::datatype_guid}, + {"int", KQLFunctionValue::datatype_int}, + {"long", KQLFunctionValue::datatype_long}, + {"real", KQLFunctionValue::datatype_real}, + {"double", KQLFunctionValue::datatype_real}, + {"string", KQLFunctionValue::datatype_string}, + {"timespan", KQLFunctionValue::datatype_timespan}, + {"time", KQLFunctionValue::datatype_timespan}, + {"decimal", KQLFunctionValue::datatype_decimal}, + {"round", KQLFunctionValue::round} + }; + + +std::unique_ptr KQLFunctionFactory::get(String & kql_function) +{ + if (kql_functions.find(kql_function) == kql_functions.end()) + return nullptr; + + auto kql_function_id = kql_functions[kql_function]; + switch (kql_function_id) + { + case KQLFunctionValue::none: + return nullptr; + + case KQLFunctionValue::timespan: + return std::make_unique(); + + case KQLFunctionValue::ago: + return std::make_unique(); + + case KQLFunctionValue::datetime_add: + return std::make_unique(); + + case KQLFunctionValue::datetime_part: + return std::make_unique(); + + case KQLFunctionValue::datetime_diff: + return std::make_unique(); + + case KQLFunctionValue::dayofmonth: + return std::make_unique(); + + case KQLFunctionValue::dayofweek: + return std::make_unique(); + + case KQLFunctionValue::dayofyear: + return std::make_unique(); + + case KQLFunctionValue::endofday: + return std::make_unique(); + + case KQLFunctionValue::endofweek: + return std::make_unique(); + + case KQLFunctionValue::endofyear: + return std::make_unique(); + + case KQLFunctionValue::endofmonth: + return std::make_unique(); + + case KQLFunctionValue::monthofyear: + return std::make_unique(); + + case KQLFunctionValue::format_datetime: + return std::make_unique(); + + case KQLFunctionValue::format_timespan: + return std::make_unique(); + + case KQLFunctionValue::getmonth: + return std::make_unique(); + + case KQLFunctionValue::getyear: + return std::make_unique(); + + case KQLFunctionValue::hourofday: + return std::make_unique(); + + case KQLFunctionValue::make_timespan: + return std::make_unique(); + + case KQLFunctionValue::make_datetime: + return std::make_unique(); + + case KQLFunctionValue::now: + return std::make_unique(); + + case KQLFunctionValue::startofday: + return std::make_unique(); + + case KQLFunctionValue::startofmonth: + return std::make_unique(); + + case KQLFunctionValue::startofweek: + return std::make_unique(); + + case KQLFunctionValue::startofyear: + return std::make_unique(); + + case KQLFunctionValue::unixtime_microseconds_todatetime: + return std::make_unique(); + + case KQLFunctionValue::unixtime_milliseconds_todatetime: + return std::make_unique(); + + case KQLFunctionValue::unixtime_nanoseconds_todatetime: + return std::make_unique(); + + case KQLFunctionValue::unixtime_seconds_todatetime: + return std::make_unique(); + + case KQLFunctionValue::week_of_year: + return std::make_unique(); + + case KQLFunctionValue::base64_encode_tostring: + return std::make_unique(); + + case KQLFunctionValue::base64_encode_fromguid: + return std::make_unique(); + + case KQLFunctionValue::base64_decode_tostring: + return std::make_unique(); + + case KQLFunctionValue::base64_decode_toarray: + return std::make_unique(); + + case KQLFunctionValue::base64_decode_toguid: + return std::make_unique(); + + case KQLFunctionValue::countof: + return std::make_unique(); + + case KQLFunctionValue::extract: + return std::make_unique(); + + case KQLFunctionValue::extract_all: + return std::make_unique(); + + case KQLFunctionValue::extract_json: + return std::make_unique(); + + case KQLFunctionValue::has_any_index: + return std::make_unique(); + + case KQLFunctionValue::indexof: + return std::make_unique(); + + case KQLFunctionValue::isempty: + return std::make_unique(); + + case KQLFunctionValue::isnan: + return std::make_unique(); + + case KQLFunctionValue::isnotempty: + return std::make_unique(); + + case KQLFunctionValue::isnotnull: + return std::make_unique(); + + case KQLFunctionValue::isnull: + return std::make_unique(); + + case KQLFunctionValue::parse_command_line: + return std::make_unique(); + + case KQLFunctionValue::parse_csv: + return std::make_unique(); + + case KQLFunctionValue::parse_json: + return std::make_unique(); + + case KQLFunctionValue::parse_url: + return std::make_unique(); + + case KQLFunctionValue::parse_urlquery: + return std::make_unique(); + + case KQLFunctionValue::parse_version: + return std::make_unique(); + + case KQLFunctionValue::replace_regex: + return std::make_unique(); + + case KQLFunctionValue::reverse: + return std::make_unique(); + + case KQLFunctionValue::split: + return std::make_unique(); + + case KQLFunctionValue::strcat: + return std::make_unique(); + + case KQLFunctionValue::strcat_delim: + return std::make_unique(); + + case KQLFunctionValue::strcmp: + return std::make_unique(); + + case KQLFunctionValue::strlen: + return std::make_unique(); + + case KQLFunctionValue::strrep: + return std::make_unique(); + + case KQLFunctionValue::substring: + return std::make_unique(); + + case KQLFunctionValue::tolower: + return std::make_unique(); + + case KQLFunctionValue::toupper: + return std::make_unique(); + + case KQLFunctionValue::translate: + return std::make_unique(); + + case KQLFunctionValue::trim: + return std::make_unique(); + + case KQLFunctionValue::trim_end: + return std::make_unique(); + + case KQLFunctionValue::trim_start: + return std::make_unique(); + + case KQLFunctionValue::url_decode: + return std::make_unique(); + + case KQLFunctionValue::url_encode: + return std::make_unique(); + + case KQLFunctionValue::array_concat: + return std::make_unique(); + + case KQLFunctionValue::array_iif: + return std::make_unique(); + + case KQLFunctionValue::array_index_of: + return std::make_unique(); + + case KQLFunctionValue::array_length: + return std::make_unique(); + + case KQLFunctionValue::array_reverse: + return std::make_unique(); + + case KQLFunctionValue::array_rotate_left: + return std::make_unique(); + + case KQLFunctionValue::array_rotate_right: + return std::make_unique(); + + case KQLFunctionValue::array_shift_left: + return std::make_unique(); + + case KQLFunctionValue::array_shift_right: + return std::make_unique(); + + case KQLFunctionValue::array_slice: + return std::make_unique(); + + case KQLFunctionValue::array_sort_asc: + return std::make_unique(); + + case KQLFunctionValue::array_sort_desc: + return std::make_unique(); + + case KQLFunctionValue::array_split: + return std::make_unique(); + + case KQLFunctionValue::array_sum: + return std::make_unique(); + + case KQLFunctionValue::bag_keys: + return std::make_unique(); + + case KQLFunctionValue::bag_merge: + return std::make_unique(); + + case KQLFunctionValue::bag_remove_keys: + return std::make_unique(); + + case KQLFunctionValue::jaccard_index: + return std::make_unique(); + + case KQLFunctionValue::pack: + return std::make_unique(); + + case KQLFunctionValue::pack_all: + return std::make_unique(); + + case KQLFunctionValue::pack_array: + return std::make_unique(); + + case KQLFunctionValue::repeat: + return std::make_unique(); + + case KQLFunctionValue::set_difference: + return std::make_unique(); + + case KQLFunctionValue::set_has_element: + return std::make_unique(); + + case KQLFunctionValue::set_intersect: + return std::make_unique(); + + case KQLFunctionValue::set_union: + return std::make_unique(); + + case KQLFunctionValue::treepath: + return std::make_unique(); + + case KQLFunctionValue::zip: + return std::make_unique(); + + case KQLFunctionValue::tobool: + return std::make_unique(); + + case KQLFunctionValue::todatetime: + return std::make_unique(); + + case KQLFunctionValue::todouble: + return std::make_unique(); + + case KQLFunctionValue::toint: + return std::make_unique(); + + case KQLFunctionValue::tolong: + return std::make_unique(); + + case KQLFunctionValue::tostring: + return std::make_unique(); + + case KQLFunctionValue::totimespan: + return std::make_unique(); + + case KQLFunctionValue::todecimal: + return std::make_unique(); + + case KQLFunctionValue::arg_max: + return std::make_unique(); + + case KQLFunctionValue::arg_min: + return std::make_unique(); + + case KQLFunctionValue::avg: + return std::make_unique(); + + case KQLFunctionValue::avgif: + return std::make_unique(); + + case KQLFunctionValue::binary_all_and: + return std::make_unique(); + + case KQLFunctionValue::binary_all_or: + return std::make_unique(); + + case KQLFunctionValue::binary_all_xor: + return std::make_unique(); + + case KQLFunctionValue::buildschema: + return std::make_unique(); + + case KQLFunctionValue::count: + return std::make_unique(); + + case KQLFunctionValue::countif: + return std::make_unique(); + + case KQLFunctionValue::dcount: + return std::make_unique(); + + case KQLFunctionValue::dcountif: + return std::make_unique(); + + case KQLFunctionValue::make_bag: + return std::make_unique(); + + case KQLFunctionValue::make_bag_if: + return std::make_unique(); + + case KQLFunctionValue::make_list: + return std::make_unique(); + + case KQLFunctionValue::make_list_if: + return std::make_unique(); + + case KQLFunctionValue::make_list_with_nulls: + return std::make_unique(); + + case KQLFunctionValue::make_set: + return std::make_unique(); + + case KQLFunctionValue::make_set_if: + return std::make_unique(); + + case KQLFunctionValue::max: + return std::make_unique(); + + case KQLFunctionValue::maxif: + return std::make_unique(); + + case KQLFunctionValue::min: + return std::make_unique(); + + case KQLFunctionValue::minif: + return std::make_unique(); + + case KQLFunctionValue::percentile: + return std::make_unique(); + + case KQLFunctionValue::percentilew: + return std::make_unique(); + + case KQLFunctionValue::percentiles: + return std::make_unique(); + + case KQLFunctionValue::percentiles_array: + return std::make_unique(); + + case KQLFunctionValue::percentilesw: + return std::make_unique(); + + case KQLFunctionValue::percentilesw_array: + return std::make_unique(); + + case KQLFunctionValue::stdev: + return std::make_unique(); + + case KQLFunctionValue::stdevif: + return std::make_unique(); + + case KQLFunctionValue::sum: + return std::make_unique(); + + case KQLFunctionValue::sumif: + return std::make_unique(); + + case KQLFunctionValue::take_any: + return std::make_unique(); + + case KQLFunctionValue::take_anyif: + return std::make_unique(); + + case KQLFunctionValue::variance: + return std::make_unique(); + + case KQLFunctionValue::varianceif: + return std::make_unique(); + + case KQLFunctionValue::series_fir: + return std::make_unique(); + + case KQLFunctionValue::series_iir: + return std::make_unique(); + + case KQLFunctionValue::series_fit_line: + return std::make_unique(); + + case KQLFunctionValue::series_fit_line_dynamic: + return std::make_unique(); + + case KQLFunctionValue::series_fit_2lines: + return std::make_unique(); + + case KQLFunctionValue::series_fit_2lines_dynamic: + return std::make_unique(); + + case KQLFunctionValue::series_outliers: + return std::make_unique(); + + case KQLFunctionValue::series_periods_detect: + return std::make_unique(); + + case KQLFunctionValue::series_periods_validate: + return std::make_unique(); + + case KQLFunctionValue::series_stats_dynamic: + return std::make_unique(); + + case KQLFunctionValue::series_stats: + return std::make_unique(); + + case KQLFunctionValue::series_fill_backward: + return std::make_unique(); + + case KQLFunctionValue::series_fill_const: + return std::make_unique(); + + case KQLFunctionValue::series_fill_forward: + return std::make_unique(); + + case KQLFunctionValue::series_fill_linear: + return std::make_unique(); + + case KQLFunctionValue::ipv4_compare: + return std::make_unique(); + + case KQLFunctionValue::ipv4_is_in_range: + return std::make_unique(); + + case KQLFunctionValue::ipv4_is_match: + return std::make_unique(); + + case KQLFunctionValue::ipv4_is_private: + return std::make_unique(); + + case KQLFunctionValue::ipv4_netmask_suffix: + return std::make_unique(); + + case KQLFunctionValue::parse_ipv4: + return std::make_unique(); + + case KQLFunctionValue::parse_ipv4_mask: + return std::make_unique(); + + case KQLFunctionValue::ipv6_compare: + return std::make_unique(); + + case KQLFunctionValue::ipv6_is_match: + return std::make_unique(); + + case KQLFunctionValue::parse_ipv6: + return std::make_unique(); + + case KQLFunctionValue::parse_ipv6_mask: + return std::make_unique(); + + case KQLFunctionValue::format_ipv4: + return std::make_unique(); + + case KQLFunctionValue::format_ipv4_mask: + return std::make_unique(); + + case KQLFunctionValue::binary_and: + return std::make_unique(); + + case KQLFunctionValue::binary_not: + return std::make_unique(); + + case KQLFunctionValue::binary_or: + return std::make_unique(); + + case KQLFunctionValue::binary_shift_left: + return std::make_unique(); + + case KQLFunctionValue::binary_shift_right: + return std::make_unique(); + + case KQLFunctionValue::binary_xor: + return std::make_unique(); + + case KQLFunctionValue::bitset_count_ones: + return std::make_unique(); + + case KQLFunctionValue::bin: + return std::make_unique(); + + case KQLFunctionValue::bin_at: + return std::make_unique(); + + case KQLFunctionValue::datatype_bool: + return std::make_unique(); + + case KQLFunctionValue::datatype_datetime: + return std::make_unique(); + + case KQLFunctionValue::datatype_dynamic: + return std::make_unique(); + + case KQLFunctionValue::datatype_guid: + return std::make_unique(); + + case KQLFunctionValue::datatype_int: + return std::make_unique(); + + case KQLFunctionValue::datatype_long: + return std::make_unique(); + + case KQLFunctionValue::datatype_real: + return std::make_unique(); + + case KQLFunctionValue::datatype_string: + return std::make_unique(); + + case KQLFunctionValue::datatype_timespan: + return std::make_unique(); + + case KQLFunctionValue::datatype_decimal: + return std::make_unique(); + + case KQLFunctionValue::round: + return std::make_unique(); + } +} + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h b/src/Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h new file mode 100644 index 00000000000..653d89d42e2 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h @@ -0,0 +1,220 @@ +#pragma once + +#include +#include +#include +namespace DB +{ +enum class KQLFunctionValue : uint16_t +{ + none, + timespan, + ago, + datetime_add, + datetime_part, + datetime_diff, + dayofmonth, + dayofweek, + dayofyear, + endofday, + endofweek, + endofyear, + endofmonth, + monthofyear, + format_datetime, + format_timespan, + getmonth, + getyear, + hourofday, + make_timespan, + make_datetime, + now, + startofday, + startofmonth, + startofweek, + startofyear, + todatetime, + totimespan, + unixtime_microseconds_todatetime, + unixtime_milliseconds_todatetime, + unixtime_nanoseconds_todatetime, + unixtime_seconds_todatetime, + week_of_year, + + base64_encode_tostring, + base64_encode_fromguid, + base64_decode_tostring, + base64_decode_toarray, + base64_decode_toguid, + countof, + extract, + extract_all, + extract_json, + has_any_index, + indexof, + isempty, + isnan, + isnotempty, + isnotnull, + isnull, + parse_command_line, + parse_csv, + parse_json, + parse_url, + parse_urlquery, + parse_version, + replace_regex, + reverse, + split, + strcat, + strcat_delim, + strcmp, + strlen, + strrep, + substring, + tolower, + toupper, + translate, + trim, + trim_end, + trim_start, + url_decode, + url_encode, + + array_concat, + array_iif, + array_index_of, + array_length, + array_reverse, + array_rotate_left, + array_rotate_right, + array_shift_left, + array_shift_right, + array_slice, + array_sort_asc, + array_sort_desc, + array_split, + array_sum, + bag_keys, + bag_merge, + bag_remove_keys, + jaccard_index, + pack, + pack_all, + pack_array, + repeat, + set_difference, + set_has_element, + set_intersect, + set_union, + treepath, + zip, + + tobool, + todouble, + toint, + tolong, + tostring, + todecimal, + + arg_max, + arg_min, + avg, + avgif, + binary_all_and, + binary_all_or, + binary_all_xor, + buildschema, + count, + countif, + dcount, + dcountif, + make_bag, + make_bag_if, + make_list, + make_list_if, + make_list_with_nulls, + make_set, + make_set_if, + max, + maxif, + min, + minif, + percentile, + percentilew, + percentiles, + percentiles_array, + percentilesw, + percentilesw_array, + stdev, + stdevif, + sum, + sumif, + take_any, + take_anyif, + variance, + varianceif, + + series_fir, + series_iir, + series_fit_line, + series_fit_line_dynamic, + series_fit_2lines, + series_fit_2lines_dynamic, + series_outliers, + series_periods_detect, + series_periods_validate, + series_stats_dynamic, + series_stats, + series_fill_backward, + series_fill_const, + series_fill_forward, + series_fill_linear, + + ipv4_compare, + ipv4_is_in_range, + ipv4_is_match, + ipv4_is_private, + ipv4_netmask_suffix, + parse_ipv4, + parse_ipv4_mask, + ipv6_compare, + ipv6_is_match, + parse_ipv6, + parse_ipv6_mask, + format_ipv4, + format_ipv4_mask, + + binary_and, + binary_not, + binary_or, + binary_shift_left, + binary_shift_right, + binary_xor, + bitset_count_ones, + + bin, + bin_at, + + datatype_bool, + datatype_datetime, + datatype_dynamic, + datatype_guid, + datatype_int, + datatype_long, + datatype_real, + datatype_string, + datatype_timespan, + datatype_decimal, + round +}; +class KQLFunctionFactory +{ +public: + static std::unique_ptr get(String & kql_function); + +protected: + static std::unordered_map kql_functions; +}; + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.cpp b/src/Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.cpp new file mode 100644 index 00000000000..1e509f9ef0e --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.cpp @@ -0,0 +1,103 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +bool Bin::convertImpl(String & out, IParser::Pos & pos) +{ + double bin_size; + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + String origal_expr(pos->begin, pos->end); + String value = getConvertedArgument(fn_name, pos); + + ++pos; + String round_to = getConvertedArgument(fn_name, pos); + + //remove sapce between minus and number + round_to.erase(std::remove_if(round_to.begin(), round_to.end(), isspace), round_to.end()); + + auto t = std::format("toFloat64({})", value); + + bin_size = std::stod(round_to); + + if (origal_expr == "datetime" || origal_expr == "date") + { + out = std::format("toDateTime64(toInt64({0}/{1}) * {1}, 9, 'UTC')", t, bin_size); + } + else if (origal_expr == "timespan" || origal_expr == "time" || ParserKQLDateTypeTimespan().parseConstKQLTimespan(origal_expr)) + { + String bin_value = std::format("toInt64({0}/{1}) * {1}", t, bin_size); + out = std::format( + "concat(toString(toInt32((({}) as x) / 3600)),':', toString(toInt32(x % 3600 / 60)),':',toString(toInt32(x % 3600 % 60)))", + bin_value); + } + else + { + out = std::format("toInt64({0} / {1}) * {1}", t, bin_size); + } + return true; +} + +bool BinAt::convertImpl(String & out, IParser::Pos & pos) +{ + double bin_size; + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + String origal_expr(pos->begin, pos->end); + String expression_str = getConvertedArgument(fn_name, pos); + + ++pos; + String bin_size_str = getConvertedArgument(fn_name, pos); + + ++pos; + String fixed_point_str = getConvertedArgument(fn_name, pos); + + auto t1 = std::format("toFloat64({})", fixed_point_str); + auto t2 = std::format("toFloat64({})", expression_str); + int dir = t2 >= t1 ? 0 : -1; + bin_size = std::stod(bin_size_str); + + if (origal_expr == "datetime" || origal_expr == "date") + { + out = std::format("toDateTime64({} + toInt64(({} - {}) / {} + {}) * {}, 9, 'UTC')", t1, t2, t1, bin_size, dir, bin_size); + } + else if (origal_expr == "timespan" || origal_expr == "time" || ParserKQLDateTypeTimespan().parseConstKQLTimespan(origal_expr)) + { + String bin_value = std::format("{} + toInt64(({} - {}) / {} + {}) * {}", t1, t2, t1, bin_size, dir, bin_size); + out = std::format( + "concat(toString(toInt32((({}) as x) / 3600)),':', toString(toInt32(x % 3600 / 60)), ':', toString(toInt32(x % 3600 % 60)))", + bin_value); + } + else + { + out = std::format("{} + toInt64(({} - {}) / {} + {}) * {}", t1, t2, t1, bin_size, dir, bin_size); + } + return true; +} + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.h b/src/Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.h new file mode 100644 index 00000000000..ba10bab6553 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.h @@ -0,0 +1,21 @@ +#pragma once + +#include +#include +namespace DB +{ +class Bin : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "bin()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class BinAt : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "bin_at()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLIPFunctions.cpp b/src/Parsers/Kusto/KustoFunctions/KQLIPFunctions.cpp new file mode 100644 index 00000000000..6f853b16fbc --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLIPFunctions.cpp @@ -0,0 +1,275 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +bool Ipv4Compare::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto lhs = getArgument(function_name, pos, ArgumentState::Raw); + const auto rhs = getArgument(function_name, pos, ArgumentState::Raw); + const auto mask = getOptionalArgument(function_name, pos, ArgumentState::Parsed); + out = std::format( + "if(isNull({0} as lhs_ip_{5}) or isNull({1} as lhs_mask_{5}) " + "or isNull({2} as rhs_ip_{5}) or isNull({3} as rhs_mask_{5}), null, " + "sign(IPv4StringToNumOrNull(toString((tupleElement(IPv4CIDRToRange(assumeNotNull(lhs_ip_{5}), " + "toUInt8(min2({4}, min2(assumeNotNull(lhs_mask_{5}), assumeNotNull(rhs_mask_{5})))) as mask_{5}), 1))))" + " - IPv4StringToNumOrNull(toString((tupleElement(IPv4CIDRToRange(assumeNotNull(rhs_ip_{5}), mask_{5}), 1))))))", + kqlCallToExpression("parse_ipv4", {lhs}, pos.max_depth), + kqlCallToExpression("ipv4_netmask_suffix", {lhs}, pos.max_depth), + kqlCallToExpression("parse_ipv4", {rhs}, pos.max_depth), + kqlCallToExpression("ipv4_netmask_suffix", {rhs}, pos.max_depth), + mask ? *mask : "32", + generateUniqueIdentifier()); + return true; +} + +bool Ipv4IsInRange::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto ip_address = getArgument(function_name, pos, ArgumentState::Parsed); + const auto ip_range = getArgument(function_name, pos, ArgumentState::Raw); + out = std::format( + "if(isNull(IPv4StringToNumOrNull({0}) as ip_{3}) " + "or isNull({1} as range_start_ip_{3}) or isNull({2} as range_mask_{3}), null, " + "bitXor(range_start_ip_{3}, bitAnd(ip_{3}, bitNot(toUInt32(intExp2(toInt32(32 - range_mask_{3})) - 1)))) = 0) ", + ip_address, + kqlCallToExpression("parse_ipv4", {ip_range}, pos.max_depth), + kqlCallToExpression("ipv4_netmask_suffix", {ip_range}, pos.max_depth), + generateUniqueIdentifier()); + return true; +} + +bool Ipv4IsMatch::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto lhs = getArgument(function_name, pos, ArgumentState::Raw); + const auto rhs = getArgument(function_name, pos, ArgumentState::Raw); + const auto mask = getOptionalArgument(function_name, pos, ArgumentState::Raw); + out = std::format("equals({}, 0)", kqlCallToExpression("ipv4_compare", {lhs, rhs, mask ? *mask : "32"}, pos.max_depth)); + return true; +} + +bool Ipv4IsPrivate::convertImpl(String & out, IParser::Pos & pos) +{ + static const std::array s_private_subnets{"10.0.0.0/8", "172.16.0.0/12", "192.168.0.0/16"}; + + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto ip_address = getArgument(function_name, pos); + const auto unique_identifier = generateUniqueIdentifier(); + + out += std::format( + "multiIf(length(splitByChar('/', {0}) as tokens_{1}) > 2 or isNull(toIPv4OrNull(tokens_{1}[1]) as nullable_ip_{1}) " + "or length(tokens_{1}) = 2 and isNull(toUInt8OrNull(tokens_{1}[-1]) as mask_{1}), null, " + "ignore(assumeNotNull(nullable_ip_{1}) as ip_{1}, " + "IPv4CIDRToRange(ip_{1}, assumeNotNull(mask_{1})) as range_{1}, IPv4NumToString(tupleElement(range_{1}, 1)) as begin_{1}, " + "IPv4NumToString(tupleElement(range_{1}, 2)) as end_{1}), null, ", + ip_address, + unique_identifier); + for (int i = 0; i < std::ssize(s_private_subnets); ++i) + { + if (i > 0) + out += " or "; + + const auto & subnet = s_private_subnets[i]; + out += std::format( + "length(tokens_{1}) = 1 and isIPAddressInRange(IPv4NumToString(ip_{1}), '{0}') or " + "length(tokens_{1}) = 2 and isIPAddressInRange(begin_{1}, '{0}') and isIPAddressInRange(end_{1}, '{0}')", + subnet, + unique_identifier); + } + + out.push_back(')'); + return true; +} + +bool Ipv4NetmaskSuffix::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto ip_range = getArgument(function_name, pos); + out = std::format( + "multiIf(length(splitByChar('/', {0}) as tokens_{1}) > 2 or not isIPv4String(tokens_{1}[1]), null, " + "length(tokens_{1}) = 1, 32, isNull(toUInt8OrNull(tokens_{1}[-1]) as mask_{1}), null, toUInt8(min2(mask_{1}, 32)))", + ip_range, + generateUniqueIdentifier()); + return true; +} + +bool ParseIpv4::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto ip_address = getArgument(function_name, pos); + out = std::format( + "multiIf(length(splitByChar('/', {0}) as tokens_{1}) = 1, IPv4StringToNumOrNull(tokens_{1}[1]) as ip_{1}, " + "length(tokens_{1}) = 2 and isNotNull(ip_{1}) and isNotNull(toUInt8OrNull(tokens_{1}[-1]) as mask_{1}), " + "IPv4StringToNumOrNull(toString(tupleElement(IPv4CIDRToRange(toIPv4(assumeNotNull(ip_{1})), assumeNotNull(mask_{1})), 1))), null)", + ip_address, + generateUniqueIdentifier()); + return true; +} + +bool ParseIpv4Mask::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto ip_address = getArgument(function_name, pos); + const auto mask = getArgument(function_name, pos); + out = std::format( + "if(isNull(toIPv4OrNull({0}) as ip_{2}) or isNull(toUInt8OrNull(toString({1})) as mask_{2}), null, " + "toUInt32(tupleElement(IPv4CIDRToRange(assumeNotNull(ip_{2}), arrayMax([0, arrayMin([32, assumeNotNull(mask_{2})])])), 1)))", + ip_address, + mask, + generateUniqueIdentifier()); + return true; +} + +bool Ipv6Compare::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto lhs = getArgument(function_name, pos); + const auto rhs = getArgument(function_name, pos); + const auto mask = getOptionalArgument(function_name, pos); + const auto calculated_mask = mask ? *mask : "128"; + out = std::format( + "if(length(splitByChar('/', {1}) as lhs_tokens_{0}) > 2 or length(splitByChar('/', {2}) as rhs_tokens_{0}) > 2 " + "or isNull(IPv6StringToNumOrNull(lhs_tokens_{0}[1]) as lhs_ipv6_{0}) or length(lhs_tokens_{0}) = 2 " + "and isNull((if(isIPv4String(lhs_tokens_{0}[1]), 96, 0) + toUInt8OrNull(lhs_tokens_{0}[-1])) as lhs_suffix_{0}) " + "or isNull(IPv6StringToNumOrNull(rhs_tokens_{0}[1]) as rhs_ipv6_{0}) or length(rhs_tokens_{0}) = 2 " + "and isNull((if(isIPv4String(rhs_tokens_{0}[1]), 96, 0) + toUInt8OrNull(rhs_tokens_{0}[-1])) as rhs_suffix_{0}) " + "or isNull(toUInt8(min2({3}, min2(ifNull(lhs_suffix_{0}, 128), ifNull(rhs_suffix_{0}, 128)))) as suffix_{0}) " + "or isNull(bitShiftLeft(bitShiftRight(bitNot(reinterpretAsFixedString(0::UInt128)), (128 - suffix_{0}) as zeroes_{0}), " + "zeroes_{0}) as mask_{0}) or isNull(bitAnd(lhs_ipv6_{0}, mask_{0}) as lhs_base_{0}) " + "or isNull(bitAnd(rhs_ipv6_{0}, mask_{0}) as rhs_base_{0}), null, " + "multiIf(lhs_base_{0} < rhs_base_{0}, -1, lhs_base_{0} > rhs_base_{0}, 1, 0))", + generateUniqueIdentifier(), + lhs, + rhs, + calculated_mask); + return true; +} + +bool Ipv6IsMatch::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto lhs = getArgument(function_name, pos, ArgumentState::Raw); + const auto rhs = getArgument(function_name, pos, ArgumentState::Raw); + const auto mask = getOptionalArgument(function_name, pos, ArgumentState::Raw); + out = std::format("equals({}, 0)", kqlCallToExpression("ipv6_compare", {lhs, rhs, mask ? *mask : "128"}, pos.max_depth)); + return true; +} + +bool ParseIpv6::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto ip_address = getArgument(function_name, pos); + out = std::format( + "if(length(splitByChar('/', assumeNotNull({0})) as tokens_{1}) > 2 or isNull(IPv6StringToNumOrNull(tokens_{1}[1]) as ip_{1}) " + "or length(tokens_{1}) = 2 and isNull(toUInt8OrNull(tokens_{1}[-1]) as mask_{1}), null, " + "arrayStringConcat(flatten(extractAllGroups(lower(hex(tupleElement(IPv6CIDRToRange(assumeNotNull(ip_{1}), toUInt8(ifNull(mask_{1} " + "+ if(isIPv4String(tokens_{1}[1]), 96, 0), 128))), 1))), '([\\da-f]{{4}})')), ':'))", + ip_address, + generateUniqueIdentifier()); + return true; +} + +bool ParseIpv6Mask::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto ip_address = getArgument(function_name, pos, ArgumentState::Raw); + const auto mask = getArgument(function_name, pos, ArgumentState::Raw); + const auto unique_identifier = generateUniqueIdentifier(); + out = std::format( + "if(empty({0} as ipv4_{3}), {1}, {2})", + kqlCallToExpression("format_ipv4", {"trim_start('::', " + ip_address + ")", mask + " - 96"}, pos.max_depth), + kqlCallToExpression("parse_ipv6", {"strcat(tostring(parse_ipv6(" + ip_address + ")), '/', tostring(" + mask + "))"}, pos.max_depth), + kqlCallToExpression("parse_ipv6", {"ipv4_" + unique_identifier}, pos.max_depth), + unique_identifier); + return true; +} + +bool FormatIpv4::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto ip_address = getArgument(function_name, pos, ArgumentState::Raw); + const auto mask = getOptionalArgument(function_name, pos, ArgumentState::Parsed); + out = std::format( + "ifNull(if(isNotNull(toUInt32OrNull(toString({0})) as param_as_uint32_{3}) and toTypeName({0}) = 'String' or ({1}) < 0 " + "or isNull(ifNull(param_as_uint32_{3}, {2}) as ip_as_number_{3}), null, " + "IPv4NumToString(bitAnd(ip_as_number_{3}, bitNot(toUInt32(intExp2(toInt32(32 - ({1}))) - 1))))), '')", + ParserKQLBase::getExprFromToken(ip_address, pos.max_depth), + mask ? *mask : "32", + kqlCallToExpression("parse_ipv4", {"tostring(" + ip_address + ")"}, pos.max_depth), + generateUniqueIdentifier()); + return true; +} + +bool FormatIpv4Mask::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto ip_address = getArgument(function_name, pos, ArgumentState::Raw); + const auto mask = getOptionalArgument(function_name, pos, ArgumentState::Raw); + const auto calculated_mask = mask ? *mask : "32"; + out = std::format( + "if(empty({1} as formatted_ip_{2}) or position(toTypeName({0}), 'Int') = 0 or not {0} between 0 and 32, '', " + "concat(formatted_ip_{2}, '/', toString(toInt64(min2({0}, ifNull({3} as suffix_{2}, 32))))))", + ParserKQLBase::getExprFromToken(calculated_mask, pos.max_depth), + kqlCallToExpression("format_ipv4", {ip_address, calculated_mask}, pos.max_depth), + generateUniqueIdentifier(), + kqlCallToExpression("ipv4_netmask_suffix", {"tostring(" + ip_address + ")"}, pos.max_depth)); + return true; +} +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLIPFunctions.h b/src/Parsers/Kusto/KustoFunctions/KQLIPFunctions.h new file mode 100644 index 00000000000..a654e90ce18 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLIPFunctions.h @@ -0,0 +1,98 @@ +#pragma once + +#include +#include +namespace DB +{ +class Ipv4Compare : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "ipv4_compare()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Ipv4IsInRange : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "ipv4_is_in_range()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Ipv4IsMatch : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "ipv4_is_match()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Ipv4IsPrivate : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "ipv4_is_private()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Ipv4NetmaskSuffix : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "ipv4_netmask_suffix()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ParseIpv4 : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "parse_ipv4()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ParseIpv4Mask : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "parse_ipv4_mask()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Ipv6Compare : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "ipv6_compare()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Ipv6IsMatch : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "ipv6_is_match()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ParseIpv6 : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "parse_ipv6()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ParseIpv6Mask : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "parse_ipv6_mask()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class FormatIpv4 : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "format_ipv4()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class FormatIpv4Mask : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "format_ipv4_mask()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLMathematicalFunctions.cpp b/src/Parsers/Kusto/KustoFunctions/KQLMathematicalFunctions.cpp new file mode 100644 index 00000000000..4225b093ca2 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLMathematicalFunctions.cpp @@ -0,0 +1,23 @@ +#include "KQLMathematicalFunctions.h" + +#include + +namespace DB +{ +bool IsNan::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto argument = getArgument(function_name, pos); + out = std::format("if(toTypeName({0}) = 'Float64', isNaN({0}), throwIf(true, 'Expected argument of data type real'))", argument); + + return true; +} + +bool Round::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "round"); +} +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLMathematicalFunctions.h b/src/Parsers/Kusto/KustoFunctions/KQLMathematicalFunctions.h new file mode 100644 index 00000000000..561f4e67410 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLMathematicalFunctions.h @@ -0,0 +1,21 @@ +#pragma once + +#include "IParserKQLFunction.h" + +namespace DB +{ +class IsNan : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "isnan()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Round : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "round()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.cpp b/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.cpp new file mode 100644 index 00000000000..0f9ca67d6dc --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.cpp @@ -0,0 +1,773 @@ +#include +#include +#include +#include +#include + +#include +#include +#include + +#include + + +namespace DB::ErrorCodes +{ +extern const int SYNTAX_ERROR; +extern const int BAD_ARGUMENTS; +extern const int UNKNOWN_TYPE; + +} + +namespace DB +{ + +bool Base64EncodeToString::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "base64Encode"); +} + +bool Base64EncodeFromGuid::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto argument = getArgument(function_name, pos); + out = std::format( + "if(toTypeName({0}) not in ['UUID', 'Nullable(UUID)'], toString(throwIf(true, 'Expected guid as argument')), " + "base64Encode(UUIDStringToNum(toString({0}), 2)))", + argument, + generateUniqueIdentifier()); + return true; +} + +bool Base64DecodeToString::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "base64Decode"); +} + +bool Base64DecodeToArray::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String str = getConvertedArgument(fn_name, pos); + + out = std::format("arrayMap(x -> (reinterpretAsUInt8(x)), splitByRegexp ('',base64Decode({})))", str); + + return true; +} + +bool Base64DecodeToGuid::convertImpl(String & out, IParser::Pos & pos) +{ + const auto function_name = getKQLFunctionName(pos); + if (function_name.empty()) + return false; + + const auto argument = getArgument(function_name, pos); + out = std::format("toUUIDOrNull(UUIDNumToString(toFixedString(base64Decode({}), 16), 2))", argument); + + return true; +} + +bool CountOf::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String source = getConvertedArgument(fn_name, pos); + + ++pos; + const String search = getConvertedArgument(fn_name, pos); + + String kind = "'normal'"; + if (pos->type == TokenType::Comma) + { + ++pos; + kind = getConvertedArgument(fn_name, pos); + } + assert(kind == "'normal'" || kind == "'regex'"); + + if (kind == "'normal'") + out = "countSubstrings(" + source + ", " + search + ")"; + else + out = "countMatches(" + source + ", " + search + ")"; + return true; +} + +bool Extract::convertImpl(String & out, IParser::Pos & pos) +{ + ParserKeyword s_kql("typeof"); + ParserToken open_bracket(TokenType::OpeningRoundBracket); + ParserToken close_bracket(TokenType::ClosingRoundBracket); + Expected expected; + + std::unordered_map type_cast + = {{"bool", "Boolean"}, + {"boolean", "Boolean"}, + {"datetime", "DateTime"}, + {"date", "DateTime"}, + {"guid", "UUID"}, + {"int", "Int32"}, + {"long", "Int64"}, + {"real", "Float64"}, + {"double", "Float64"}, + {"string", "String"}, + {"decimal", "Decimal"}}; + + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + String regex = getConvertedArgument(fn_name, pos); + + ++pos; + size_t capture_group = stoi(getConvertedArgument(fn_name, pos)); + + ++pos; + String source = getConvertedArgument(fn_name, pos); + + String type_literal; + + if (pos->type == TokenType::Comma) + { + ++pos; + + if (s_kql.ignore(pos, expected)) + { + if (!open_bracket.ignore(pos, expected)) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near typeof"); + + type_literal = String(pos->begin, pos->end); + + if (type_cast.find(type_literal) == type_cast.end()) + throw Exception(ErrorCodes::UNKNOWN_TYPE, "{} is not a supported kusto data type for extract", type_literal); + + type_literal = type_cast[type_literal]; + ++pos; + + if (!close_bracket.ignore(pos, expected)) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near typeof"); + } + } + + if (capture_group == 0) + { + String tmp_regex; + for (auto c : regex) + { + if (c != '(' && c != ')') + tmp_regex += c; + } + regex = std::move(tmp_regex); + } + else + { + size_t group_idx = 0; + size_t str_idx = -1; + for (size_t i = 0; i < regex.length(); ++i) + { + if (regex[i] == '(') + { + ++group_idx; + if (group_idx == capture_group) + { + str_idx = i + 1; + break; + } + } + } + String tmp_regex; + if (str_idx > 0) + { + for (size_t i = str_idx; i < regex.length(); ++i) + { + if (regex[i] == ')') + break; + tmp_regex += regex[i]; + } + } + regex = "'" + tmp_regex + "'"; + } + + out = "extract(" + source + ", " + regex + ")"; + + if (type_literal == "Decimal") + { + out = std::format("countSubstrings({0}, '.') > 1 ? NULL: {0}, length(substr({0}, position({0},'.') + 1)))", out); + out = std::format("toDecimal128OrNull({0})", out); + } + else + { + if (type_literal == "Boolean") + out = std::format("toInt64OrNull({})", out); + + if (!type_literal.empty()) + out = "accurateCastOrNull(" + out + ", '" + type_literal + "')"; + } + return true; +} + +bool ExtractAll::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String regex = getConvertedArgument(fn_name, pos); + + ++pos; + const String second_arg = getConvertedArgument(fn_name, pos); + + String third_arg; + if (pos->type == TokenType::Comma) + { + ++pos; + third_arg = getConvertedArgument(fn_name, pos); + } + + if (!third_arg.empty()) // currently the captureGroups not supported + return false; + + out = "extractAllGroups(" + second_arg + ", " + regex + ")"; + return true; +} + +bool ExtractJson::convertImpl(String & out, IParser::Pos & pos) +{ + String datatype = "String"; + ParserKeyword s_kql("typeof"); + ParserToken open_bracket(TokenType::OpeningRoundBracket); + ParserToken close_bracket(TokenType::ClosingRoundBracket); + Expected expected; + + std::unordered_map type_cast + = {{"bool", "Boolean"}, + {"boolean", "Boolean"}, + {"datetime", "DateTime"}, + {"date", "DateTime"}, + {"dynamic", "Array"}, + {"guid", "UUID"}, + {"int", "Int32"}, + {"long", "Int64"}, + {"real", "Float64"}, + {"double", "Float64"}, + {"string", "String"}, + {"decimal", "Decimal"}}; + + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String json_datapath = getConvertedArgument(fn_name, pos); + ++pos; + const String json_datasource = getConvertedArgument(fn_name, pos); + if (pos->type == TokenType::Comma) + { + ++pos; + if (s_kql.ignore(pos, expected)) + { + if (!open_bracket.ignore(pos, expected)) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near typeof"); + + datatype = String(pos->begin, pos->end); + + if (type_cast.find(datatype) == type_cast.end()) + throw Exception(ErrorCodes::UNKNOWN_TYPE, "{} is not a supported kusto data type for {}", datatype, fn_name); + datatype = type_cast[datatype]; + ++pos; + + if (!close_bracket.ignore(pos, expected)) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near typeof"); + } + } + const auto json_val = std::format("JSON_VALUE({0},{1})", json_datasource, json_datapath); + + if (datatype == "Decimal") + { + out = std::format("countSubstrings({0}, '.') > 1 ? NULL: length(substr({0}, position({0},'.') + 1)))", json_val); + out = std::format("toDecimal128OrNull({0}::String, {1})", json_val, out); + } + else + { + if (datatype == "Boolean") + out = std::format("toInt64OrNull({})", json_val); + + if (!datatype.empty()) + out = std::format("accurateCastOrNull({},'{}')", json_val, datatype); + } + return true; +} + +bool HasAnyIndex::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String source = getConvertedArgument(fn_name, pos); + + ++pos; + const String lookup = getConvertedArgument(fn_name, pos); + String src_array = std::format("splitByChar(' ',{})", source); + out = std::format( + "if(empty({1}), -1, indexOf(arrayMap(x->(x in {0}), if(empty({1}),[''], arrayMap(x->(toString(x)),{1}))),1) - 1)", + src_array, + lookup); + return true; +} + +bool IndexOf::convertImpl(String & out, IParser::Pos & pos) +{ + int start_index = 0, length = -1, occurrence = 1; + + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String source = getConvertedArgument(fn_name, pos); + + ++pos; + const String lookup = getConvertedArgument(fn_name, pos); + + if (pos->type == TokenType::Comma) + { + ++pos; + start_index = stoi(getConvertedArgument(fn_name, pos)); + + if (pos->type == TokenType::Comma) + { + ++pos; + length = stoi(getConvertedArgument(fn_name, pos)); + + if (pos->type == TokenType::Comma) + { + ++pos; + occurrence = stoi(getConvertedArgument(fn_name, pos)); + } + } + } + + if (pos->type == TokenType::ClosingRoundBracket) + { + if (occurrence < 0 || length < -1) + out = ""; + else if (length == -1) + out = "position(" + source + ", " + lookup + ", " + std::to_string(start_index + 1) + ") - 1"; + else + { + } + + return true; + } + + return false; +} + +bool IsEmpty::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "empty"); +} + +bool IsNotEmpty::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "notEmpty"); +} + +bool IsNotNull::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "isNotNull"); +} + +bool ParseCommandLine::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String json_string = getConvertedArgument(fn_name, pos); + + ++pos; + const String type = getConvertedArgument(fn_name, pos); + + if (type != "'windows'") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Supported type argument is windows for {}", fn_name); + + out = std::format( + "if(empty({0}) OR hasAll(splitByChar(' ', {0}) , ['']) , arrayMap(x->null, splitByChar(' ', '')), splitByChar(' ', {0}))", + json_string); + return true; +} + +bool IsNull::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "isNull"); +} + +bool ParseCSV::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + String csv_string = getConvertedArgument(fn_name, pos); + + out = std::format( + "if(position({0} ,'\n')::UInt8, (splitByChar(',', substring({0}, 1, position({0},'\n') -1))), (splitByChar(',', substring({0}, 1, " + "length({0})))))", + csv_string); + return true; +} + +bool ParseJson::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + if (String(pos->begin, pos->end) == "dynamic") + { + --pos; + auto arg = getArgument(fn_name, pos); + auto result = kqlCallToExpression("dynamic", {arg}, pos.max_depth); + out = std::format("{}", result); + } + else + { + auto arg = getConvertedArgument(fn_name, pos); + out = std::format("if (isValidJSON({0}) , JSON_QUERY({0}, '$') , toJSONString({0}))", arg); + } + return true; +} + +bool ParseURL::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String url = getConvertedArgument(fn_name, pos); + + const String scheme = std::format(R"(concat('"Scheme":"', protocol({0}),'"'))", url); + const String host = std::format(R"(concat('"Host":"', domain({0}),'"'))", url); + const String port = std::format(R"(concat('"Port":"', toString(port({0})),'"'))", url); + const String path = std::format(R"(concat('"Path":"', path({0}),'"'))", url); + const String username_pwd = std::format("netloc({0})", url); + const String query_string = std::format("queryString({0})", url); + const String fragment = std::format(R"(concat('"Fragment":"',fragment({0}),'"'))", url); + const String username = std::format( + R"(concat('"Username":"', arrayElement(splitByChar(':',arrayElement(splitByChar('@',{0}) ,1)),1),'"'))", username_pwd); + const String password = std::format( + R"(concat('"Password":"', arrayElement(splitByChar(':',arrayElement(splitByChar('@',{0}) ,1)),2),'"'))", username_pwd); + const String query_parameters = std::format( + R"(concat('"Query Parameters":', concat('{{"', replace(replace({}, '=', '":"'),'&','","') ,'"}}')))", query_string); + + out = std::format( + "concat('{{',{},',',{},',',{},',',{},',',{},',',{},',',{},',',{},'}}')", + scheme, + host, + port, + path, + username, + password, + query_parameters, + fragment); + return true; +} + +bool ParseURLQuery::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + ++pos; + const String query = getConvertedArgument(fn_name, pos); + + const String query_string = std::format("if (position({},'?') > 0, queryString({}), {})", query, query, query); + const String query_parameters = std::format( + R"(concat('"Query Parameters":', concat('{{"', replace(replace({}, '=', '":"'),'&','","') ,'"}}')))", query_string); + out = std::format("concat('{{',{},'}}')", query_parameters); + return true; +} + +bool ParseVersion::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + String arg; + ++pos; + arg = getConvertedArgument(fn_name, pos); + out = std::format( + "length(splitByChar('.', {0})) > 4 OR length(splitByChar('.', {0})) < 1 OR match({0}, '.*[a-zA-Z]+.*') = 1 ? " + "toDecimal128OrNull('NULL' , 0) : toDecimal128OrNull(substring(arrayStringConcat(arrayMap(x -> leftPad(x, 8, '0'), arrayMap(x -> " + "if(empty(x), '0', x), arrayResize(splitByChar('.', {0}), 4)))), 8),0)", + arg); + return true; +} + +bool ReplaceRegex::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "replaceRegexpAll"); +} + +bool Reverse::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + + auto arg = getConvertedArgument(fn_name, pos); + + out = std::format("reverse(accurateCastOrNull({} , 'String'))", arg); + + return true; +} + +bool Split::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String source = getConvertedArgument(fn_name, pos); + + ++pos; + const String delimiter = getConvertedArgument(fn_name, pos); + auto split_res = std::format("empty({0}) ? splitByString(' ' , {1}) : splitByString({0} , {1})", delimiter, source); + int requested_index = -1; + + if (pos->type == TokenType::Comma) + { + ++pos; + auto arg = getConvertedArgument(fn_name, pos); + // remove space between minus and value + arg.erase(remove_if(arg.begin(), arg.end(), isspace), arg.end()); + requested_index = std::stoi(arg); + requested_index += 1; + out = std::format( + "multiIf(length({0}) >= {1} AND {1} > 0, arrayPushBack([],arrayElement({0}, {1})), {1}=0, {0}, arrayPushBack([] " + ",arrayElement(NULL,1)))", + split_res, + requested_index); + } + else + out = split_res; + return true; +} + +bool StrCat::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "concat"); +} + +bool StrCatDelim::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String delimiter = getConvertedArgument(fn_name, pos); + + int arg_count = 0; + String args; + + while (!pos->isEnd() && pos->type != TokenType::Semicolon && pos->type != TokenType::ClosingRoundBracket) + { + ++pos; + String arg = getConvertedArgument(fn_name, pos); + if (args.empty()) + args = "concat(" + arg; + else + args = args + ", " + delimiter + ", " + arg; + ++arg_count; + } + args += ")"; + + if (arg_count < 2 || arg_count > 64) + throw Exception(ErrorCodes::SYNTAX_ERROR, "argument count out of bound in function: {}", fn_name); + + out = std::move(args); + return true; +} + +bool StrCmp::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + ++pos; + const String string1 = getConvertedArgument(fn_name, pos); + ++pos; + const String string2 = getConvertedArgument(fn_name, pos); + + out = std::format("multiIf({0} == {1}, 0, {0} < {1}, -1, 1)", string1, string2); + return true; +} + +bool StrLen::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "lengthUTF8"); +} + +bool StrRep::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + + ++pos; + const String value = getConvertedArgument(fn_name, pos); + + ++pos; + const String multiplier = getConvertedArgument(fn_name, pos); + + if (pos->type == TokenType::Comma) + { + ++pos; + const String delimiter = getConvertedArgument(fn_name, pos); + const String repeated_str = "repeat(concat(" + value + "," + delimiter + ")," + multiplier + ")"; + out = "substr(" + repeated_str + ", 1, length(" + repeated_str + ") - length(" + delimiter + "))"; + } + else + out = "repeat(" + value + ", " + multiplier + ")"; + + return true; +} + +bool SubString::convertImpl(String & out, IParser::Pos & pos) +{ + String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + + ++pos; + String source = getConvertedArgument(fn_name, pos); + + ++pos; + String starting_index = getConvertedArgument(fn_name, pos); + + if (pos->type == TokenType::Comma) + { + ++pos; + auto length = getConvertedArgument(fn_name, pos); + + if (starting_index.empty()) + throw Exception(ErrorCodes::SYNTAX_ERROR, "number of arguments do not match in function: {}", fn_name); + else + out = "if(toInt64(length(" + source + ")) <= 0, '', substr(" + source + ", " + "((" + starting_index + "% toInt64(length(" + + source + ")) + toInt64(length(" + source + "))) % toInt64(length(" + source + "))) + 1, " + length + ") )"; + } + else + out = "if(toInt64(length(" + source + ")) <= 0, '', substr(" + source + "," + "((" + starting_index + "% toInt64(length(" + source + + ")) + toInt64(length(" + source + "))) % toInt64(length(" + source + "))) + 1))"; + + return true; +} + +bool ToLower::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "lower"); +} + +bool ToUpper::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "upper"); +} + +bool Translate::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + + if (fn_name.empty()) + return false; + + ++pos; + String from = getConvertedArgument(fn_name, pos); + ++pos; + String to = getConvertedArgument(fn_name, pos); + ++pos; + String source = getConvertedArgument(fn_name, pos); + + String len_diff = std::format("length({}) - length({})", from, to); + String to_str = std::format( + "multiIf(length({1}) = 0, {0}, {2} > 0, concat({1},repeat(substr({1},length({1}),1),toUInt16({2}))),{2} < 0, " + "substr({1},1,length({0})),{1})", + from, + to, + len_diff); + out = std::format("if (length({3}) = 0,'',translate({0},{1},{2}))", source, from, to_str, to); + return true; +} + +bool Trim::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + const auto regex = getArgument(fn_name, pos, ArgumentState::Raw); + const auto source = getArgument(fn_name, pos, ArgumentState::Raw); + out = kqlCallToExpression("trim_start", {regex, std::format("trim_end({0}, {1})", regex, source)}, pos.max_depth); + + return true; +} + +bool TrimEnd::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + const auto regex = getArgument(fn_name, pos); + const auto source = getArgument(fn_name, pos); + out = std::format("replaceRegexpOne({0}, concat({1}, '$'), '')", source, regex); + + return true; +} + +bool TrimStart::convertImpl(String & out, IParser::Pos & pos) +{ + const String fn_name = getKQLFunctionName(pos); + if (fn_name.empty()) + return false; + + const auto regex = getArgument(fn_name, pos); + const auto source = getArgument(fn_name, pos); + out = std::format("replaceRegexpOne({0}, concat('^', {1}), '')", source, regex); + + return true; +} + +bool URLDecode::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "decodeURLComponent"); +} + +bool URLEncode::convertImpl(String & out, IParser::Pos & pos) +{ + return directMapping(out, pos, "encodeURLComponent"); +} + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h b/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h new file mode 100644 index 00000000000..492a59263ec --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h @@ -0,0 +1,275 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ +class Base64EncodeToString : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "base64_encode_tostring()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Base64EncodeFromGuid : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "base64_encode_fromguid()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Base64DecodeToString : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "base64_decode_tostring()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Base64DecodeToArray : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "base64_decode_toarray()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Base64DecodeToGuid : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "base64_decode_toguid()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class CountOf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "countof()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Extract : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "extract()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ExtractAll : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "extract_all()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ExtractJson : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "extract_json(), extractjson()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class HasAnyIndex : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "has_any_index()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class IndexOf : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "indexof()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class IsEmpty : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "isempty()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class IsNotEmpty : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "isnotempty()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class IsNotNull : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "isnotnull()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class IsNull : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "isnull()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ParseCommandLine : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "parse_command_line()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ParseCSV : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "parse_csv()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ParseJson : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "parse_json()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ParseURL : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "parse_url()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ParseURLQuery : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "parse_urlquery()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ParseVersion : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "parse_version()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ReplaceRegex : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "replace_regex()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Reverse : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "reverse()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Split : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "split()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class StrCat : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "strcat()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class StrCatDelim : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "strcat_delim()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class StrCmp : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "strcmp()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class StrLen : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "strlen()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class StrRep : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "strrep()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SubString : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "substring()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ToLower : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "tolower()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class ToUpper : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "toupper()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Translate : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "translate()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class Trim : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "trim()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class TrimEnd : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "trim_end()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class TrimStart : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "trim_start()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class URLDecode : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "url_decode()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class URLEncode : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "url_encode()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.cpp b/src/Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.cpp new file mode 100644 index 00000000000..9a716536210 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.cpp @@ -0,0 +1,126 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +bool SeriesFir::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool SeriesIir::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool SeriesFitLine::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool SeriesFitLineDynamic::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool SeriesFit2lines::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool SeriesFit2linesDynamic::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool SeriesOutliers::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool SeriesPeriodsDetect::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool SeriesPeriodsValidate::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool SeriesStatsDynamic::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool SeriesStats::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool SeriesFillBackward::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool SeriesFillConst::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool SeriesFillForward::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +bool SeriesFillLinear::convertImpl(String & out, IParser::Pos & pos) +{ + String res = String(pos->begin, pos->end); + out = res; + return false; +} + +} diff --git a/src/Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.h b/src/Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.h new file mode 100644 index 00000000000..486e2bc6391 --- /dev/null +++ b/src/Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.h @@ -0,0 +1,112 @@ +#pragma once + +#include +#include +namespace DB +{ +class SeriesFir : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_fir()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SeriesIir : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_iir()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SeriesFitLine : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_fit_line()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SeriesFitLineDynamic : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_fit_line_dynamic()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SeriesFit2lines : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_fit_2lines()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SeriesFit2linesDynamic : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_fit_2lines_dynamic()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SeriesOutliers : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_outliers()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SeriesPeriodsDetect : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_periods_detect()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SeriesPeriodsValidate : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_periods_validate()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SeriesStatsDynamic : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_stats_dynamic()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SeriesStats : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_stats()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SeriesFillBackward : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_fill_backward()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SeriesFillConst : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_fill_const()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SeriesFillForward : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_fill_forward()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +class SeriesFillLinear : public IParserKQLFunction +{ +protected: + const char * getName() const override { return "series_fill_linear()"; } + bool convertImpl(String & out, IParser::Pos & pos) override; +}; + +} diff --git a/src/Parsers/Kusto/ParserKQLDateTypeTimespan.cpp b/src/Parsers/Kusto/ParserKQLDateTypeTimespan.cpp new file mode 100644 index 00000000000..c4f84d576cb --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLDateTypeTimespan.cpp @@ -0,0 +1,202 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +bool ParserKQLDateTypeTimespan ::parseImpl(Pos & pos, [[maybe_unused]] ASTPtr & node, Expected & expected) +{ + String token; + const char * current_word = pos->begin; + expected.add(pos, current_word); + + if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral) + token = String(pos->begin + 1, pos->end - 1); + else + token = String(pos->begin, pos->end); + if (!parseConstKQLTimespan(token)) + return false; + + return true; +} + +double ParserKQLDateTypeTimespan::toSeconds() +{ + switch (time_span_unit) + { + case KQLTimespanUint::day: + return time_span * 24 * 60 * 60; + case KQLTimespanUint::hour: + return time_span * 60 * 60; + case KQLTimespanUint::minute: + return time_span * 60; + case KQLTimespanUint::second: + return time_span; + case KQLTimespanUint::millisec: + return time_span / 1000.0; + case KQLTimespanUint::microsec: + return time_span / 1000000.0; + case KQLTimespanUint::nanosec: + return time_span / 1000000000.0; + case KQLTimespanUint::tick: + return time_span / 10000000.0; + } +} + +bool ParserKQLDateTypeTimespan ::parseConstKQLTimespan(const String & text) +{ + std::unordered_map timespan_suffixes + = {{"d", KQLTimespanUint::day}, + {"day", KQLTimespanUint::day}, + {"days", KQLTimespanUint::day}, + {"h", KQLTimespanUint::hour}, + {"hr", KQLTimespanUint::hour}, + {"hrs", KQLTimespanUint::hour}, + {"hour", KQLTimespanUint::hour}, + {"hours", KQLTimespanUint::hour}, + {"m", KQLTimespanUint::minute}, + {"min", KQLTimespanUint::minute}, + {"minute", KQLTimespanUint::minute}, + {"minutes", KQLTimespanUint::minute}, + {"s", KQLTimespanUint::second}, + {"sec", KQLTimespanUint::second}, + {"second", KQLTimespanUint::second}, + {"seconds", KQLTimespanUint::second}, + {"ms", KQLTimespanUint::millisec}, + {"milli", KQLTimespanUint::millisec}, + {"millis", KQLTimespanUint::millisec}, + {"millisec", KQLTimespanUint::millisec}, + {"millisecond", KQLTimespanUint::millisec}, + {"milliseconds", KQLTimespanUint::millisec}, + {"micro", KQLTimespanUint::microsec}, + {"micros", KQLTimespanUint::microsec}, + {"microsec", KQLTimespanUint::microsec}, + {"microsecond", KQLTimespanUint::microsec}, + {"microseconds", KQLTimespanUint::microsec}, + {"nano", KQLTimespanUint::nanosec}, + {"nanos", KQLTimespanUint::nanosec}, + {"nanosec", KQLTimespanUint::nanosec}, + {"nanosecond", KQLTimespanUint::nanosec}, + {"nanoseconds", KQLTimespanUint::nanosec}, + {"tick", KQLTimespanUint::tick}, + {"ticks", KQLTimespanUint::tick}}; + + int days = 0, hours = 0, minutes = 0, seconds = 0, sec_scale_len = 0; + double nanoseconds = 00.00; + + const char * ptr = text.c_str(); + bool sign = false; + + auto scan_digit = [&](const char * start) -> int + { + const auto * index = start; + while (isdigit(*index)) + ++index; + return index > start ? static_cast(index - start) : -1; + }; + if (*ptr == '-') + { + sign = true; + ++ptr; + } + auto number_len = scan_digit(ptr); + if (number_len <= 0) + return false; + + days = std::stoi(String(ptr, ptr + number_len)); + + if (*(ptr + number_len) == '.') + { + auto fraction_len = scan_digit(ptr + number_len + 1); + if (fraction_len >= 0) + { + hours = std::stoi(String(ptr + number_len + 1, ptr + number_len + 1 + fraction_len)); + number_len += fraction_len + 1; + } + } + else if (*(ptr + number_len) == '\0') + { + if (sign) + time_span = -(std::stoi(String(ptr, ptr + number_len))) * 86400; + else + time_span = std::stoi(String(ptr, ptr + number_len)) * 86400; + + time_span_unit = KQLTimespanUint::second; + return true; + } + else + { + hours = days; + days = 0; + } + + if (*(ptr + number_len) != ':') + { + String timespan_suffix(ptr + number_len, ptr + text.size()); + + trim(timespan_suffix); + if (timespan_suffixes.find(timespan_suffix) == timespan_suffixes.end()) + return false; + + time_span = std::stod(String(ptr, ptr + number_len)); + time_span_unit = timespan_suffixes[timespan_suffix]; + + return true; + } + + if (hours > 23) + return false; + + auto min_len = scan_digit(ptr + number_len + 1); + if (min_len < 0) + return false; + + minutes = std::stoi(String(ptr + number_len + 1, ptr + number_len + 1 + min_len)); + if (minutes > 59) + return false; + + number_len += min_len + 1; + if (*(ptr + number_len) == ':') + { + auto sec_len = scan_digit(ptr + number_len + 1); + if (sec_len > 0) + { + seconds = std::stoi(String(ptr + number_len + 1, ptr + number_len + 1 + sec_len)); + if (seconds > 59) + return false; + + number_len += sec_len + 1; + if (*(ptr + number_len) == '.') + { + sec_scale_len = scan_digit(ptr + number_len + 1); + if (sec_scale_len > 0) + { + nanoseconds = std::stoi(String(ptr + number_len + 1, ptr + number_len + 1 + sec_scale_len)); + + if (nanoseconds > 1000000000) + return false; + } + } + } + } + auto exponent = 9 - sec_scale_len; // max supported length of fraction of seconds is 9 + nanoseconds = nanoseconds * pow(10, exponent); + + if (sign) + time_span = -(days * 86400 + hours * 3600 + minutes * 60 + seconds + (nanoseconds / 1000000000)); + else + time_span = days * 86400 + hours * 3600 + minutes * 60 + seconds + (nanoseconds / 1000000000); + + time_span_unit = KQLTimespanUint::second; + + return true; +} + +} diff --git a/src/Parsers/Kusto/ParserKQLDateTypeTimespan.h b/src/Parsers/Kusto/ParserKQLDateTypeTimespan.h new file mode 100644 index 00000000000..232bc01d7e9 --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLDateTypeTimespan.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ParserKQLDateTypeTimespan : public ParserKQLBase +{ +public: + enum class KQLTimespanUint : uint8_t + { + day, + hour, + minute, + second, + millisec, + microsec, + nanosec, + tick + }; + bool parseConstKQLTimespan(const String & text); + double toSeconds(); + +protected: + const char * getName() const override { return "KQLDateTypeTimespan"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + +private: + double time_span; + KQLTimespanUint time_span_unit; +}; + +} diff --git a/src/Parsers/Kusto/ParserKQLDistinct.cpp b/src/Parsers/Kusto/ParserKQLDistinct.cpp new file mode 100644 index 00000000000..2de4d2c28e7 --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLDistinct.cpp @@ -0,0 +1,26 @@ +#include +#include +#include +namespace DB +{ + +bool ParserKQLDistinct::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr select_expression_list; + String expr; + + expr = getExprFromToken(pos); + + Tokens tokens(expr.c_str(), expr.c_str() + expr.size()); + IParser::Pos new_pos(tokens, pos.max_depth); + + if (!ParserNotEmptyExpressionList(false).parse(new_pos, select_expression_list, expected)) + return false; + + node->as()->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_expression_list)); + node->as()->distinct = true; + + return true; +} + +} diff --git a/src/Parsers/Kusto/ParserKQLDistinct.h b/src/Parsers/Kusto/ParserKQLDistinct.h new file mode 100644 index 00000000000..e3471f351b2 --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLDistinct.h @@ -0,0 +1,16 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ParserKQLDistinct : public ParserKQLBase +{ +protected: + const char * getName() const override { return "KQL distinct"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/Kusto/ParserKQLExtend.cpp b/src/Parsers/Kusto/ParserKQLExtend.cpp new file mode 100644 index 00000000000..ac5beb80576 --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLExtend.cpp @@ -0,0 +1,90 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +bool ParserKQLExtend ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr select_query; + int32_t new_column_index = 1; + + String extend_expr = getExprFromToken(pos); + + String except_str; + String new_extend_str; + Tokens ntokens(extend_expr.c_str(), extend_expr.c_str() + extend_expr.size()); + IParser::Pos npos(ntokens, pos.max_depth); + + String alias; + + auto apply_alias = [&] + { + if (alias.empty()) + { + alias = std::format("Column{}", new_column_index); + ++new_column_index; + new_extend_str += " AS"; + } + else + except_str = except_str.empty() ? " except " + alias : except_str + " except " + alias; + + new_extend_str = new_extend_str + " " + alias; + + alias.clear(); + }; + + int32_t round_bracket_count = 0; + int32_t square_bracket_count = 0; + while (!npos->isEnd()) + { + if (npos->type == TokenType::OpeningRoundBracket) + ++round_bracket_count; + if (npos->type == TokenType::OpeningSquareBracket) + ++square_bracket_count; + if (npos->type == TokenType::ClosingRoundBracket) + --round_bracket_count; + if (npos->type == TokenType::ClosingSquareBracket) + --square_bracket_count; + + auto expr = String(npos->begin, npos->end); + if (expr == "AS") + { + ++npos; + alias = String(npos->begin, npos->end); + } + + if (npos->type == TokenType::Comma && square_bracket_count == 0 && round_bracket_count == 0) + { + apply_alias(); + new_extend_str += ", "; + } + else + new_extend_str = new_extend_str.empty() ? expr : new_extend_str + " " + expr; + + ++npos; + } + apply_alias(); + + String expr = std::format("SELECT * {}, {} from prev", except_str, new_extend_str); + Tokens tokens(expr.c_str(), expr.c_str() + expr.size()); + IParser::Pos new_pos(tokens, pos.max_depth); + + if (!ParserSelectQuery().parse(new_pos, select_query, expected)) + return false; + if (!setSubQuerySource(select_query, node, false, false)) + return false; + + node = select_query; + return true; +} + +} diff --git a/src/Parsers/Kusto/ParserKQLExtend.h b/src/Parsers/Kusto/ParserKQLExtend.h new file mode 100644 index 00000000000..90d92b6a1c2 --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLExtend.h @@ -0,0 +1,17 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ParserKQLExtend : public ParserKQLBase +{ +protected: + const char * getName() const override { return "KQL extend"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/Kusto/ParserKQLFilter.cpp b/src/Parsers/Kusto/ParserKQLFilter.cpp index 3a399bdccdb..74d8610ecd4 100644 --- a/src/Parsers/Kusto/ParserKQLFilter.cpp +++ b/src/Parsers/Kusto/ParserKQLFilter.cpp @@ -1,19 +1,19 @@ #include -#include #include -#include +#include #include #include +#include namespace DB { -bool ParserKQLFilter :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +bool ParserKQLFilter::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { String expr = getExprFromToken(pos); ASTPtr where_expression; - Tokens token_filter(expr.c_str(), expr.c_str()+expr.size()); + Tokens token_filter(expr.c_str(), expr.c_str() + expr.size()); IParser::Pos pos_filter(token_filter, pos.max_depth); if (!ParserExpressionWithOptionalAlias(false).parse(pos_filter, where_expression, expected)) return false; diff --git a/src/Parsers/Kusto/ParserKQLLimit.cpp b/src/Parsers/Kusto/ParserKQLLimit.cpp index bb8e08fd378..910f0e8e1a3 100644 --- a/src/Parsers/Kusto/ParserKQLLimit.cpp +++ b/src/Parsers/Kusto/ParserKQLLimit.cpp @@ -1,15 +1,13 @@ -#include -#include -#include -#include -#include #include -#include +#include +#include +#include +#include namespace DB { -bool ParserKQLLimit :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +bool ParserKQLLimit::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr limit_length; diff --git a/src/Parsers/Kusto/ParserKQLMVExpand.cpp b/src/Parsers/Kusto/ParserKQLMVExpand.cpp new file mode 100644 index 00000000000..bf5fbe64d0b --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLMVExpand.cpp @@ -0,0 +1,311 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB::ErrorCodes +{ +extern const int UNKNOWN_TYPE; +} + +namespace DB +{ + +std::unordered_map ParserKQLMVExpand::type_cast + = {{"bool", "Boolean"}, + {"boolean", "Boolean"}, + {"datetime", "DateTime"}, + {"date", "DateTime"}, + {"guid", "UUID"}, + {"int", "Int32"}, + {"long", "Int64"}, + {"real", "Float64"}, + {"double", "Float64"}, + {"string", "String"}}; + +bool ParserKQLMVExpand::parseColumnArrayExprs(ColumnArrayExprs & column_array_exprs, Pos & pos, Expected & expected) +{ + ParserToken equals(TokenType::Equals); + ParserToken open_bracket(TokenType::OpeningRoundBracket); + ParserToken close_bracket(TokenType::ClosingRoundBracket); + ParserToken comma(TokenType::Comma); + + ParserKeyword s_to("to"); + ParserKeyword s_type("typeof"); + uint16_t bracket_count = 0; + Pos expr_begin_pos = pos; + Pos expr_end_pos = pos; + + String alias; + String column_array_expr; + String to_type; + --expr_end_pos; + + while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon) + { + if (pos->type == TokenType::OpeningRoundBracket) + ++bracket_count; + + if (pos->type == TokenType::ClosingRoundBracket) + --bracket_count; + + if (String(pos->begin, pos->end) == "=") + { + --pos; + alias = String(pos->begin, pos->end); + ++pos; + ++pos; + expr_begin_pos = pos; + } + + auto add_columns = [&] + { + column_array_expr = getExprFromToken(String(expr_begin_pos->begin, expr_end_pos->end), pos.max_depth); + + if (alias.empty()) + { + alias = expr_begin_pos == expr_end_pos ? column_array_expr : String(expr_begin_pos->begin, expr_begin_pos->end) + "_"; + } + column_array_exprs.push_back(ColumnArrayExpr(alias, column_array_expr, to_type)); + }; + + if (s_to.ignore(pos, expected)) + { + --pos; + --pos; + expr_end_pos = pos; + ++pos; + ++pos; + + column_array_expr = String(expr_begin_pos->begin, expr_end_pos->end); + + if (!s_type.ignore(pos, expected)) + return false; + if (!open_bracket.ignore(pos, expected)) + return false; + to_type = String(pos->begin, pos->end); + + if (type_cast.find(to_type) == type_cast.end()) + throw Exception(ErrorCodes::UNKNOWN_TYPE, "{} is not a supported kusto data type for mv-expand", to_type); + + ++pos; + if (!close_bracket.ignore(pos, expected)) + return false; + --pos; + } + + if ((pos->type == TokenType::Comma && bracket_count == 0) || String(pos->begin, pos->end) == "limit" + || pos->type == TokenType::Semicolon) + { + if (column_array_expr.empty()) + { + expr_end_pos = pos; + --expr_end_pos; + } + add_columns(); + expr_begin_pos = pos; + expr_end_pos = pos; + ++expr_begin_pos; + + alias.clear(); + column_array_expr.clear(); + to_type.clear(); + + if (pos->type == TokenType::Semicolon) + break; + } + + if (String(pos->begin, pos->end) == "limit") + break; + if (!pos->isEnd()) + ++pos; + if (pos->isEnd() || pos->type == TokenType::PipeMark || pos->type == TokenType::Semicolon) + { + if (expr_end_pos < expr_begin_pos) + { + expr_end_pos = pos; + --expr_end_pos; + } + add_columns(); + break; + } + } + return true; +} + +bool ParserKQLMVExpand::parserMVExpand(KQLMVExpand & kql_mv_expand, Pos & pos, Expected & expected) +{ + ParserKeyword s_bagexpansion("bagexpansion"); + ParserKeyword s_kind("kind"); + ParserKeyword s_with_itemindex("with_itemindex"); + ParserKeyword s_limit("limit"); + + ParserToken equals(TokenType::Equals); + ParserToken comma(TokenType::Comma); + + auto & column_array_exprs = kql_mv_expand.column_array_exprs; + auto & bagexpansion = kql_mv_expand.bagexpansion; + auto & with_itemindex = kql_mv_expand.with_itemindex; + auto & limit = kql_mv_expand.limit; + + if (s_bagexpansion.ignore(pos, expected)) + { + if (!equals.ignore(pos, expected)) + return false; + bagexpansion = String(pos->begin, pos->end); + ++pos; + } + else if (s_kind.ignore(pos, expected)) + { + if (!equals.ignore(pos, expected)) + return false; + bagexpansion = String(pos->begin, pos->end); + ++pos; + } + + if (s_with_itemindex.ignore(pos, expected)) + { + if (!equals.ignore(pos, expected)) + return false; + with_itemindex = String(pos->begin, pos->end); + ++pos; + } + + if (!parseColumnArrayExprs(column_array_exprs, pos, expected)) + return false; + + if (s_limit.ignore(pos, expected)) + limit = String(pos->begin, pos->end); + + return true; +} + +bool ParserKQLMVExpand::genQuery(KQLMVExpand & kql_mv_expand, ASTPtr & select_node, int32_t max_depth) +{ + String expand_str; + String cast_type_column_remove, cast_type_column_rename; + String cast_type_column_restore, cast_type_column_restore_name; + String row_count_str; + String extra_columns; + String input = "dummy_input"; + for (auto column : kql_mv_expand.column_array_exprs) + { + if (column.alias == column.column_array_expr) + expand_str = expand_str.empty() ? String("ARRAY JOIN ") + column.alias : expand_str + "," + column.alias; + else + { + expand_str = expand_str.empty() ? std::format("ARRAY JOIN {} AS {} ", column.column_array_expr, column.alias) + : expand_str + std::format(", {} AS {}", column.column_array_expr, column.alias); + extra_columns = extra_columns + ", " + column.alias; + } + + if (!column.to_type.empty()) + { + cast_type_column_remove + = cast_type_column_remove.empty() ? " Except " + column.alias : cast_type_column_remove + " Except " + column.alias; + String rename_str; + + if (type_cast[column.to_type] == "Boolean") + rename_str = std::format( + "accurateCastOrNull(toInt64OrNull(toString({0})),'{1}') as {0}_ali", column.alias, type_cast[column.to_type]); + else + rename_str = std::format("accurateCastOrNull({0},'{1}') as {0}_ali", column.alias, type_cast[column.to_type]); + + cast_type_column_rename = cast_type_column_rename.empty() ? rename_str : cast_type_column_rename + "," + rename_str; + cast_type_column_restore = cast_type_column_restore.empty() + ? std::format(" Except {}_ali ", column.alias) + : cast_type_column_restore + std::format(" Except {}_ali ", column.alias); + cast_type_column_restore_name = cast_type_column_restore_name.empty() + ? std::format("{0}_ali as {0}", column.alias) + : cast_type_column_restore_name + std::format(", {0}_ali as {0}", column.alias); + } + + if (!kql_mv_expand.with_itemindex.empty()) + { + row_count_str = row_count_str.empty() ? "length(" + column.alias + ")" : row_count_str + ", length(" + column.alias + ")"; + } + } + + String columns = "*"; + if (!row_count_str.empty()) + { + expand_str += std::format(", range(0, arrayMax([{}])) AS {} ", row_count_str, kql_mv_expand.with_itemindex); + columns = kql_mv_expand.with_itemindex + " , " + columns; + } + + if (!kql_mv_expand.limit.empty()) + expand_str += " LIMIT " + kql_mv_expand.limit; + + auto query = std::format("(Select {} {} From {} {})", columns, extra_columns, input, expand_str); + + ASTPtr sub_query_node; + Expected expected; + + if (cast_type_column_remove.empty()) + { + query = std::format("Select {} {} From {} {}", columns, extra_columns, input, expand_str); + if (!parseSQLQueryByString(std::make_unique(), query, sub_query_node, max_depth)) + return false; + if (!setSubQuerySource(sub_query_node, select_node, false, false)) + return false; + select_node = std::move(sub_query_node); + } + else + { + query = std::format("(Select {} {} From {} {})", columns, extra_columns, input, expand_str); + if (!parseSQLQueryByString(std::make_unique(), query, sub_query_node, max_depth)) + return false; + if (!setSubQuerySource(sub_query_node, select_node, true, false)) + return false; + select_node = std::move(sub_query_node); + + auto rename_query = std::format("(Select * {}, {} From {})", cast_type_column_remove, cast_type_column_rename, "query"); + if (!parseSQLQueryByString(std::make_unique(), rename_query, sub_query_node, max_depth)) + return false; + if (!setSubQuerySource(sub_query_node, select_node, true, true)) + return false; + + select_node = std::move(sub_query_node); + query = std::format("Select * {}, {} from {}", cast_type_column_restore, cast_type_column_restore_name, "rename_query"); + + if (!parseSQLQueryByString(std::make_unique(), query, sub_query_node, max_depth)) + return false; + sub_query_node->as()->setExpression(ASTSelectQuery::Expression::TABLES, std::move(select_node)); + select_node = std::move(sub_query_node); + } + return true; +} + +bool ParserKQLMVExpand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr setting; + ASTPtr select_expression_list; + auto begin = pos; + + KQLMVExpand kql_mv_expand; + if (!parserMVExpand(kql_mv_expand, pos, expected)) + return false; + if (!genQuery(kql_mv_expand, node, pos.max_depth)) + return false; + + const String setting_str = "enable_unaligned_array_join = 1"; + Tokens token_settings(setting_str.c_str(), setting_str.c_str() + setting_str.size()); + IParser::Pos pos_settings(token_settings, pos.max_depth); + + if (!ParserSetQuery(true).parse(pos_settings, setting, expected)) + return false; + node->as()->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(setting)); + + pos = begin; + return true; +} + +} diff --git a/src/Parsers/Kusto/ParserKQLMVExpand.h b/src/Parsers/Kusto/ParserKQLMVExpand.h new file mode 100644 index 00000000000..61f206bb00d --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLMVExpand.h @@ -0,0 +1,41 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ParserKQLMVExpand : public ParserKQLBase +{ +protected: + static std::unordered_map type_cast; + + struct ColumnArrayExpr + { + String alias; + String column_array_expr; + String to_type; + ColumnArrayExpr(String alias_, String column_array_expr_, String to_type_) + : alias(alias_), column_array_expr(column_array_expr_), to_type(to_type_) + { + } + }; + using ColumnArrayExprs = std::vector; + + struct KQLMVExpand + { + ColumnArrayExprs column_array_exprs; + String bagexpansion; + String with_itemindex; + String limit; + }; + + static bool parseColumnArrayExprs(ColumnArrayExprs & column_array_exprs, Pos & pos, Expected & expected); + static bool parserMVExpand(KQLMVExpand & kql_mv_expand, Pos & pos, Expected & expected); + static bool genQuery(KQLMVExpand & kql_mv_expand, ASTPtr & select_node, int32_t max_depth); + + const char * getName() const override { return "KQL mv-expand"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/src/Parsers/Kusto/ParserKQLMakeSeries.cpp b/src/Parsers/Kusto/ParserKQLMakeSeries.cpp new file mode 100644 index 00000000000..a3727653049 --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLMakeSeries.cpp @@ -0,0 +1,426 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +bool ParserKQLMakeSeries ::parseAggregationColumns(AggregationColumns & aggregation_columns, Pos & pos) +{ + std::unordered_set allowed_aggregation( + {"avg", + "avgif", + "count", + "countif", + "dcount", + "dcountif", + "max", + "maxif", + "min", + "minif", + "percentile", + "take_any", + "stdev", + "sum", + "sumif", + "variance"}); + + Expected expected; + ParserKeyword s_default("default"); + ParserToken equals(TokenType::Equals); + ParserToken open_bracket(TokenType::OpeningRoundBracket); + ParserToken close_bracket(TokenType::ClosingRoundBracket); + ParserToken comma(TokenType::Comma); + + while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon) + { + String alias; + String aggregation_fun; + String column; + double default_value = 0; + + String first_token(pos->begin, pos->end); + + ++pos; + if (equals.ignore(pos, expected)) + { + alias = std::move(first_token); + aggregation_fun = String(pos->begin, pos->end); + ++pos; + } + else + aggregation_fun = std::move(first_token); + + if (allowed_aggregation.find(aggregation_fun) == allowed_aggregation.end()) + return false; + + if (open_bracket.ignore(pos, expected)) + column = String(pos->begin, pos->end); + else + return false; + + ++pos; + if (!close_bracket.ignore(pos, expected)) + return false; + + if (s_default.ignore(pos, expected)) + { + if (!equals.ignore(pos, expected)) + return false; + + default_value = std::stod(String(pos->begin, pos->end)); + ++pos; + } + if (alias.empty()) + alias = std::format("{}_{}", aggregation_fun, column); + aggregation_columns.push_back(AggregationColumn(alias, aggregation_fun, column, default_value)); + + if (!comma.ignore(pos, expected)) + break; + } + return true; +} + +bool ParserKQLMakeSeries ::parseFromToStepClause(FromToStepClause & from_to_step, Pos & pos) +{ + auto begin = pos; + auto from_pos = begin; + auto to_pos = begin; + auto step_pos = begin; + auto end_pos = begin; + + while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon) + { + if (String(pos->begin, pos->end) == "from") + from_pos = pos; + if (String(pos->begin, pos->end) == "to") + to_pos = pos; + if (String(pos->begin, pos->end) == "step") + step_pos = pos; + if (String(pos->begin, pos->end) == "by") + { + end_pos = pos; + break; + } + ++pos; + } + + if (end_pos == begin) + end_pos = pos; + + if (String(step_pos->begin, step_pos->end) != "step") + return false; + + if (String(from_pos->begin, from_pos->end) == "from") + { + ++from_pos; + auto end_from_pos = (to_pos != begin) ? to_pos : step_pos; + --end_from_pos; + from_to_step.from_str = String(from_pos->begin, end_from_pos->end); + } + + if (String(to_pos->begin, to_pos->end) == "to") + { + ++to_pos; + --step_pos; + from_to_step.to_str = String(to_pos->begin, step_pos->end); + ++step_pos; + } + --end_pos; + ++step_pos; + from_to_step.step_str = String(step_pos->begin, end_pos->end); + + if (String(step_pos->begin, step_pos->end) == "time" || String(step_pos->begin, step_pos->end) == "timespan" + || ParserKQLDateTypeTimespan().parseConstKQLTimespan(from_to_step.step_str)) + { + from_to_step.is_timespan = true; + from_to_step.step = std::stod(getExprFromToken(from_to_step.step_str, pos.max_depth)); + } + else + from_to_step.step = std::stod(from_to_step.step_str); + + return true; +} + +bool ParserKQLMakeSeries ::makeSeries(KQLMakeSeries & kql_make_series, ASTPtr & select_node, const uint32_t & max_depth) +{ + const uint64_t era_diff + = 62135596800; // this magic number is the differicen is second form 0001-01-01 (Azure start time ) and 1970-01-01 (CH start time) + + String start_str, end_str; + String sub_query, main_query; + + auto & aggregation_columns = kql_make_series.aggregation_columns; + auto & from_to_step = kql_make_series.from_to_step; + auto & subquery_columns = kql_make_series.subquery_columns; + auto & axis_column = kql_make_series.axis_column; + auto & group_expression = kql_make_series.group_expression; + auto step = from_to_step.step; + + if (!kql_make_series.from_to_step.from_str.empty()) + start_str = getExprFromToken(kql_make_series.from_to_step.from_str, max_depth); + + if (!kql_make_series.from_to_step.to_str.empty()) + end_str = getExprFromToken(from_to_step.to_str, max_depth); + + auto date_type_cast = [&](String & src) + { + Tokens tokens(src.c_str(), src.c_str() + src.size()); + IParser::Pos pos(tokens, max_depth); + String res; + while (!pos->isEnd()) + { + String tmp = String(pos->begin, pos->end); + if (tmp == "parseDateTime64BestEffortOrNull") + tmp = "toDateTime64"; + + res = res.empty() ? tmp : res + " " + tmp; + ++pos; + } + return res; + }; + + start_str = date_type_cast(start_str); + end_str = date_type_cast(end_str); + + String bin_str, start, end; + + uint64_t diff = 0; + String axis_column_format; + String axis_str; + + auto get_group_expression_alias = [&] + { + std::vector group_expression_tokens; + Tokens tokens(group_expression.c_str(), group_expression.c_str() + group_expression.size()); + IParser::Pos pos(tokens, max_depth); + while (!pos->isEnd()) + { + if (String(pos->begin, pos->end) == "AS") + { + if (!group_expression_tokens.empty()) + group_expression_tokens.pop_back(); + ++pos; + group_expression_tokens.push_back(String(pos->begin, pos->end)); + } + else + group_expression_tokens.push_back(String(pos->begin, pos->end)); + ++pos; + } + String res; + for (auto const & token : group_expression_tokens) + res = res + token + " "; + return res; + }; + + auto group_expression_alias = get_group_expression_alias(); + + if (from_to_step.is_timespan) + { + axis_column_format = std::format("toFloat64(toDateTime64({}, 9, 'UTC'))", axis_column); + } + else + axis_column_format = std::format("toFloat64({})", axis_column); + + if (!start_str.empty()) // has from + { + bin_str = std::format( + "toFloat64({0}) + (toInt64((({1} - toFloat64({0})) / {2})) * {2}) AS {3}_ali", + start_str, + axis_column_format, + step, + axis_column); + start = std::format("toUInt64({})", start_str); + } + else + { + if (from_to_step.is_timespan) + diff = era_diff; + bin_str = std::format(" toFloat64(toInt64(({0} + {1}) / {2}) * {2}) AS {3}_ali ", axis_column_format, diff, step, axis_column); + } + + if (!end_str.empty()) + end = std::format("toUInt64({})", end_str); + + String range, condition; + + if (!start_str.empty() && !end_str.empty()) + { + range = std::format("range({}, {}, toUInt64({}))", start, end, step); + condition = std::format("where toInt64({0}) >= {1} and toInt64({0}) < {2}", axis_column_format, start, end); + } + else if (start_str.empty() && !end_str.empty()) + { + range = std::format("range(low, {} + {}, toUInt64({}))", end, diff, step); + condition = std::format("where toInt64({0}) - {1} < {2}", axis_column_format, diff, end); + } + else if (!start_str.empty() && end_str.empty()) + { + range = std::format("range({}, high, toUInt64({}))", start, step); + condition = std::format("where toInt64({}) >= {}", axis_column_format, start); + } + else + { + range = std::format("range(low, high, toUInt64({}))", step); + condition = " "; + } + + auto range_len = std::format("length({})", range); + + String sub_sub_query; + if (group_expression.empty()) + sub_sub_query = std::format( + " (Select {0}, {1} FROM {2} {4} GROUP BY {3}_ali ORDER BY {3}_ali) ", + subquery_columns, + bin_str, + "table_name", + axis_column, + condition); + else + sub_sub_query = std::format( + " (Select {0}, {1}, {2} FROM {3} {5} GROUP BY {0}, {4}_ali ORDER BY {4}_ali) ", + group_expression, + subquery_columns, + bin_str, + "table_name", + axis_column, + condition); + + ASTPtr sub_query_node; + + if (!ParserSimpleCHSubquery(select_node).parseByString(sub_sub_query, sub_query_node, max_depth)) + return false; + select_node->as()->setExpression(ASTSelectQuery::Expression::TABLES, std::move(sub_query_node)); + + if (!group_expression.empty()) + main_query = std::format("{} ", group_expression_alias); + + auto axis_and_agg_alias_list = axis_column; + auto final_axis_agg_alias_list = std::format("tupleElement(zipped,1) AS {}", axis_column); + int idx = 2; + for (auto agg_column : aggregation_columns) + { + String agg_group_column = std::format( + "arrayConcat(groupArray({}_ali) as ga, arrayMap(x -> ({}),range(0, toUInt32({} - length(ga) < 0 ? 0 : {} - length(ga)),1)))" + "as {}", + agg_column.alias, + agg_column.default_value, + range_len, + range_len, + agg_column.alias); + main_query = main_query.empty() ? agg_group_column : main_query + ", " + agg_group_column; + + axis_and_agg_alias_list += ", " + agg_column.alias; + final_axis_agg_alias_list += std::format(", tupleElement(zipped,{}) AS {}", idx, agg_column.alias); + } + + if (from_to_step.is_timespan) + axis_str = std::format( + "arrayDistinct(arrayConcat(groupArray(toDateTime64({0}_ali - {1},9,'UTC')), arrayMap(x->(toDateTime64(x - {1} ,9,'UTC'))," + "{2}))) as {0}", + axis_column, + diff, + range); + else + axis_str + = std::format("arrayDistinct(arrayConcat(groupArray({0}_ali), arrayMap(x->(toFloat64(x)), {1}))) as {0}", axis_column, range); + + main_query += ", " + axis_str; + auto sub_group_by = group_expression.empty() ? "" : std::format("GROUP BY {}", group_expression_alias); + + sub_query = std::format( + "( SELECT toUInt64(min({}_ali)) AS low, toUInt64(max({}_ali))+ {} AS high, arraySort(arrayZip({})) as zipped, {} FROM {} {} )", + axis_column, + axis_column, + step, + axis_and_agg_alias_list, + main_query, + sub_sub_query, + sub_group_by); + + if (group_expression.empty()) + main_query = std::format("{}", final_axis_agg_alias_list); + else + main_query = std::format("{},{}", group_expression_alias, final_axis_agg_alias_list); + + if (!ParserSimpleCHSubquery(select_node).parseByString(sub_query, sub_query_node, max_depth)) + return false; + select_node->as()->setExpression(ASTSelectQuery::Expression::TABLES, std::move(sub_query_node)); + + kql_make_series.sub_query = std::move(sub_query); + kql_make_series.main_query = std::move(main_query); + + return true; +} + +bool ParserKQLMakeSeries ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + auto begin = pos; + ParserKeyword s_on("on"); + ParserKeyword s_by("by"); + + ParserToken equals(TokenType::Equals); + ParserToken comma(TokenType::Comma); + + ASTPtr select_expression_list; + + KQLMakeSeries kql_make_series; + auto & aggregation_columns = kql_make_series.aggregation_columns; + auto & from_to_step = kql_make_series.from_to_step; + auto & subquery_columns = kql_make_series.subquery_columns; + auto & axis_column = kql_make_series.axis_column; + auto & group_expression = kql_make_series.group_expression; + + ParserKQLDateTypeTimespan time_span; + + //const auto make_series_parameters = getMakeSeriesParameters(pos); + + if (!parseAggregationColumns(aggregation_columns, pos)) + return false; + + if (!s_on.ignore(pos, expected)) + return false; + + axis_column = String(pos->begin, pos->end); + ++pos; + + if (!parseFromToStepClause(from_to_step, pos)) + return false; + + if (s_by.ignore(pos, expected)) + { + group_expression = getExprFromToken(pos); + if (group_expression.empty()) + return false; + } + + for (auto agg_column : aggregation_columns) + { + String column_str = std::format("{}({}) AS {}_ali", agg_column.aggregation_fun, agg_column.column, agg_column.alias); + if (subquery_columns.empty()) + subquery_columns = column_str; + else + subquery_columns += ", " + column_str; + } + + makeSeries(kql_make_series, node, pos.max_depth); + + Tokens token_main_query(kql_make_series.main_query.c_str(), kql_make_series.main_query.c_str() + kql_make_series.main_query.size()); + IParser::Pos pos_main_query(token_main_query, pos.max_depth); + + if (!ParserNotEmptyExpressionList(true).parse(pos_main_query, select_expression_list, expected)) + return false; + node->as()->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_expression_list)); + + pos = begin; + return true; +} + +} diff --git a/src/Parsers/Kusto/ParserKQLMakeSeries.h b/src/Parsers/Kusto/ParserKQLMakeSeries.h new file mode 100644 index 00000000000..ef7cc4976f6 --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLMakeSeries.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ParserKQLMakeSeries : public ParserKQLBase +{ +protected: + struct AggregationColumn + { + String alias; + String aggregation_fun; + String column; + double default_value; + AggregationColumn(String alias_, String aggregation_fun_, String column_, double default_value_) + : alias(alias_), aggregation_fun(aggregation_fun_), column(column_), default_value(default_value_) + { + } + }; + using AggregationColumns = std::vector; + + struct FromToStepClause + { + String from_str; + String to_str; + String step_str; + bool is_timespan = false; + double step; + }; + + struct KQLMakeSeries + { + AggregationColumns aggregation_columns; + FromToStepClause from_to_step; + String axis_column; + String group_expression; + String subquery_columns; + String sub_query; + String main_query; + }; + + static bool makeSeries(KQLMakeSeries & kql_make_series, ASTPtr & select_node, const uint32_t & max_depth); + static bool parseAggregationColumns(AggregationColumns & aggregation_columns, Pos & pos); + static bool parseFromToStepClause(FromToStepClause & from_to_step, Pos & pos); + + const char * getName() const override { return "KQL make-series"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/Kusto/ParserKQLOperators.cpp b/src/Parsers/Kusto/ParserKQLOperators.cpp index 1575cffcc39..17b9bd74d0b 100644 --- a/src/Parsers/Kusto/ParserKQLOperators.cpp +++ b/src/Parsers/Kusto/ParserKQLOperators.cpp @@ -1,9 +1,131 @@ #include -#include -#include -#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include "KustoFunctions/IParserKQLFunction.h" +namespace +{ + +enum class KQLOperatorValue : uint16_t +{ + none, + between, + not_between, + contains, + not_contains, + contains_cs, + not_contains_cs, + endswith, + not_endswith, + endswith_cs, + not_endswith_cs, + equal, //=~ + not_equal, //!~ + equal_cs, //= + not_equal_cs, //!= + has, + not_has, + has_all, + has_any, + has_cs, + not_has_cs, + hasprefix, + not_hasprefix, + hasprefix_cs, + not_hasprefix_cs, + hassuffix, + not_hassuffix, + hassuffix_cs, + not_hassuffix_cs, + in_cs, //in + not_in_cs, //!in + in, //in~ + not_in, //!in~ + matches_regex, + startswith, + not_startswith, + startswith_cs, + not_startswith_cs, +}; + +const std::unordered_map KQLOperator = { + {"between", KQLOperatorValue::between}, + {"!between", KQLOperatorValue::not_between}, + {"contains", KQLOperatorValue::contains}, + {"!contains", KQLOperatorValue::not_contains}, + {"contains_cs", KQLOperatorValue::contains_cs}, + {"!contains_cs", KQLOperatorValue::not_contains_cs}, + {"endswith", KQLOperatorValue::endswith}, + {"!endswith", KQLOperatorValue::not_endswith}, + {"endswith_cs", KQLOperatorValue::endswith_cs}, + {"!endswith_cs", KQLOperatorValue::not_endswith_cs}, + {"=~", KQLOperatorValue::equal}, + {"!~", KQLOperatorValue::not_equal}, + {"==", KQLOperatorValue::equal_cs}, + {"!=", KQLOperatorValue::not_equal_cs}, + {"has", KQLOperatorValue::has}, + {"!has", KQLOperatorValue::not_has}, + {"has_all", KQLOperatorValue::has_all}, + {"has_any", KQLOperatorValue::has_any}, + {"has_cs", KQLOperatorValue::has_cs}, + {"!has_cs", KQLOperatorValue::not_has_cs}, + {"hasprefix", KQLOperatorValue::hasprefix}, + {"!hasprefix", KQLOperatorValue::not_hasprefix}, + {"hasprefix_cs", KQLOperatorValue::hasprefix_cs}, + {"!hasprefix_cs", KQLOperatorValue::not_hasprefix_cs}, + {"hassuffix", KQLOperatorValue::hassuffix}, + {"!hassuffix", KQLOperatorValue::not_hassuffix}, + {"hassuffix_cs", KQLOperatorValue::hassuffix_cs}, + {"!hassuffix_cs", KQLOperatorValue::not_hassuffix_cs}, + {"in", KQLOperatorValue::in_cs}, + {"!in", KQLOperatorValue::not_in_cs}, + {"in~", KQLOperatorValue::in}, + {"!in~", KQLOperatorValue::not_in}, + {"matches regex", KQLOperatorValue::matches_regex}, + {"startswith", KQLOperatorValue::startswith}, + {"!startswith", KQLOperatorValue::not_startswith}, + {"startswith_cs", KQLOperatorValue::startswith_cs}, + {"!startswith_cs", KQLOperatorValue::not_startswith_cs}, +}; + +void rebuildSubqueryForInOperator(DB::ASTPtr & node, bool useLowerCase) +{ + //A sub-query for in operator in kql can have multiple columns, but only takes the first column. + //A sub-query for in operator in ClickHouse can not have multiple columns + //So only take the first column if there are multiple columns. + //select * not working for subquery. (a tabular statement without project) + + const auto selectColumns = node->children[0]->children[0]->as()->select(); + while (selectColumns->children.size() > 1) + selectColumns->children.pop_back(); + + if (useLowerCase) + { + auto args = std::make_shared(); + args->children.push_back(selectColumns->children[0]); + auto func_lower = std::make_shared(); + func_lower->name = "lower"; + func_lower->children.push_back(selectColumns->children[0]); + func_lower->arguments = args; + if (selectColumns->children[0]->as()) + func_lower->alias = std::move(selectColumns->children[0]->as()->alias); + else if (selectColumns->children[0]->as()) + func_lower->alias = std::move(selectColumns->children[0]->as()->alias); + + auto funcs = std::make_shared(); + funcs->children.push_back(func_lower); + selectColumns->children[0] = std::move(funcs); + } +} + +} namespace DB { @@ -12,7 +134,7 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; } -String KQLOperators::genHasAnyAllOpExpr(std::vector &tokens, IParser::Pos &token_pos,String kql_op, String ch_op) +String KQLOperators::genHasAnyAllOpExpr(std::vector & tokens, IParser::Pos & token_pos, String kql_op, String ch_op) { String new_expr; Expected expected; @@ -28,7 +150,7 @@ String KQLOperators::genHasAnyAllOpExpr(std::vector &tokens, IParser::Po while (!token_pos->isEnd() && token_pos->type != TokenType::PipeMark && token_pos->type != TokenType::Semicolon) { - auto tmp_arg = String(token_pos->begin, token_pos->end); + auto tmp_arg = IParserKQLFunction::getExpression(token_pos); if (token_pos->type == TokenType::Comma) new_expr = new_expr + logic_op; else @@ -37,33 +159,114 @@ String KQLOperators::genHasAnyAllOpExpr(std::vector &tokens, IParser::Po ++token_pos; if (token_pos->type == TokenType::ClosingRoundBracket) break; - } tokens.pop_back(); return new_expr; } -String KQLOperators::genInOpExpr(IParser::Pos &token_pos, String kql_op, String ch_op) +String genEqOpExprCis(std::vector & tokens, DB::IParser::Pos & token_pos, const DB::String & ch_op) { - String new_expr; + DB::String tmp_arg(token_pos->begin, token_pos->end); - ParserToken s_lparen(TokenType::OpeningRoundBracket); + if (tokens.empty() || tmp_arg != "~") + return tmp_arg; - ASTPtr select; - Expected expected; + DB::String new_expr; + new_expr += "lower(" + tokens.back() + ")" + " "; + new_expr += ch_op + " "; + ++token_pos; + + if (token_pos->type == DB::TokenType::StringLiteral || token_pos->type == DB::TokenType::QuotedIdentifier) + new_expr += "lower('" + DB::IParserKQLFunction::escapeSingleQuotes(String(token_pos->begin + 1, token_pos->end - 1)) + "')"; + else + new_expr += "lower(" + DB::IParserKQLFunction::getExpression(token_pos) + ")"; + + tokens.pop_back(); + return new_expr; +} + +String genInOpExprCis(std::vector & tokens, DB::IParser::Pos & token_pos, const DB::String & kql_op, const DB::String & ch_op) +{ + DB::ParserKQLTableFunction kqlfun_p; + DB::ParserToken s_lparen(DB::TokenType::OpeningRoundBracket); + + DB::ASTPtr select; + DB::Expected expected; + DB::String new_expr; ++token_pos; if (!s_lparen.ignore(token_pos, expected)) - throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op); + throw DB::Exception(DB::ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op); + + if (tokens.empty()) + throw DB::Exception(DB::ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op); + + new_expr = "lower(" + tokens.back() + ") "; + tokens.pop_back(); + auto pos = token_pos; + if (kqlfun_p.parse(pos, select, expected)) + { + rebuildSubqueryForInOperator(select, true); + new_expr += ch_op + " (" + serializeAST(*select) + ")"; + token_pos = pos; + return new_expr; + } + + --token_pos; + --token_pos; + + new_expr += ch_op; + while (!token_pos->isEnd() && token_pos->type != DB::TokenType::PipeMark && token_pos->type != DB::TokenType::Semicolon) + { + auto tmp_arg = DB::String(token_pos->begin, token_pos->end); + if (token_pos->type != DB::TokenType::Comma && token_pos->type != DB::TokenType::ClosingRoundBracket + && token_pos->type != DB::TokenType::OpeningRoundBracket && token_pos->type != DB::TokenType::OpeningSquareBracket + && token_pos->type != DB::TokenType::ClosingSquareBracket && tmp_arg != "~" && tmp_arg != "dynamic") + { + if (token_pos->type == DB::TokenType::StringLiteral || token_pos->type == DB::TokenType::QuotedIdentifier) + new_expr += "lower('" + DB::IParserKQLFunction::escapeSingleQuotes(String(token_pos->begin + 1, token_pos->end - 1)) + "')"; + else + new_expr += "lower(" + tmp_arg + ")"; + } + else if (tmp_arg != "~" && tmp_arg != "dynamic" && tmp_arg != "[" && tmp_arg != "]") + new_expr += tmp_arg; + + if (token_pos->type == DB::TokenType::ClosingRoundBracket) + break; + ++token_pos; + } + return new_expr; +} + +std::string genInOpExpr(DB::IParser::Pos & token_pos, const std::string & kql_op, const std::string & ch_op) +{ + DB::ParserKQLTableFunction kqlfun_p; + DB::ParserToken s_lparen(DB::TokenType::OpeningRoundBracket); + + DB::ASTPtr select; + DB::Expected expected; + + ++token_pos; + if (!s_lparen.ignore(token_pos, expected)) + throw DB::Exception(DB::ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op); + + auto pos = token_pos; + if (kqlfun_p.parse(pos, select, expected)) + { + rebuildSubqueryForInOperator(select, false); + auto new_expr = ch_op + " (" + serializeAST(*select) + ")"; + token_pos = pos; + return new_expr; + } --token_pos; --token_pos; return ch_op; - } -String KQLOperators::genHaystackOpExpr(std::vector &tokens,IParser::Pos &token_pos,String kql_op, String ch_op, WildcardsPos wildcards_pos, WildcardsPos space_pos) +String KQLOperators::genHaystackOpExpr( + std::vector & tokens, IParser::Pos & token_pos, String kql_op, String ch_op, WildcardsPos wildcards_pos, WildcardsPos space_pos) { String new_expr, left_wildcards, right_wildcards, left_space, right_space; @@ -73,7 +276,7 @@ String KQLOperators::genHaystackOpExpr(std::vector &tokens,IParser::Pos break; case WildcardsPos::left: - left_wildcards ="%"; + left_wildcards = "%"; break; case WildcardsPos::right: @@ -81,7 +284,7 @@ String KQLOperators::genHaystackOpExpr(std::vector &tokens,IParser::Pos break; case WildcardsPos::both: - left_wildcards ="%"; + left_wildcards = "%"; right_wildcards = "%"; break; } @@ -92,7 +295,7 @@ String KQLOperators::genHaystackOpExpr(std::vector &tokens,IParser::Pos break; case WildcardsPos::left: - left_space =" "; + left_space = " "; break; case WildcardsPos::right: @@ -100,7 +303,7 @@ String KQLOperators::genHaystackOpExpr(std::vector &tokens,IParser::Pos break; case WildcardsPos::both: - left_space =" "; + left_space = " "; right_space = " "; break; } @@ -108,11 +311,13 @@ String KQLOperators::genHaystackOpExpr(std::vector &tokens,IParser::Pos ++token_pos; if (!tokens.empty() && ((token_pos)->type == TokenType::StringLiteral || token_pos->type == TokenType::QuotedIdentifier)) - new_expr = ch_op +"(" + tokens.back() +", '"+left_wildcards + left_space + String(token_pos->begin + 1,token_pos->end - 1) + right_space + right_wildcards + "')"; + new_expr = ch_op + "(" + tokens.back() + ", '" + left_wildcards + left_space + String(token_pos->begin + 1, token_pos->end - 1) + + right_space + right_wildcards + "')"; else if (!tokens.empty() && ((token_pos)->type == TokenType::BareWord)) { - auto tmp_arg = String(token_pos->begin, token_pos->end); - new_expr = ch_op +"(" + tokens.back() +", concat('" + left_wildcards + left_space + "', " + tmp_arg +", '"+ right_space + right_wildcards + "'))"; + auto tmp_arg = IParserKQLFunction::getExpression(token_pos); + new_expr = ch_op + "(" + tokens.back() + ", concat('" + left_wildcards + left_space + "', " + tmp_arg + ", '" + right_space + + right_wildcards + "'))"; } else throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op); @@ -120,7 +325,7 @@ String KQLOperators::genHaystackOpExpr(std::vector &tokens,IParser::Pos return new_expr; } -bool KQLOperators::convert(std::vector &tokens,IParser::Pos &pos) +bool KQLOperators::convert(std::vector & tokens, IParser::Pos & pos) { auto begin = pos; @@ -128,7 +333,7 @@ bool KQLOperators::convert(std::vector &tokens,IParser::Pos &pos) { KQLOperatorValue op_value = KQLOperatorValue::none; - auto token = String(pos->begin,pos->end); + auto token = String(pos->begin, pos->end); String op = token; if (token == "!") @@ -136,15 +341,15 @@ bool KQLOperators::convert(std::vector &tokens,IParser::Pos &pos) ++pos; if (pos->isEnd() || pos->type == TokenType::PipeMark || pos->type == TokenType::Semicolon) throw Exception(ErrorCodes::SYNTAX_ERROR, "Invalid negative operator"); - op ="!"+String(pos->begin,pos->end); + op = "!" + String(pos->begin, pos->end); } else if (token == "matches") { ++pos; if (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon) { - if (String(pos->begin,pos->end) == "regex") - op +=" regex"; + if (String(pos->begin, pos->end) == "regex") + op += " regex"; else --pos; } @@ -157,8 +362,8 @@ bool KQLOperators::convert(std::vector &tokens,IParser::Pos &pos) ++pos; if (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon) { - if (String(pos->begin,pos->end) == "~") - op +="~"; + if (String(pos->begin, pos->end) == "~") + op += "~"; else --pos; } @@ -171,7 +376,7 @@ bool KQLOperators::convert(std::vector &tokens,IParser::Pos &pos) return false; } - op_value = KQLOperator[op]; + op_value = KQLOperator.at(op); String new_expr; @@ -179,172 +384,179 @@ bool KQLOperators::convert(std::vector &tokens,IParser::Pos &pos) tokens.push_back(op); else { + if (tokens.empty()) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", op); + auto last_op = tokens.back(); auto last_pos = pos; switch (op_value) { - case KQLOperatorValue::contains: - new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::both); - break; + case KQLOperatorValue::contains: + new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::both); + break; - case KQLOperatorValue::not_contains: - new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::both); - break; + case KQLOperatorValue::not_contains: + new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::both); + break; - case KQLOperatorValue::contains_cs: - new_expr = genHaystackOpExpr(tokens, pos, op, "like", WildcardsPos::both); - break; + case KQLOperatorValue::contains_cs: + new_expr = genHaystackOpExpr(tokens, pos, op, "like", WildcardsPos::both); + break; - case KQLOperatorValue::not_contains_cs: - new_expr = genHaystackOpExpr(tokens, pos, op, "not like", WildcardsPos::both); - break; + case KQLOperatorValue::not_contains_cs: + new_expr = genHaystackOpExpr(tokens, pos, op, "not like", WildcardsPos::both); + break; - case KQLOperatorValue::endswith: - new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::left); - break; + case KQLOperatorValue::endswith: + new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::left); + break; - case KQLOperatorValue::not_endswith: - new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::left); - break; + case KQLOperatorValue::not_endswith: + new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::left); + break; - case KQLOperatorValue::endswith_cs: - new_expr = genHaystackOpExpr(tokens, pos, op, "endsWith", WildcardsPos::none); - break; + case KQLOperatorValue::endswith_cs: + new_expr = genHaystackOpExpr(tokens, pos, op, "endsWith", WildcardsPos::none); + break; - case KQLOperatorValue::not_endswith_cs: - new_expr = genHaystackOpExpr(tokens, pos, op, "not endsWith", WildcardsPos::none); - break; + case KQLOperatorValue::not_endswith_cs: + new_expr = genHaystackOpExpr(tokens, pos, op, "not endsWith", WildcardsPos::none); + break; - case KQLOperatorValue::equal: - break; + case KQLOperatorValue::equal: + new_expr = genEqOpExprCis(tokens, pos, "=="); + break; - case KQLOperatorValue::not_equal: - break; + case KQLOperatorValue::not_equal: + new_expr = genEqOpExprCis(tokens, pos, "!="); + break; - case KQLOperatorValue::equal_cs: - new_expr = "=="; - break; + case KQLOperatorValue::equal_cs: + new_expr = "=="; + break; - case KQLOperatorValue::not_equal_cs: - new_expr = "!="; - break; - case KQLOperatorValue::has: - new_expr = genHaystackOpExpr(tokens, pos, op, "hasTokenCaseInsensitive", WildcardsPos::none); - break; + case KQLOperatorValue::not_equal_cs: + new_expr = "!="; + break; + case KQLOperatorValue::has: + new_expr = genHaystackOpExpr(tokens, pos, op, "hasTokenCaseInsensitive", WildcardsPos::none); + break; - case KQLOperatorValue::not_has: - new_expr = genHaystackOpExpr(tokens, pos, op, "not hasTokenCaseInsensitive", WildcardsPos::none); - break; + case KQLOperatorValue::not_has: + new_expr = genHaystackOpExpr(tokens, pos, op, "not hasTokenCaseInsensitive", WildcardsPos::none); + break; - case KQLOperatorValue::has_all: - new_expr = genHasAnyAllOpExpr(tokens, pos, "has_all", "hasTokenCaseInsensitive"); - break; + case KQLOperatorValue::has_all: + new_expr = genHasAnyAllOpExpr(tokens, pos, "has_all", "hasTokenCaseInsensitive"); + break; - case KQLOperatorValue::has_any: - new_expr = genHasAnyAllOpExpr(tokens, pos, "has_any", "hasTokenCaseInsensitive"); - break; + case KQLOperatorValue::has_any: + new_expr = genHasAnyAllOpExpr(tokens, pos, "has_any", "hasTokenCaseInsensitive"); + break; - case KQLOperatorValue::has_cs: - new_expr = genHaystackOpExpr(tokens, pos, op, "hasToken", WildcardsPos::none); - break; + case KQLOperatorValue::has_cs: + new_expr = genHaystackOpExpr(tokens, pos, op, "hasToken", WildcardsPos::none); + break; - case KQLOperatorValue::not_has_cs: - new_expr = genHaystackOpExpr(tokens, pos, op, "not hasToken", WildcardsPos::none); - break; + case KQLOperatorValue::not_has_cs: + new_expr = genHaystackOpExpr(tokens, pos, op, "not hasToken", WildcardsPos::none); + break; - case KQLOperatorValue::hasprefix: - new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::right); - new_expr += " or "; - tokens.push_back(last_op); - new_expr += genHaystackOpExpr(tokens, last_pos, op, "ilike", WildcardsPos::both, WildcardsPos::left); - break; + case KQLOperatorValue::hasprefix: + new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::right); + new_expr += " or "; + tokens.push_back(last_op); + new_expr += genHaystackOpExpr(tokens, last_pos, op, "ilike", WildcardsPos::both, WildcardsPos::left); + break; - case KQLOperatorValue::not_hasprefix: - new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::right); - new_expr += " and "; - tokens.push_back(last_op); - new_expr += genHaystackOpExpr(tokens, last_pos, op, "not ilike", WildcardsPos::both, WildcardsPos::left); - break; + case KQLOperatorValue::not_hasprefix: + new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::right); + new_expr += " and "; + tokens.push_back(last_op); + new_expr += genHaystackOpExpr(tokens, last_pos, op, "not ilike", WildcardsPos::both, WildcardsPos::left); + break; - case KQLOperatorValue::hasprefix_cs: - new_expr = genHaystackOpExpr(tokens, pos, op, "startsWith", WildcardsPos::none); - new_expr += " or "; - tokens.push_back(last_op); - new_expr += genHaystackOpExpr(tokens, last_pos, op, "like", WildcardsPos::both, WildcardsPos::left); - break; + case KQLOperatorValue::hasprefix_cs: + new_expr = genHaystackOpExpr(tokens, pos, op, "startsWith", WildcardsPos::none); + new_expr += " or "; + tokens.push_back(last_op); + new_expr += genHaystackOpExpr(tokens, last_pos, op, "like", WildcardsPos::both, WildcardsPos::left); + break; - case KQLOperatorValue::not_hasprefix_cs: - new_expr = genHaystackOpExpr(tokens, pos, op, "not startsWith", WildcardsPos::none); - new_expr += " and "; - tokens.push_back(last_op); - new_expr += genHaystackOpExpr(tokens, last_pos, op, "not like", WildcardsPos::both, WildcardsPos::left); - break; + case KQLOperatorValue::not_hasprefix_cs: + new_expr = genHaystackOpExpr(tokens, pos, op, "not startsWith", WildcardsPos::none); + new_expr += " and "; + tokens.push_back(last_op); + new_expr += genHaystackOpExpr(tokens, last_pos, op, "not like", WildcardsPos::both, WildcardsPos::left); + break; - case KQLOperatorValue::hassuffix: - new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::left); - new_expr += " or "; - tokens.push_back(last_op); - new_expr += genHaystackOpExpr(tokens, last_pos, op, "ilike", WildcardsPos::both, WildcardsPos::right); - break; + case KQLOperatorValue::hassuffix: + new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::left); + new_expr += " or "; + tokens.push_back(last_op); + new_expr += genHaystackOpExpr(tokens, last_pos, op, "ilike", WildcardsPos::both, WildcardsPos::right); + break; - case KQLOperatorValue::not_hassuffix: - new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::left); - new_expr += " and "; - tokens.push_back(last_op); - new_expr += genHaystackOpExpr(tokens, last_pos, op, "not ilike", WildcardsPos::both, WildcardsPos::right); - break; + case KQLOperatorValue::not_hassuffix: + new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::left); + new_expr += " and "; + tokens.push_back(last_op); + new_expr += genHaystackOpExpr(tokens, last_pos, op, "not ilike", WildcardsPos::both, WildcardsPos::right); + break; - case KQLOperatorValue::hassuffix_cs: - new_expr = genHaystackOpExpr(tokens, pos, op, "endsWith", WildcardsPos::none); - new_expr += " or "; - tokens.push_back(last_op); - new_expr += genHaystackOpExpr(tokens, last_pos, op, "like", WildcardsPos::both, WildcardsPos::right); - break; + case KQLOperatorValue::hassuffix_cs: + new_expr = genHaystackOpExpr(tokens, pos, op, "endsWith", WildcardsPos::none); + new_expr += " or "; + tokens.push_back(last_op); + new_expr += genHaystackOpExpr(tokens, last_pos, op, "like", WildcardsPos::both, WildcardsPos::right); + break; - case KQLOperatorValue::not_hassuffix_cs: - new_expr = genHaystackOpExpr(tokens, pos, op, "not endsWith", WildcardsPos::none); - new_expr += " and "; - tokens.push_back(last_op); - new_expr += genHaystackOpExpr(tokens, last_pos, op, "not like", WildcardsPos::both, WildcardsPos::right); - break; + case KQLOperatorValue::not_hassuffix_cs: + new_expr = genHaystackOpExpr(tokens, pos, op, "not endsWith", WildcardsPos::none); + new_expr += " and "; + tokens.push_back(last_op); + new_expr += genHaystackOpExpr(tokens, last_pos, op, "not like", WildcardsPos::both, WildcardsPos::right); + break; - case KQLOperatorValue::in_cs: - new_expr = genInOpExpr(pos,op,"in"); - break; + case KQLOperatorValue::in_cs: + new_expr = genInOpExpr(pos, op, "in"); + break; - case KQLOperatorValue::not_in_cs: - new_expr = genInOpExpr(pos,op,"not in"); - break; + case KQLOperatorValue::not_in_cs: + new_expr = genInOpExpr(pos, op, "not in"); + break; - case KQLOperatorValue::in: - break; + case KQLOperatorValue::in: + new_expr = genInOpExprCis(tokens, pos, op, "in"); + break; - case KQLOperatorValue::not_in: - break; + case KQLOperatorValue::not_in: + new_expr = genInOpExprCis(tokens, pos, op, "not in"); + break; - case KQLOperatorValue::matches_regex: - new_expr = genHaystackOpExpr(tokens, pos, op, "match", WildcardsPos::none); - break; + case KQLOperatorValue::matches_regex: + new_expr = genHaystackOpExpr(tokens, pos, op, "match", WildcardsPos::none); + break; - case KQLOperatorValue::startswith: - new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::right); - break; + case KQLOperatorValue::startswith: + new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::right); + break; - case KQLOperatorValue::not_startswith: - new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::right); - break; + case KQLOperatorValue::not_startswith: + new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::right); + break; - case KQLOperatorValue::startswith_cs: - new_expr = genHaystackOpExpr(tokens, pos, op, "startsWith", WildcardsPos::none); - break; + case KQLOperatorValue::startswith_cs: + new_expr = genHaystackOpExpr(tokens, pos, op, "startsWith", WildcardsPos::none); + break; - case KQLOperatorValue::not_startswith_cs: - new_expr = genHaystackOpExpr(tokens, pos, op, "not startsWith", WildcardsPos::none); - break; + case KQLOperatorValue::not_startswith_cs: + new_expr = genHaystackOpExpr(tokens, pos, op, "not startsWith", WildcardsPos::none); + break; - default: - break; + default: + break; } tokens.push_back(new_expr); @@ -356,4 +568,3 @@ bool KQLOperators::convert(std::vector &tokens,IParser::Pos &pos) } } - diff --git a/src/Parsers/Kusto/ParserKQLOperators.h b/src/Parsers/Kusto/ParserKQLOperators.h index 72e25cc3cf9..2b11976ef7b 100644 --- a/src/Parsers/Kusto/ParserKQLOperators.h +++ b/src/Parsers/Kusto/ParserKQLOperators.h @@ -1,18 +1,18 @@ #pragma once +#include #include #include -#include namespace DB { class KQLOperators { public: - bool convert(std::vector &tokens,IParser::Pos &pos); -protected: + static bool convert(std::vector & tokens, IParser::Pos & pos); - enum class WildcardsPos:uint8_t +protected: + enum class WildcardsPos : uint8_t { none, left, @@ -20,87 +20,14 @@ protected: both }; - enum class KQLOperatorValue : uint16_t - { - none, - contains, - not_contains, - contains_cs, - not_contains_cs, - endswith, - not_endswith, - endswith_cs, - not_endswith_cs, - equal, /// =~ - not_equal, /// !~ - equal_cs, /// = - not_equal_cs, /// != - has, - not_has, - has_all, - has_any, - has_cs, - not_has_cs, - hasprefix, - not_hasprefix, - hasprefix_cs, - not_hasprefix_cs, - hassuffix, - not_hassuffix, - hassuffix_cs, - not_hassuffix_cs, - in_cs, /// in - not_in_cs, /// !in - in, /// in~ - not_in, /// !in~ - matches_regex, - startswith, - not_startswith, - startswith_cs, - not_startswith_cs, - }; - - std::unordered_map KQLOperator = - { - {"contains" , KQLOperatorValue::contains}, - {"!contains" , KQLOperatorValue::not_contains}, - {"contains_cs" , KQLOperatorValue::contains_cs}, - {"!contains_cs" , KQLOperatorValue::not_contains_cs}, - {"endswith" , KQLOperatorValue::endswith}, - {"!endswith" , KQLOperatorValue::not_endswith}, - {"endswith_cs" , KQLOperatorValue::endswith_cs}, - {"!endswith_cs" , KQLOperatorValue::not_endswith_cs}, - {"=~" , KQLOperatorValue::equal}, - {"!~" , KQLOperatorValue::not_equal}, - {"==" , KQLOperatorValue::equal_cs}, - {"!=" , KQLOperatorValue::not_equal_cs}, - {"has" , KQLOperatorValue::has}, - {"!has" , KQLOperatorValue::not_has}, - {"has_all" , KQLOperatorValue::has_all}, - {"has_any" , KQLOperatorValue::has_any}, - {"has_cs" , KQLOperatorValue::has_cs}, - {"!has_cs" , KQLOperatorValue::not_has_cs}, - {"hasprefix" , KQLOperatorValue::hasprefix}, - {"!hasprefix" , KQLOperatorValue::not_hasprefix}, - {"hasprefix_cs" , KQLOperatorValue::hasprefix_cs}, - {"!hasprefix_cs" , KQLOperatorValue::not_hasprefix_cs}, - {"hassuffix" , KQLOperatorValue::hassuffix}, - {"!hassuffix" , KQLOperatorValue::not_hassuffix}, - {"hassuffix_cs" , KQLOperatorValue::hassuffix_cs}, - {"!hassuffix_cs" , KQLOperatorValue::not_hassuffix_cs}, - {"in" , KQLOperatorValue::in_cs}, - {"!in" , KQLOperatorValue::not_in_cs}, - {"in~" , KQLOperatorValue::in}, - {"!in~" , KQLOperatorValue::not_in}, - {"matches regex" , KQLOperatorValue::matches_regex}, - {"startswith" , KQLOperatorValue::startswith}, - {"!startswith" , KQLOperatorValue::not_startswith}, - {"startswith_cs" , KQLOperatorValue::startswith_cs}, - {"!startswith_cs" , KQLOperatorValue::not_startswith_cs}, - }; - static String genHaystackOpExpr(std::vector &tokens,IParser::Pos &token_pos,String kql_op, String ch_op, WildcardsPos wildcards_pos, WildcardsPos space_pos = WildcardsPos::none); - static String genInOpExpr(IParser::Pos &token_pos,String kql_op, String ch_op); - static String genHasAnyAllOpExpr(std::vector &tokens,IParser::Pos &token_pos,String kql_op, String ch_op); + static String genHaystackOpExpr( + std::vector & tokens, + IParser::Pos & token_pos, + String kql_op, + String ch_op, + WildcardsPos wildcards_pos, + WildcardsPos space_pos = WildcardsPos::none); + static String genHasAnyAllOpExpr(std::vector & tokens, IParser::Pos & token_pos, String kql_op, String ch_op); }; } diff --git a/src/Parsers/Kusto/ParserKQLPrint.cpp b/src/Parsers/Kusto/ParserKQLPrint.cpp new file mode 100644 index 00000000000..bd9980ea96d --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLPrint.cpp @@ -0,0 +1,22 @@ +#include +#include +#include +namespace DB +{ + +bool ParserKQLPrint::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr select_expression_list; + const String expr = getExprFromToken(pos); + + Tokens tokens(expr.c_str(), expr.c_str() + expr.size()); + IParser::Pos new_pos(tokens, pos.max_depth); + + if (!ParserNotEmptyExpressionList(true).parse(new_pos, select_expression_list, expected)) + return false; + node->as()->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_expression_list)); + + return true; +} + +} diff --git a/src/Parsers/Kusto/ParserKQLPrint.h b/src/Parsers/Kusto/ParserKQLPrint.h new file mode 100644 index 00000000000..c962d7fa428 --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLPrint.h @@ -0,0 +1,16 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ParserKQLPrint : public ParserKQLBase +{ +protected: + const char * getName() const override { return "KQL project"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/Kusto/ParserKQLProject.cpp b/src/Parsers/Kusto/ParserKQLProject.cpp index e978323d821..fdc458b7707 100644 --- a/src/Parsers/Kusto/ParserKQLProject.cpp +++ b/src/Parsers/Kusto/ParserKQLProject.cpp @@ -1,20 +1,20 @@ #include -#include #include +#include namespace DB { -bool ParserKQLProject :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +bool ParserKQLProject ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr select_expression_list; String expr; expr = getExprFromToken(pos); - Tokens tokens(expr.c_str(), expr.c_str()+expr.size()); + Tokens tokens(expr.c_str(), expr.c_str() + expr.size()); IParser::Pos new_pos(tokens, pos.max_depth); - if (!ParserNotEmptyExpressionList(true).parse(new_pos, select_expression_list, expected)) + if (!ParserNotEmptyExpressionList(false).parse(new_pos, select_expression_list, expected)) return false; node->as()->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_expression_list)); diff --git a/src/Parsers/Kusto/ParserKQLQuery.cpp b/src/Parsers/Kusto/ParserKQLQuery.cpp index 04ee36705a9..47986943662 100644 --- a/src/Parsers/Kusto/ParserKQLQuery.cpp +++ b/src/Parsers/Kusto/ParserKQLQuery.cpp @@ -1,24 +1,124 @@ #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB { -String ParserKQLBase :: getExprFromToken(const String & text, const uint32_t & max_depth) +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} + +bool ParserKQLBase::parseByString(const String expr, ASTPtr & node, const uint32_t max_depth) +{ + Expected expected; + + Tokens tokens(expr.c_str(), expr.c_str() + expr.size()); + IParser::Pos pos(tokens, max_depth); + return parse(pos, node, expected); +} + +bool ParserKQLBase::parseSQLQueryByString(ParserPtr && parser, String & query, ASTPtr & select_node, int32_t max_depth) +{ + Expected expected; + Tokens token_subquery(query.c_str(), query.c_str() + query.size()); + IParser::Pos pos_subquery(token_subquery, max_depth); + if (!parser->parse(pos_subquery, select_node, expected)) + return false; + return true; +}; + +bool ParserKQLBase::setSubQuerySource(ASTPtr & select_query, ASTPtr & source, bool dest_is_subquery, bool src_is_subquery) +{ + ASTPtr table_expr; + if (!dest_is_subquery) + { + if (!select_query || !select_query->as()->tables() + || select_query->as()->tables()->as()->children.empty()) + return false; + table_expr = select_query->as()->tables()->as()->children[0]; + table_expr->as()->table_expression + = source->as()->tables()->children[0]->as()->table_expression; + return true; + } + + if (!select_query || select_query->as()->children.empty() + || !select_query->as()->children[0]->as()->table_expression + || select_query->as() + ->children[0] + ->as() + ->table_expression->as() + ->subquery->children.empty() + || select_query->as() + ->children[0] + ->as() + ->table_expression->as() + ->subquery->children[0] + ->as() + ->list_of_selects->children.empty() + || select_query->as() + ->children[0] + ->as() + ->table_expression->as() + ->subquery->children[0] + ->as() + ->list_of_selects->children[0] + ->as() + ->tables() + ->as() + ->children.empty()) + return false; + + table_expr = select_query->as() + ->children[0] + ->as() + ->table_expression->as() + ->subquery->children[0] + ->as() + ->list_of_selects->children[0] + ->as() + ->tables() + ->as() + ->children[0]; + + if (!src_is_subquery) + { + table_expr->as()->table_expression + = source->as()->tables()->children[0]->as()->table_expression; + } + else + { + table_expr->as()->table_expression + = source->children[0]->as()->table_expression; + } + + return true; +} + +String ParserKQLBase::getExprFromToken(const String & text, const uint32_t max_depth) { Tokens tokens(text.c_str(), text.c_str() + text.size()); IParser::Pos pos(tokens, max_depth); @@ -26,72 +126,179 @@ String ParserKQLBase :: getExprFromToken(const String & text, const uint32_t & m return getExprFromToken(pos); } -String ParserKQLBase :: getExprFromPipe(Pos & pos) +String ParserKQLBase::getExprFromPipe(Pos & pos) { - uint16_t bracket_count = 0; - auto begin = pos; + BracketCount bracket_count; auto end = pos; while (!end->isEnd() && end->type != TokenType::Semicolon) { - if (end->type == TokenType::OpeningRoundBracket) - ++bracket_count; - - if (end->type == TokenType::OpeningRoundBracket) - --bracket_count; - - if (end->type == TokenType::PipeMark && bracket_count == 0) + bracket_count.count(end); + if (end->type == TokenType::PipeMark && bracket_count.isZero()) break; ++end; } - --end; - return String(begin->begin, end->end); + if (end != pos) + --end; + return (pos <= end) ? String(pos->begin, end->end) : ""; } -String ParserKQLBase :: getExprFromToken(Pos & pos) +String ParserKQLBase::getExprFromToken(Pos & pos) { - String res; - std::vector tokens; - String alias; + std::vector comma_pos; + comma_pos.push_back(pos); - while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon) + size_t paren_count = 0; + while (!pos->isEnd() && pos->type != TokenType::Semicolon) { - String token = String(pos->begin,pos->end); + if (pos->type == TokenType::PipeMark && paren_count == 0) + break; - if (token == "=") + if (pos->type == TokenType::OpeningRoundBracket) + ++paren_count; + if (pos->type == TokenType::ClosingRoundBracket) + --paren_count; + + if (pos->type == TokenType::Comma && paren_count == 0) { ++pos; - if (String(pos->begin,pos->end) != "~") - { - alias = tokens.back(); - tokens.pop_back(); - } + comma_pos.push_back(pos); --pos; } - else if (!KQLOperators().convert(tokens,pos)) - { - tokens.push_back(token); - } - - if (pos->type == TokenType::Comma && !alias.empty()) - { - tokens.pop_back(); - tokens.push_back("AS"); - tokens.push_back(alias); - tokens.push_back(","); - alias.clear(); - } ++pos; } - if (!alias.empty()) + std::vector columns; + auto set_columns = [&](Pos & start_pos, Pos & end_pos) { - tokens.push_back("AS"); - tokens.push_back(alias); + bool has_alias = false; + auto equal_pos = start_pos; + auto columms_start_pos = start_pos; + auto it_pos = start_pos; + if (String(it_pos->begin, it_pos->end) == "=") + throw Exception(ErrorCodes::SYNTAX_ERROR, "Invalid equal symbol (=)"); + + BracketCount bracket_count; + while (it_pos < end_pos) + { + bracket_count.count(it_pos); + if (String(it_pos->begin, it_pos->end) == "=") + { + ++it_pos; + if (String(it_pos->begin, it_pos->end) != "~" && bracket_count.isZero()) + { + if (has_alias) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Invalid equal symbol (=)"); + has_alias = true; + } + + --it_pos; + if (equal_pos == start_pos) + equal_pos = it_pos; + } + ++it_pos; + } + + if (has_alias) + { + columms_start_pos = equal_pos; + ++columms_start_pos; + } + String column_str; + String function_name; + std::vector tokens; + + while (columms_start_pos < end_pos) + { + if (!KQLOperators::convert(tokens, columms_start_pos)) + { + if (columms_start_pos->type == TokenType::BareWord && function_name.empty()) + function_name = String(columms_start_pos->begin, columms_start_pos->end); + + auto expr = IParserKQLFunction::getExpression(columms_start_pos); + tokens.push_back(expr); + } + ++columms_start_pos; + } + + for (const auto & token : tokens) + column_str = column_str.empty() ? token : column_str + " " + token; + + if (has_alias) + { + --equal_pos; + if (start_pos == equal_pos) + { + String new_column_str; + if (start_pos->type != TokenType::BareWord) + throw Exception( + ErrorCodes::SYNTAX_ERROR, "{} is not a valid alias", std::string_view(start_pos->begin, start_pos->end)); + + if (function_name == "array_sort_asc" || function_name == "array_sort_desc") + new_column_str = std::format("{0}[1] AS {1}", column_str, String(start_pos->begin, start_pos->end)); + else + new_column_str = std::format("{0} AS {1}", column_str, String(start_pos->begin, start_pos->end)); + + columns.push_back(new_column_str); + } + else + { + String whole_alias(start_pos->begin, equal_pos->end); + + if (function_name != "array_sort_asc" && function_name != "array_sort_desc") + throw Exception(ErrorCodes::SYNTAX_ERROR, "{} is not a valid alias", whole_alias); + + if (start_pos->type != TokenType::OpeningRoundBracket && equal_pos->type != TokenType::ClosingRoundBracket) + throw Exception(ErrorCodes::SYNTAX_ERROR, "{} is not a valid alias for {}", whole_alias, function_name); + + String alias_inside; + bool comma_meet = false; + size_t index = 1; + ++start_pos; + while (start_pos < equal_pos) + { + if (start_pos->type == TokenType::Comma) + { + alias_inside.clear(); + if (comma_meet) + throw Exception(ErrorCodes::SYNTAX_ERROR, "{} has invalid alias for {}", whole_alias, function_name); + comma_meet = true; + } + else + { + if (!alias_inside.empty() || start_pos->type != TokenType::BareWord) + throw Exception(ErrorCodes::SYNTAX_ERROR, "{} has invalid alias for {}", whole_alias, function_name); + + alias_inside = String(start_pos->begin, start_pos->end); + auto new_column_str = std::format("{0}[{1}] AS {2}", column_str, index, alias_inside); + columns.push_back(new_column_str); + comma_meet = false; + ++index; + } + ++start_pos; + } + } + } + else + columns.push_back(column_str); + }; + + size_t cloumn_size = comma_pos.size(); + for (size_t i = 0; i < cloumn_size; ++i) + { + if (i == cloumn_size - 1) + set_columns(comma_pos[i], pos); + else + { + auto end_pos = comma_pos[i + 1]; + --end_pos; + set_columns(comma_pos[i], end_pos); + } } - for (auto const &token : tokens) - res = res.empty()? token : res +" " + token; + String res; + for (const auto & token : columns) + res = res.empty() ? token : res + "," + token; return res; } @@ -103,12 +310,22 @@ std::unique_ptr ParserKQLQuery::getOperator(String & op_name) return std::make_unique(); else if (op_name == "project") return std::make_unique(); + else if (op_name == "distinct") + return std::make_unique(); + else if (op_name == "extend") + return std::make_unique(); else if (op_name == "sort by" || op_name == "order by") return std::make_unique(); else if (op_name == "summarize") return std::make_unique(); else if (op_name == "table") return std::make_unique(); + else if (op_name == "make-series") + return std::make_unique(); + else if (op_name == "mv-expand") + return std::make_unique(); + else if (op_name == "print") + return std::make_unique(); else return nullptr; } @@ -127,49 +344,78 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) node = select_query; ASTPtr tables; - std::unordered_map kql_parser = - { - { "filter", {"filter", false, false, 3}}, - { "where", {"filter", false, false, 3}}, - { "limit", {"limit", false, true, 3}}, - { "take", {"limit", false, true, 3}}, - { "project", {"project", false, false, 3}}, - { "sort by", {"order by", false, false, 4}}, - { "order by", {"order by", false, false, 4}}, - { "table", {"table", false, false, 3}}, - { "summarize", {"summarize", true, true, 3}} - }; + std::unordered_map kql_parser + = {{"filter", {"filter", false, false, 3}}, + {"where", {"filter", false, false, 3}}, + {"limit", {"limit", false, true, 3}}, + {"take", {"limit", false, true, 3}}, + {"project", {"project", false, false, 3}}, + {"distinct", {"distinct", false, true, 3}}, + {"extend", {"extend", true, true, 3}}, + {"sort by", {"order by", false, false, 4}}, + {"order by", {"order by", false, false, 4}}, + {"table", {"table", false, false, 3}}, + {"print", {"print", false, true, 3}}, + {"summarize", {"summarize", true, true, 3}}, + {"make-series", {"make-series", true, true, 5}}, + {"mv-expand", {"mv-expand", true, true, 5}}}; std::vector> operation_pos; String table_name(pos->begin, pos->end); - operation_pos.push_back(std::make_pair("table", pos)); + if (table_name == "print") + operation_pos.emplace_back(table_name, pos); + else + operation_pos.emplace_back("table", pos); + ++pos; + uint16_t bracket_count = 0; while (!pos->isEnd() && pos->type != TokenType::Semicolon) { if (pos->type == TokenType::OpeningRoundBracket) ++bracket_count; - if (pos->type == TokenType::OpeningRoundBracket) + if (pos->type == TokenType::ClosingRoundBracket) --bracket_count; if (pos->type == TokenType::PipeMark && bracket_count == 0) { ++pos; String kql_operator(pos->begin, pos->end); - if (kql_operator == "order" || kql_operator == "sort") + + auto validate_kql_operator = [&] { - ++pos; - ParserKeyword s_by("by"); - if (s_by.ignore(pos,expected)) + if (kql_operator == "order" || kql_operator == "sort") { - kql_operator = "order by"; - --pos; + ++pos; + ParserKeyword s_by("by"); + if (s_by.ignore(pos, expected)) + { + kql_operator = "order by"; + --pos; + } } - } - if (pos->type != TokenType::BareWord || kql_parser.find(kql_operator) == kql_parser.end()) + else + { + auto op_pos_begin = pos; + ++pos; + ParserToken s_dash(TokenType::Minus); + if (s_dash.ignore(pos, expected)) + { + String tmp_op(op_pos_begin->begin, pos->end); + kql_operator = tmp_op; + } + else + --pos; + } + if (kql_parser.find(kql_operator) == kql_parser.end()) + return false; + return true; + }; + + if (!validate_kql_operator()) return false; ++pos; operation_pos.push_back(std::make_pair(kql_operator, pos)); @@ -190,16 +436,27 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (operation_pos.size() == 1) { - if (!kql_operator_p->parse(npos, node, expected)) - return false; + if (kql_operator_str == "print") + { + ++npos; + if (!ParserKQLPrint().parse(npos, node, expected)) + return false; + } + else if (kql_operator_str == "table") + { + if (!kql_operator_p->parse(npos, node, expected)) + return false; + } } else if (operation_pos.size() == 2 && operation_pos.front().first == "table") { - if (!kql_operator_p->parse(npos, node, expected)) - return false; npos = operation_pos.front().second; if (!ParserKQLTable().parse(npos, node, expected)) return false; + + npos = operation_pos.back().second; + if (!kql_operator_p->parse(npos, node, expected)) + return false; } else { @@ -207,7 +464,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) auto last_pos = operation_pos.back().second; auto last_op = operation_pos.back().first; - auto set_main_query_clause =[&](String & op, Pos & op_pos) + auto set_main_query_clause = [&](const String & op, Pos & op_pos) { auto op_str = ParserKQLBase::getExprFromPipe(op_pos); if (op == "project") @@ -224,12 +481,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) operation_pos.pop_back(); - if (kql_parser[last_op].need_input) - { - if (!kql_operator_p->parse(npos, node, expected)) - return false; - } - else + if (!kql_parser[last_op].need_input) { while (!operation_pos.empty()) { @@ -240,6 +492,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) break; if (!project_clause.empty() && prev_op == "project") break; + set_main_query_clause(prev_op, prev_pos); operation_pos.pop_back(); last_op = prev_op; @@ -249,7 +502,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!operation_pos.empty()) { - for (auto i = 0; i< kql_parser[last_op].backspace_steps; ++i) + for (auto i = 0; i < kql_parser[last_op].backspace_steps; ++i) --last_pos; String sub_query = std::format("({})", String(operation_pos.front().second->begin, last_pos->end)); @@ -266,7 +519,10 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - auto set_query_clasue =[&](String op_str, String op_calsue) + if (!kql_operator_p->parse(npos, node, expected)) + return false; + + auto set_query_clasue = [&](String op_str, String op_calsue) { auto oprator = getOperator(op_str); if (oprator) @@ -279,7 +535,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return true; }; - if (!select_query->select()) + if (!node->as()->select()) { if (project_clause.empty()) project_clause = "*"; @@ -301,23 +557,23 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return true; } - if (!select_query->select()) + if (!node->as()->select()) { auto expr = String("*"); - Tokens tokens(expr.c_str(), expr.c_str()+expr.size()); + Tokens tokens(expr.c_str(), expr.c_str() + expr.size()); IParser::Pos new_pos(tokens, pos.max_depth); if (!std::make_unique()->parse(new_pos, node, expected)) return false; } - return true; + return true; } bool ParserKQLSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr select_node; - if (!ParserKQLTaleFunction().parse(pos, select_node, expected)) + if (!ParserKQLTableFunction().parse(pos, select_node, expected)) return false; ASTPtr node_subquery = std::make_shared(); @@ -339,4 +595,46 @@ bool ParserKQLSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return true; } +bool ParserSimpleCHSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr sub_select_node; + ParserSelectWithUnionQuery select; + + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + if (!select.parse(pos, sub_select_node, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + if (parent_select_node && parent_select_node->as()->tables()) + { + auto select_query = sub_select_node->as()->list_of_selects->children[0]; + select_query->as()->setExpression( + ASTSelectQuery::Expression::TABLES, parent_select_node->as()->tables()); + } + + ASTPtr node_subquery = std::make_shared(); + node_subquery->children.push_back(sub_select_node); + + ASTPtr node_table_expr = std::make_shared(); + node_table_expr->as()->subquery = node_subquery; + + node_table_expr->children.emplace_back(node_subquery); + + ASTPtr node_table_in_select_query_emlement = std::make_shared(); + node_table_in_select_query_emlement->as()->table_expression = node_table_expr; + + ASTPtr res = std::make_shared(); + + res->children.emplace_back(node_table_in_select_query_emlement); + + node = res; + return true; +} + } diff --git a/src/Parsers/Kusto/ParserKQLQuery.h b/src/Parsers/Kusto/ParserKQLQuery.h index 42f5f84f031..516acb09dea 100644 --- a/src/Parsers/Kusto/ParserKQLQuery.h +++ b/src/Parsers/Kusto/ParserKQLQuery.h @@ -1,7 +1,7 @@ #pragma once -#include #include +#include namespace DB { @@ -9,24 +9,57 @@ class ParserKQLBase : public IParserBase { public: static String getExprFromToken(Pos & pos); + static String getExprFromToken(const String & text, const uint32_t max_depth); static String getExprFromPipe(Pos & pos); - static String getExprFromToken(const String & text, const uint32_t & max_depth); + static bool setSubQuerySource(ASTPtr & select_query, ASTPtr & source, bool dest_is_subquery, bool src_is_subquery); + static bool parseSQLQueryByString(ParserPtr && parser, String & query, ASTPtr & select_node, int32_t max_depth); + bool parseByString(const String expr, ASTPtr & node, const uint32_t max_depth); }; class ParserKQLQuery : public IParserBase { - protected: - static std::unique_ptr getOperator(String &op_name); + static std::unique_ptr getOperator(String & op_name); const char * getName() const override { return "KQL query"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; -class ParserKQLSubquery : public IParserBase +class ParserKQLSubquery : public ParserKQLBase { protected: const char * getName() const override { return "KQL subquery"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +class ParserSimpleCHSubquery : public ParserKQLBase +{ +public: + ParserSimpleCHSubquery(ASTPtr parent_select_node_ = nullptr) { parent_select_node = parent_select_node_; } + +protected: + const char * getName() const override { return "Simple ClickHouse subquery"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + ASTPtr parent_select_node; +}; + +class BracketCount +{ +public: + void count(IParser::Pos & pos) + { + if (pos->type == TokenType::OpeningRoundBracket) + ++round_bracket_count; + if (pos->type == TokenType::ClosingRoundBracket) + --round_bracket_count; + if (pos->type == TokenType::OpeningSquareBracket) + ++square_bracket_count; + if (pos->type == TokenType::ClosingSquareBracket) + --square_bracket_count; + } + bool isZero() const { return round_bracket_count == 0 && square_bracket_count == 0; } + +private: + int16_t round_bracket_count = 0; + int16_t square_bracket_count = 0; +}; } diff --git a/src/Parsers/Kusto/ParserKQLSort.cpp b/src/Parsers/Kusto/ParserKQLSort.cpp index ef4b84b17c7..193b75567aa 100644 --- a/src/Parsers/Kusto/ParserKQLSort.cpp +++ b/src/Parsers/Kusto/ParserKQLSort.cpp @@ -1,17 +1,17 @@ #include -#include -#include #include +#include +#include #include #include namespace DB { -bool ParserKQLSort :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +bool ParserKQLSort::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { bool has_dir = false; - std::vector has_directions; + std::vector has_directions; ParserOrderByExpressionList order_list; ASTPtr order_expression_list; @@ -35,6 +35,7 @@ bool ParserKQLSort :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected) has_directions.push_back(has_dir); has_dir = false; } + ++new_pos; } has_directions.push_back(has_dir); @@ -43,7 +44,7 @@ bool ParserKQLSort :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (!has_directions[i]) { - auto *order_expr = order_expression_list->children[i]->as(); + auto * order_expr = order_expression_list->children[i]->as(); order_expr->direction = -1; // default desc if (!order_expr->nulls_direction_was_explicitly_specified) order_expr->nulls_direction = -1; @@ -53,7 +54,6 @@ bool ParserKQLSort :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } node->as()->setExpression(ASTSelectQuery::Expression::ORDER_BY, std::move(order_expression_list)); - return true; } diff --git a/src/Parsers/Kusto/ParserKQLStatement.cpp b/src/Parsers/Kusto/ParserKQLStatement.cpp index 21e480234d3..f61e2e99ffb 100644 --- a/src/Parsers/Kusto/ParserKQLStatement.cpp +++ b/src/Parsers/Kusto/ParserKQLStatement.cpp @@ -1,10 +1,11 @@ -#include -#include #include #include +#include +#include +#include #include #include -#include +#include namespace DB { @@ -14,8 +15,7 @@ bool ParserKQLStatement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKQLWithOutput query_with_output_p(end, allow_settings_after_format_in_insert); ParserSetQuery set_p; - bool res = query_with_output_p.parse(pos, node, expected) - || set_p.parse(pos, node, expected); + bool res = query_with_output_p.parse(pos, node, expected) || set_p.parse(pos, node, expected); return res; } @@ -36,6 +36,7 @@ bool ParserKQLWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte bool ParserKQLWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { + // will support union next phase ASTPtr kql_query; if (!ParserKQLQuery().parse(pos, kql_query, expected)) @@ -58,33 +59,40 @@ bool ParserKQLWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & exp return true; } -bool ParserKQLTaleFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +bool ParserKQLTableFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKQLWithUnionQuery kql_p; ASTPtr select; ParserToken s_lparen(TokenType::OpeningRoundBracket); auto begin = pos; - auto paren_count = 0 ; + auto paren_count = 0; String kql_statement; if (s_lparen.ignore(pos, expected)) { - ++paren_count; - while (!pos->isEnd()) + if (pos->type == TokenType::HereDoc) { - if (pos->type == TokenType::ClosingRoundBracket) - --paren_count; - if (pos->type == TokenType::OpeningRoundBracket) - ++paren_count; - - if (paren_count == 0) - break; - - kql_statement = kql_statement + " " + String(pos->begin,pos->end); - ++pos; + kql_statement = String(pos->begin + 2, pos->end - 2); } + else + { + ++paren_count; + auto pos_start = pos; + while (!pos->isEnd()) + { + if (pos->type == TokenType::ClosingRoundBracket) + --paren_count; + if (pos->type == TokenType::OpeningRoundBracket) + ++paren_count; + if (paren_count == 0) + break; + ++pos; + } + kql_statement = String(pos_start->begin, (--pos)->end); + } + ++pos; Tokens token_kql(kql_statement.c_str(), kql_statement.c_str() + kql_statement.size()); IParser::Pos pos_kql(token_kql, pos.max_depth); @@ -95,8 +103,7 @@ bool ParserKQLTaleFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expec return true; } } - pos = begin; + pos = begin; return false; -}; - +} } diff --git a/src/Parsers/Kusto/ParserKQLStatement.h b/src/Parsers/Kusto/ParserKQLStatement.h index ef44b2d6c8a..fe9b9adfa2a 100644 --- a/src/Parsers/Kusto/ParserKQLStatement.h +++ b/src/Parsers/Kusto/ParserKQLStatement.h @@ -12,14 +12,14 @@ private: bool allow_settings_after_format_in_insert; const char * getName() const override { return "KQL Statement"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + public: explicit ParserKQLStatement(const char * end_, bool allow_settings_after_format_in_insert_ = false) - : end(end_) - , allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_) - {} + : end(end_), allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_) + { + } }; - class ParserKQLWithOutput : public IParserBase { protected: @@ -27,11 +27,12 @@ protected: bool allow_settings_after_format_in_insert; const char * getName() const override { return "KQL with output"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + public: explicit ParserKQLWithOutput(const char * end_, bool allow_settings_after_format_in_insert_ = false) - : end(end_) - , allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_) - {} + : end(end_), allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_) + { + } }; class ParserKQLWithUnionQuery : public IParserBase @@ -41,7 +42,7 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; -class ParserKQLTaleFunction : public IParserBase +class ParserKQLTableFunction : public IParserBase { protected: const char * getName() const override { return "KQL() function"; } @@ -49,4 +50,3 @@ protected: }; } - diff --git a/src/Parsers/Kusto/ParserKQLSummarize.cpp b/src/Parsers/Kusto/ParserKQLSummarize.cpp index 75eacb1adbd..11518fcc684 100644 --- a/src/Parsers/Kusto/ParserKQLSummarize.cpp +++ b/src/Parsers/Kusto/ParserKQLSummarize.cpp @@ -1,3 +1,8 @@ +#include +#include +#include +#include +#include #include #include #include @@ -10,15 +15,17 @@ #include #include #include +#include #include #include #include #include +#include namespace DB { -bool ParserKQLSummarize ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +bool ParserKQLSummarize::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr select_expression_list; ASTPtr group_expression_list; @@ -27,33 +34,164 @@ bool ParserKQLSummarize ::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte String expr_groupby; String expr_columns; bool groupby = false; + auto column_begin_pos = pos; - auto begin = pos; - auto pos_groupby = pos; + uint16_t bracket_count = 0; + int32_t new_column_index = 1; + + std::vector expr_aggregations; + std::vector expr_groupbys; + + std::unordered_set aggregate_functions( + {"arg_max", + "arg_min", + "avg", + "avgif", + "binary_all_and", + "binary_all_or", + "binary_all_xor", + "buildschema", + "count", + "countif", + "dcount", + "dcountif", + "make_bag", + "make_bag_if", + "make_list", + "make_list_if", + "make_list_with_nulls", + "make_set", + "make_set_if", + "max", + "maxif", + "min", + "minif", + "percentile", + "percentilew", + "percentiles", + "percentiles_array", + "percentilesw", + "percentilesw_array", + "stdev", + "stdevif", + "sum", + "sumif", + "take_any", + "take_anyif", + "variance", + "varianceif"}); + + auto apply_aliais = [&](Pos & begin_pos, Pos & end_pos, bool is_groupby) + { + auto expr = String(begin_pos->begin, end_pos->end); + auto equal_pos = begin_pos; + ++equal_pos; + if (!is_groupby) + { + if (String(equal_pos->begin, equal_pos->end) != "=") + { + String alias; + String aggregate_fun = String(begin_pos->begin, begin_pos->end); + if (aggregate_functions.find(aggregate_fun) == aggregate_functions.end()) + { + alias = std::format("Columns{}", new_column_index); + ++new_column_index; + } + else + { + alias = std::format("{}_", aggregate_fun); + auto agg_colum_pos = begin_pos; + ++agg_colum_pos; + ++agg_colum_pos; + ++agg_colum_pos; + if (agg_colum_pos->type == TokenType::Comma || agg_colum_pos->type == TokenType::ClosingRoundBracket) + { + --agg_colum_pos; + if (agg_colum_pos->type != TokenType::ClosingRoundBracket) + alias = alias + String(agg_colum_pos->begin, agg_colum_pos->end); + } + } + expr = std::format("{} = {}", alias, expr); + } + expr_aggregations.push_back(expr); + } + else + { + if (String(equal_pos->begin, equal_pos->end) != "=") + { + String groupby_fun = String(begin_pos->begin, begin_pos->end); + if (equal_pos->isEnd() || equal_pos->type == TokenType::Comma || equal_pos->type == TokenType::Semicolon + || equal_pos->type == TokenType::PipeMark) + { + expr = groupby_fun; + } + else + { + String alias; + if (groupby_fun == "bin" || groupby_fun == "bin_at") + { + auto bin_colum_pos = begin_pos; + ++bin_colum_pos; + ++bin_colum_pos; + alias = String(bin_colum_pos->begin, bin_colum_pos->end); + ++bin_colum_pos; + if (bin_colum_pos->type != TokenType::Comma) + alias.clear(); + } + if (alias.empty()) + { + alias = std::format("Columns{}", new_column_index); + ++new_column_index; + } + + expr = std::format("{} = {}", alias, expr); + } + } + expr_groupbys.push_back(expr); + } + }; while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon) { - if (String(pos->begin, pos->end) == "by") + if (pos->type == TokenType::OpeningRoundBracket) + ++bracket_count; + + if (pos->type == TokenType::ClosingRoundBracket) + --bracket_count; + + if ((bracket_count == 0 and pos->type == TokenType::Comma) || String(pos->begin, pos->end) == "by") { - groupby = true; - auto end = pos; - --end; - expr_aggregation = begin <= end ? String(begin->begin, end->end) : ""; - pos_groupby = pos; - ++pos_groupby; + auto end_pos = pos; + --end_pos; + apply_aliais(column_begin_pos, end_pos, groupby); + if (String(pos->begin, pos->end) == "by") + groupby = true; + column_begin_pos = pos; + ++column_begin_pos; } ++pos; } --pos; - if (groupby) - expr_groupby = String(pos_groupby->begin, pos->end); - else - expr_aggregation = begin <= pos ? String(begin->begin, pos->end) : ""; + apply_aliais(column_begin_pos, pos, groupby); - auto expr_aggregation_str = expr_aggregation.empty() ? "" : expr_aggregation +","; - expr_columns = groupby ? expr_aggregation_str + expr_groupby : expr_aggregation_str; + for (auto const & expr : expr_aggregations) + expr_aggregation = expr_aggregation.empty() ? expr : expr_aggregation + "," + expr; - String converted_columns = getExprFromToken(expr_columns, pos.max_depth); + for (auto const & expr : expr_groupbys) + expr_groupby = expr_groupby.empty() ? expr : expr_groupby + "," + expr; + + if (!expr_groupby.empty()) + expr_columns = expr_groupby; + + if (!expr_aggregation.empty()) + { + if (expr_columns.empty()) + expr_columns = expr_aggregation; + else + expr_columns = expr_columns + "," + expr_aggregation; + } + + String converted_columns = getExprFromToken(expr_columns, pos.max_depth); Tokens token_converted_columns(converted_columns.c_str(), converted_columns.c_str() + converted_columns.size()); IParser::Pos pos_converted_columns(token_converted_columns, pos.max_depth); @@ -65,7 +203,7 @@ bool ParserKQLSummarize ::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte if (groupby) { - String converted_groupby = getExprFromToken(expr_groupby, pos.max_depth); + String converted_groupby = getExprFromToken(expr_groupby, pos.max_depth); Tokens token_converted_groupby(converted_groupby.c_str(), converted_groupby.c_str() + converted_groupby.size()); IParser::Pos postoken_converted_groupby(token_converted_groupby, pos.max_depth); @@ -77,5 +215,4 @@ bool ParserKQLSummarize ::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte return true; } - } diff --git a/src/Parsers/Kusto/ParserKQLSummarize.h b/src/Parsers/Kusto/ParserKQLSummarize.h index 1aad02705df..e95043c1523 100644 --- a/src/Parsers/Kusto/ParserKQLSummarize.h +++ b/src/Parsers/Kusto/ParserKQLSummarize.h @@ -8,7 +8,6 @@ namespace DB class ParserKQLSummarize : public ParserKQLBase { - protected: const char * getName() const override { return "KQL summarize"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; diff --git a/src/Parsers/Kusto/ParserKQLTable.cpp b/src/Parsers/Kusto/ParserKQLTable.cpp index 6356ad688b6..4ea910ea8ee 100644 --- a/src/Parsers/Kusto/ParserKQLTable.cpp +++ b/src/Parsers/Kusto/ParserKQLTable.cpp @@ -1,45 +1,23 @@ +#include #include #include -#include #include #include -#include +#include namespace DB { -bool ParserKQLTable :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +bool ParserKQLTable ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - std::unordered_set sql_keywords - ({ - "SELECT", - "INSERT", - "CREATE", - "ALTER", - "SYSTEM", - "SHOW", - "GRANT", - "REVOKE", - "ATTACH", - "CHECK", - "DESCRIBE", - "DESC", - "DETACH", - "DROP", - "EXISTS", - "KILL", - "OPTIMIZE", - "RENAME", - "SET", - "TRUNCATE", - "USE", - "EXPLAIN" - }); + std::unordered_set sql_keywords({"SELECT", "INSERT", "CREATE", "ALTER", "SYSTEM", "SHOW", "GRANT", "REVOKE", + "ATTACH", "CHECK", "DESCRIBE", "DESC", "DETACH", "DROP", "EXISTS", "KILL", + "OPTIMIZE", "RENAME", "SET", "TRUNCATE", "USE", "EXPLAIN"}); ASTPtr tables; - String table_name(pos->begin,pos->end); + String table_name(pos->begin, pos->end); String table_name_upcase(table_name); - std::transform(table_name_upcase.begin(), table_name_upcase.end(),table_name_upcase.begin(), toupper); + std::transform(table_name_upcase.begin(), table_name_upcase.end(), table_name_upcase.begin(), toupper); if (sql_keywords.find(table_name_upcase) != sql_keywords.end()) return false; diff --git a/src/Parsers/Kusto/ParserKQLTable.h b/src/Parsers/Kusto/ParserKQLTable.h index c67dcb15156..8e03b50cb8b 100644 --- a/src/Parsers/Kusto/ParserKQLTable.h +++ b/src/Parsers/Kusto/ParserKQLTable.h @@ -8,7 +8,6 @@ namespace DB class ParserKQLTable : public ParserKQLBase { - protected: const char * getName() const override { return "KQL Table"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; diff --git a/src/Parsers/Kusto/Utilities.cpp b/src/Parsers/Kusto/Utilities.cpp new file mode 100644 index 00000000000..72c1ee86628 --- /dev/null +++ b/src/Parsers/Kusto/Utilities.cpp @@ -0,0 +1,74 @@ +#include "Utilities.h" + +#include "KustoFunctions/IParserKQLFunction.h" + +#include +#include +#include + +namespace DB +{ +String extractLiteralArgumentWithoutQuotes(const std::string & function_name, IParser::Pos & pos) +{ + ++pos; + if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral) + { + auto result = extractTokenWithoutQuotes(pos); + ++pos; + return result; + } + + --pos; + return IParserKQLFunction::getArgument(function_name, pos, IParserKQLFunction::ArgumentState::Raw); +} + +String extractTokenWithoutQuotes(IParser::Pos & pos) +{ + const auto offset = static_cast(pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral); + return {pos->begin + offset, pos->end - offset}; +} + +void setSelectAll(ASTSelectQuery & select_query) +{ + auto expression_list = std::make_shared(); + expression_list->children.push_back(std::make_shared()); + select_query.setExpression(ASTSelectQuery::Expression::SELECT, std::move(expression_list)); +} + +String wildcardToRegex(const String & wildcard) +{ + String regex; + for (char c : wildcard) + { + if (c == '*') + { + regex += ".*"; + } + else if (c == '?') + { + regex += "."; + } + else if (c == '.' || c == '+' || c == '(' || c == ')' || c == '[' || c == ']' || c == '\\' || c == '^' || c == '$') + { + regex += "\\"; + regex += c; + } + else + { + regex += c; + } + } + return regex; +} + +ASTPtr wrapInSelectWithUnion(const ASTPtr & select_query) +{ + auto select_with_union_query = std::make_shared(); + auto & list_of_selects = select_with_union_query->list_of_selects; + list_of_selects = std::make_shared(); + list_of_selects->children.push_back(select_query); + select_with_union_query->children.push_back(list_of_selects); + + return select_with_union_query; +} +} diff --git a/src/Parsers/Kusto/Utilities.h b/src/Parsers/Kusto/Utilities.h new file mode 100644 index 00000000000..f19fad72287 --- /dev/null +++ b/src/Parsers/Kusto/Utilities.h @@ -0,0 +1,13 @@ +#pragma once + +#include +#include + +namespace DB +{ +String extractLiteralArgumentWithoutQuotes(const std::string & function_name, IParser::Pos & pos); +String extractTokenWithoutQuotes(IParser::Pos & pos); +void setSelectAll(ASTSelectQuery & select_query); +String wildcardToRegex(const String & wildcard); +ASTPtr wrapInSelectWithUnion(const ASTPtr & select_query); +} diff --git a/src/Parsers/Kusto/parseKQLQuery.cpp b/src/Parsers/Kusto/parseKQLQuery.cpp new file mode 100644 index 00000000000..bcc04ef7001 --- /dev/null +++ b/src/Parsers/Kusto/parseKQLQuery.cpp @@ -0,0 +1,485 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} +namespace +{ + +/** From position in (possible multiline) query, get line number and column number in line. + * Used in syntax error message. + */ +std::pair getLineAndCol(const char * begin, const char * pos) +{ + size_t line = 0; + + const char * nl; + while ((nl = find_first_symbols<'\n'>(begin, pos)) < pos) + { + ++line; + begin = nl + 1; + } + + /// Lines numbered from 1. + return { line + 1, pos - begin + 1 }; +} + + +WriteBuffer & operator<< (WriteBuffer & out, const Expected & expected) +{ + if (expected.variants.empty()) + return out; + + if (expected.variants.size() == 1) + return out << *expected.variants.begin(); + + out << "one of: "; + bool first = true; + for (const auto & variant : expected.variants) + { + if (!first) + out << ", "; + first = false; + + out << variant; + } + return out; +} + + +/// Hilite place of syntax error. +void writeQueryWithHighlightedErrorPositions( + WriteBuffer & out, + const char * begin, + const char * end, + const Token * positions_to_hilite, /// must go in ascending order + size_t num_positions_to_hilite) +{ + const char * pos = begin; + for (size_t position_to_hilite_idx = 0; position_to_hilite_idx < num_positions_to_hilite; ++position_to_hilite_idx) + { + const char * current_position_to_hilite = positions_to_hilite[position_to_hilite_idx].begin; + + assert(current_position_to_hilite <= end); + assert(current_position_to_hilite >= begin); + + out.write(pos, current_position_to_hilite - pos); + + if (current_position_to_hilite == end) + { + out << "\033[41;1m \033[0m"; + return; + } + else + { + size_t bytes_to_hilite = UTF8::seqLength(*current_position_to_hilite); + + /// Bright on red background. + out << "\033[41;1m"; + out.write(current_position_to_hilite, bytes_to_hilite); + out << "\033[0m"; + pos = current_position_to_hilite + bytes_to_hilite; + } + } + out.write(pos, end - pos); +} + + +void writeQueryAroundTheError( + WriteBuffer & out, + const char * begin, + const char * end, + bool hilite, + const Token * positions_to_hilite, + size_t num_positions_to_hilite) +{ + if (hilite) + { + out << ":\n\n"; + writeQueryWithHighlightedErrorPositions(out, begin, end, positions_to_hilite, num_positions_to_hilite); + out << "\n\n"; + } + else + { + if (num_positions_to_hilite) + out << ": " << std::string(positions_to_hilite[0].begin, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, end - positions_to_hilite[0].begin)) << ". "; + } +} + + +void writeCommonErrorMessage( + WriteBuffer & out, + const char * begin, + const char * end, + Token last_token, + const std::string & query_description) +{ + out << "Syntax error"; + + if (!query_description.empty()) + out << " (" << query_description << ")"; + + out << ": failed at position " << (last_token.begin - begin + 1); + + if (last_token.type == TokenType::EndOfStream || last_token.type == TokenType::Semicolon) + { + out << " (end of query)"; + } + else + { + out << " ('" << std::string(last_token.begin, last_token.end - last_token.begin) << "')"; + } + + /// If query is multiline. + const char * nl = find_first_symbols<'\n'>(begin, end); + if (nl + 1 < end) + { + size_t line = 0; + size_t col = 0; + std::tie(line, col) = getLineAndCol(begin, last_token.begin); + + out << " (line " << line << ", col " << col << ")"; + } +} + + +std::string getSyntaxErrorMessage( + const char * begin, + const char * end, + Token last_token, + const Expected & expected, + bool hilite, + const std::string & query_description) +{ + WriteBufferFromOwnString out; + writeCommonErrorMessage(out, begin, end, last_token, query_description); + writeQueryAroundTheError(out, begin, end, hilite, &last_token, 1); + + if (!expected.variants.empty()) + out << "Expected " << expected; + + return out.str(); +} + + +std::string getLexicalErrorMessage( + const char * begin, + const char * end, + Token last_token, + bool hilite, + const std::string & query_description) +{ + WriteBufferFromOwnString out; + writeCommonErrorMessage(out, begin, end, last_token, query_description); + writeQueryAroundTheError(out, begin, end, hilite, &last_token, 1); + + out << getErrorTokenDescription(last_token.type); + if (last_token.size()) + { + out << ": '" << std::string_view{last_token.begin, last_token.size()} << "'"; + } + + return out.str(); +} + + +std::string getUnmatchedParenthesesErrorMessage( + const char * begin, + const char * end, + const UnmatchedParentheses & unmatched_parens, + bool hilite, + const std::string & query_description) +{ + WriteBufferFromOwnString out; + writeCommonErrorMessage(out, begin, end, unmatched_parens[0], query_description); + writeQueryAroundTheError(out, begin, end, hilite, unmatched_parens.data(), unmatched_parens.size()); + + out << "Unmatched parentheses: "; + for (const Token & paren : unmatched_parens) + out << *paren.begin; + + return out.str(); +} + +UnmatchedParentheses checkKQLUnmatchedParentheses(TokenIterator begin) +{ + std::unordered_set valid_kql_negative_suffix( + { + "between", + "contains", + "contains_cs", + "endswith", + "endswith_cs", + "~", + "=", + "has", + "has_cs", + "hasprefix", + "hasprefix_cs", + "hassuffix", + "hassuffix_cs", + "in", + "startswith", + "startswith_cs"}); + /// We have just two kind of parentheses: () and []. + UnmatchedParentheses stack; + + /// We have to iterate through all tokens until the end to avoid false positive "Unmatched parentheses" error + /// when parser failed in the middle of the query. + for (TokenIterator it = begin; !it->isEnd(); ++it) + { + if (!it.isValid()) // allow kql negative operators + { + if (it->type == TokenType::ErrorSingleExclamationMark) + { + ++it; + if (!valid_kql_negative_suffix.contains(String(it.get().begin, it.get().end))) + break; + --it; + } + else if (it->type == TokenType::ErrorWrongNumber) + { + if (!ParserKQLDateTypeTimespan().parseConstKQLTimespan(String(it.get().begin, it.get().end))) + break; + } + else + { + if (String(it.get().begin, it.get().end) == "~") + { + --it; + if (const auto prev = String(it.get().begin, it.get().end); prev != "!" && prev != "=" && prev != "in") + break; + ++it; + } + else + break; + } + } + + if (it->type == TokenType::OpeningRoundBracket || it->type == TokenType::OpeningSquareBracket) + { + stack.push_back(*it); + } + else if (it->type == TokenType::ClosingRoundBracket || it->type == TokenType::ClosingSquareBracket) + { + if (stack.empty()) + { + /// Excessive closing bracket. + stack.push_back(*it); + return stack; + } + else if ( + (stack.back().type == TokenType::OpeningRoundBracket && it->type == TokenType::ClosingRoundBracket) + || (stack.back().type == TokenType::OpeningSquareBracket && it->type == TokenType::ClosingSquareBracket)) + { + /// Valid match. + stack.pop_back(); + } + else + { + /// Closing bracket type doesn't match opening bracket type. + stack.push_back(*it); + return stack; + } + } + } + + /// If stack is not empty, we have unclosed brackets. + return stack; +} + +} + + +ASTPtr tryParseKQLQuery( + IParser & parser, + const char * & _out_query_end, /* also query begin as input parameter */ + const char * all_queries_end, + std::string & out_error_message, + bool hilite, + const std::string & query_description, + bool allow_multi_statements, + size_t max_query_size, + size_t max_parser_depth, + bool skip_insignificant) +{ + const char * query_begin = _out_query_end; + Tokens tokens(query_begin, all_queries_end, max_query_size, skip_insignificant); + /// NOTE: consider use UInt32 for max_parser_depth setting. + IParser::Pos token_iterator(tokens, static_cast(max_parser_depth)); + + if (token_iterator->isEnd() + || token_iterator->type == TokenType::Semicolon) + { + out_error_message = "Empty query"; + // Token iterator skips over comments, so we'll get this error for queries + // like this: + // " + // -- just a comment + // ; + //" + // Advance the position, so that we can use this parser for stream parsing + // even in presence of such queries. + _out_query_end = token_iterator->begin; + return nullptr; + } + + Expected expected; + ASTPtr res; + const bool parse_res = parser.parse(token_iterator, res, expected); + const auto last_token = token_iterator.max(); + _out_query_end = last_token.end; + + ASTInsertQuery * insert = nullptr; + if (parse_res) + { + if (auto * explain = res->as()) + { + if (auto explained_query = explain->getExplainedQuery()) + { + insert = explained_query->as(); + } + } + else + { + insert = res->as(); + } + } + + // If parsed query ends at data for insertion. Data for insertion could be + // in any format and not necessary be lexical correct, so we can't perform + // most of the checks. + if (insert && insert->data) + { + return res; + } + + // More granular checks for queries other than INSERT w/inline data. + /// Lexical error + if (last_token.isError()) + { + out_error_message = getLexicalErrorMessage(query_begin, all_queries_end, + last_token, hilite, query_description); + return nullptr; + } + + + /// Unmatched parentheses + UnmatchedParentheses unmatched_parens = checkKQLUnmatchedParentheses(TokenIterator(tokens)); + if (!unmatched_parens.empty()) + { + out_error_message = getUnmatchedParenthesesErrorMessage(query_begin, + all_queries_end, unmatched_parens, hilite, query_description); + return nullptr; + } + + if (!parse_res) + { + /// Generic parse error. + out_error_message = getSyntaxErrorMessage(query_begin, all_queries_end, + last_token, expected, hilite, query_description); + return nullptr; + } + + /// Excessive input after query. Parsed query must end with end of data or semicolon or data for INSERT. + if (!token_iterator->isEnd() + && token_iterator->type != TokenType::Semicolon) + { + expected.add(last_token.begin, "end of query"); + out_error_message = getSyntaxErrorMessage(query_begin, all_queries_end, + last_token, expected, hilite, query_description); + return nullptr; + } + + // Skip the semicolon that might be left after parsing the VALUES format. + while (token_iterator->type == TokenType::Semicolon) + { + ++token_iterator; + } + + // If multi-statements are not allowed, then after semicolon, there must + // be no non-space characters. + if (!allow_multi_statements + && !token_iterator->isEnd()) + { + out_error_message = getSyntaxErrorMessage(query_begin, all_queries_end, + last_token, {}, hilite, + (query_description.empty() ? std::string() : std::string(". ")) + + "Multi-statements are not allowed"); + return nullptr; + } + + return res; +} + + +ASTPtr parseKQLQueryAndMovePosition( + IParser & parser, + const char * & pos, + const char * end, + const std::string & query_description, + bool allow_multi_statements, + size_t max_query_size, + size_t max_parser_depth) +{ + std::string error_message; + ASTPtr res = tryParseKQLQuery(parser, pos, end, error_message, false, query_description, allow_multi_statements, max_query_size, max_parser_depth); + + if (res) + return res; + + throw Exception::createDeprecated(error_message, ErrorCodes::SYNTAX_ERROR); +} + +ASTPtr parseKQLQuery( + IParser & parser, + const char * begin, + const char * end, + const std::string & query_description, + size_t max_query_size, + size_t max_parser_depth) +{ + return parseKQLQueryAndMovePosition(parser, begin, end, query_description, false, max_query_size, max_parser_depth); +} + +ASTPtr parseKQLQuery( + IParser & parser, + const std::string & query, + const std::string & query_description, + size_t max_query_size, + size_t max_parser_depth) +{ + return parseKQLQuery(parser, query.data(), query.data() + query.size(), query_description, max_query_size, max_parser_depth); +} + +ASTPtr parseKQLQuery( + IParser & parser, + const std::string & query, + size_t max_query_size, + size_t max_parser_depth) +{ + return parseKQLQuery(parser, query.data(), query.data() + query.size(), parser.getName(), max_query_size, max_parser_depth); +} + +} diff --git a/src/Parsers/Kusto/parseKQLQuery.h b/src/Parsers/Kusto/parseKQLQuery.h new file mode 100644 index 00000000000..fca017e70fe --- /dev/null +++ b/src/Parsers/Kusto/parseKQLQuery.h @@ -0,0 +1,64 @@ +#pragma once + +#include +#include +#include +namespace DB +{ + +/** From position in (possible multiline) query, get line number and column number in line. + * Used in syntax error message. + */ + +} +namespace DB +{ + +class IParser; + +/// Parse query or set 'out_error_message'. +ASTPtr tryParseKQLQuery( + IParser & parser, + const char * & _out_query_end, // query start as input parameter, query end as output + const char * end, + std::string & out_error_message, + bool hilite, + const std::string & description, + bool allow_multi_statements, /// If false, check for non-space characters after semicolon and set error message if any. + size_t max_query_size, /// If (end - pos) > max_query_size and query is longer than max_query_size then throws "Max query size exceeded". + /// Disabled if zero. Is used in order to check query size if buffer can contains data for INSERT query. + size_t max_parser_depth, + bool skip_insignificant = true); /// If true, lexer will skip all insignificant tokens (e.g. whitespaces) + + +/// Parse query or throw an exception with error message. +ASTPtr parseKQLQueryAndMovePosition( + IParser & parser, + const char * & pos, /// Moved to end of parsed fragment. + const char * end, + const std::string & description, + bool allow_multi_statements, + size_t max_query_size, + size_t max_parser_depth); + +ASTPtr parseKQLQuery( + IParser & parser, + const char * begin, + const char * end, + const std::string & description, + size_t max_query_size, + size_t max_parser_depth); + +ASTPtr parseKQLQuery( + IParser & parser, + const std::string & query, + const std::string & query_description, + size_t max_query_size, + size_t max_parser_depth); + +ASTPtr parseKQLQuery( + IParser & parser, + const std::string & query, + size_t max_query_size, + size_t max_parser_depth); +} diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 2d3479934df..a26fdc1396b 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -451,7 +451,17 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & } break; } - + case Type::DROP_FORMAT_SCHEMA_CACHE: + { + if (ParserKeyword{"FOR"}.ignore(pos, expected)) + { + if (ParserKeyword{"Protobuf"}.ignore(pos, expected)) + res->schema_cache_format = "Protobuf"; + else + return false; + } + break; + } case Type::UNFREEZE: { ASTPtr ast; diff --git a/src/Parsers/tests/KQL/gtest_KQL_AggregateFunctions.cpp b/src/Parsers/tests/KQL/gtest_KQL_AggregateFunctions.cpp new file mode 100644 index 00000000000..f115f5c6b73 --- /dev/null +++ b/src/Parsers/tests/KQL/gtest_KQL_AggregateFunctions.cpp @@ -0,0 +1,73 @@ +#include +#include + +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_Aggregate, ParserKQLTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "Customers | summarize t = stdev(Age) by FirstName", + "SELECT\n FirstName,\n sqrt(varSamp(Age)) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "Customers | summarize t = stdevif(Age, Age < 10) by FirstName", + "SELECT\n FirstName,\n sqrt(varSampIf(Age, Age < 10)) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "Customers | summarize t = binary_all_and(Age) by FirstName", + "SELECT\n FirstName,\n groupBitAnd(Age) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "Customers | summarize t = binary_all_or(Age) by FirstName", + "SELECT\n FirstName,\n groupBitOr(Age) AS t\nFROM Customers\nGROUP BY FirstName" + + }, + { + "Customers | summarize t = binary_all_xor(Age) by FirstName", + "SELECT\n FirstName,\n groupBitXor(Age) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "Customers | summarize percentiles(Age, 30, 40, 50, 60, 70) by FirstName", + "SELECT\n FirstName,\n quantiles(30 / 100, 40 / 100, 50 / 100, 60 / 100, 70 / 100)(Age) AS percentiles_Age\nFROM Customers\nGROUP BY FirstName" + }, + { + "Customers | summarize t = percentiles_array(Age, 10, 20, 30, 50) by FirstName", + "SELECT\n FirstName,\n quantiles(10 / 100, 20 / 100, 30 / 100, 50 / 100)(Age) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "Customers | summarize t = percentiles_array(Age, dynamic([10, 20, 30, 50])) by FirstName", + "SELECT\n FirstName,\n quantiles(10 / 100, 20 / 100, 30 / 100, 50 / 100)(Age) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "DataTable | summarize t = percentilesw(Bucket, Frequency, 50, 75, 99.9)", + "SELECT quantilesExactWeighted(50 / 100, 75 / 100, 99.9 / 100)(Bucket, Frequency) AS t\nFROM DataTable" + }, + { + "DataTable| summarize t = percentilesw_array(Bucket, Frequency, dynamic([10, 50, 30]))", + "SELECT quantilesExactWeighted(10 / 100, 50 / 100, 30 / 100)(Bucket, Frequency) AS t\nFROM DataTable" + }, + { + "Customers | summarize t = percentile(Age, 50) by FirstName", + "SELECT\n FirstName,\n quantile(50 / 100)(Age) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "DataTable | summarize t = percentilew(Bucket, Frequency, 50)", + "SELECT quantileExactWeighted(50 / 100)(Bucket, Frequency) AS t\nFROM DataTable" + }, + { + "Customers | summarize t = make_list_with_nulls(Age) by FirstName", + "SELECT\n FirstName,\n arrayConcat(groupArray(Age), arrayMap(x -> NULL, range(0, toUInt32(count(*) - length(groupArray(Age))), 1))) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "Customers | summarize count() by bin(Age, 10)", + "SELECT\n toInt64(toFloat64(Age) / 10) * 10 AS Age,\n count() AS count_\nFROM Customers\nGROUP BY Age" + }, + { + "Customers | summarize count(Age+1) by bin(Age+1, 10)", + "SELECT\n toInt64(toFloat64(Age + 1) / 10) * 10 AS Columns1,\n count(Age + 1) AS count_\nFROM Customers\nGROUP BY Columns1" + }, + { + "Customers | summarize count(Age) by bin(Age, 10)", + "SELECT\n toInt64(toFloat64(Age) / 10) * 10 AS Age,\n count(Age) AS count_Age\nFROM Customers\nGROUP BY Age" + } +}))); diff --git a/src/Parsers/tests/KQL/gtest_KQL_Binary.cpp b/src/Parsers/tests/KQL/gtest_KQL_Binary.cpp new file mode 100644 index 00000000000..72c9c5a67eb --- /dev/null +++ b/src/Parsers/tests/KQL/gtest_KQL_Binary.cpp @@ -0,0 +1,37 @@ +#include + +#include + +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_Binary, ParserKQLTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "print binary_and(A, B)", + "SELECT bitAnd(CAST(A, 'Int64'), CAST(B, 'Int64'))" + }, + { + "print binary_not(A)", + "SELECT bitNot(CAST(A, 'Int64'))" + }, + { + "print binary_or(A, B)", + "SELECT bitOr(CAST(A, 'Int64'), CAST(B, 'Int64'))" + }, + { + "print binary_shift_left(A, B)", + "SELECT if(B < 0, NULL, bitShiftLeft(CAST(A, 'Int64'), B))" + }, + { + "print binary_shift_right(A, B)", + "SELECT if(B < 0, NULL, bitShiftRight(CAST(A, 'Int64'), B))" + }, + { + "print binary_xor(A, B)", + "SELECT bitXor(CAST(A, 'Int64'), CAST(B, 'Int64'))" + }, + { + "print bitset_count_ones(A)", + "SELECT bitCount(A)" + } +}))); diff --git a/src/Parsers/tests/KQL/gtest_KQL_Conversion.cpp b/src/Parsers/tests/KQL/gtest_KQL_Conversion.cpp new file mode 100644 index 00000000000..02e45455524 --- /dev/null +++ b/src/Parsers/tests/KQL/gtest_KQL_Conversion.cpp @@ -0,0 +1,80 @@ +#include + +#include + +INSTANTIATE_TEST_SUITE_P( + ParserKQLQuery_Conversion, + ParserKQLTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "print tobool(A)", + "SELECT multiIf(toString(A) = 'true', true, toString(A) = 'false', false, toInt64OrNull(toString(A)) != 0)" + }, + { + "print toboolean(A)", + "SELECT multiIf(toString(A) = 'true', true, toString(A) = 'false', false, toInt64OrNull(toString(A)) != 0)" + }, + { + "print todouble(A)", + "SELECT toFloat64OrNull(toString(A))" + }, + { + "print toint(A)", + "SELECT toInt32OrNull(toString(A))" + }, + { + "print tolong(A)", + "SELECT toInt64OrNull(toString(A))" + }, + { + "print toreal(A)", + "SELECT toFloat64OrNull(toString(A))" + }, + { + "print tostring(A)", + "SELECT ifNull(toString(A), '')" + }, + { + "print decimal(123.345)", + "SELECT toDecimal128(CAST('123.345', 'String'), 32)" + }, + { + "print decimal(NULL)", + "SELECT NULL" + }, + { + "print todecimal('123.45')", + "SELECT toDecimal128(CAST('123.45', 'String'), 32)" + }, + { + "print todecimal(NULL)", + "SELECT NULL" + }, + { + "print todecimal(123456.3456)", + "SELECT toDecimal128(CAST('123456.3456', 'String'), 12)" + }, + { + "print todecimal('abc')", + "SELECT NULL" + }, + { + "print todecimal('1e5')", + "SELECT toDecimal128(CAST('1e5', 'String'), 5)" + }, + { + "print decimal(1e-5)", + "SELECT toDecimal128(CAST('1e-5', 'String'), 5)" + }, + { + "print time(9nanoseconds)", + "SELECT CAST('9e-09', 'Float64')" + }, + { + "print time(1tick)", + "SELECT CAST('1e-07', 'Float64')" + } + +}))); diff --git a/src/Parsers/tests/KQL/gtest_KQL_DataType.cpp b/src/Parsers/tests/KQL/gtest_KQL_DataType.cpp new file mode 100644 index 00000000000..ed77bbbada7 --- /dev/null +++ b/src/Parsers/tests/KQL/gtest_KQL_DataType.cpp @@ -0,0 +1,65 @@ +#include + +#include + +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_DataType, ParserKQLTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "print dynamic(null)", + "SELECT NULL" + }, + { + "print dynamic(1)", + "SELECT 1" + }, + { + "print dynamic(datetime(1))", + "SELECT parseDateTime64BestEffortOrNull('1', 9, 'UTC')" + }, + { + "print dynamic(timespan(1d))", + "SELECT CAST('86400', 'Float64')" + }, + { + "print dynamic(parse_ipv4('127.0.0.1'))", + "throws" + }, + { + "print dynamic({ \"a\": 9 })", + "throws" + }, + { + "print dynamic([1, 2, 3])", + "SELECT [1, 2, 3]" + }, + { + "print dynamic([1, dynamic([2]), 3])", + "SELECT [1, [2], 3]" + }, + { + "print dynamic([[1], [2], [3]])", + "SELECT [[1], [2], [3]]" + }, + { + "print dynamic(['a', \"b\", 'c'])", + "SELECT ['a', 'b', 'c']" + }, + { + "print dynamic([1, 'a', true, false])", + "SELECT [1, 'a', true, false]" + }, + { + "print dynamic([date(1), time(1d), 1, 2])", + "SELECT [parseDateTime64BestEffortOrNull('1', 9, 'UTC'), CAST('86400', 'Float64'), 1, 2]" + }, + { + "print time('13:00:40.00000')", + "SELECT CAST('46840', 'Float64')" + }, + { + "print timespan('12.23:12:23');", + "SELECT CAST('1120343', 'Float64')" + } +}))); diff --git a/src/Parsers/tests/KQL/gtest_KQL_Distinct.cpp b/src/Parsers/tests/KQL/gtest_KQL_Distinct.cpp new file mode 100644 index 00000000000..12e8065328a --- /dev/null +++ b/src/Parsers/tests/KQL/gtest_KQL_Distinct.cpp @@ -0,0 +1,29 @@ +#include + +#include + +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_Distinct, ParserKQLTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "Customers | distinct *", + "SELECT DISTINCT *\nFROM Customers" + }, + { + "Customers | distinct Occupation", + "SELECT DISTINCT Occupation\nFROM Customers" + }, + { + "Customers | distinct Occupation, Education", + "SELECT DISTINCT\n Occupation,\n Education\nFROM Customers" + }, + { + "Customers |where Age <30| distinct Occupation, Education", + "SELECT DISTINCT\n Occupation,\n Education\nFROM Customers\nWHERE Age < 30" + }, + { + "Customers |where Age <30 | order by Age| distinct Occupation, Education", + "SELECT DISTINCT\n Occupation,\n Education\nFROM Customers\nWHERE Age < 30\nORDER BY Age DESC" + } +}))); diff --git a/src/Parsers/tests/KQL/gtest_KQL_Dynamic.cpp b/src/Parsers/tests/KQL/gtest_KQL_Dynamic.cpp new file mode 100644 index 00000000000..ab60223daf6 --- /dev/null +++ b/src/Parsers/tests/KQL/gtest_KQL_Dynamic.cpp @@ -0,0 +1,139 @@ +#include + +#include + +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_DynamicExactMatch, ParserKQLTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "print array_concat(A, B)", + "SELECT arrayConcat(A, B)" + }, + { + "print array_concat(A, B, C, D)", + "SELECT arrayConcat(A, B, C, D)" + }, + { + "print array_iff(A, B, C)", + "SELECT arrayMap(x -> multiIf(toTypeName(x.1) = 'String', NULL, toInt64(x.1) != 0, x.2, x.3), arrayZip(A, arrayResize(B, length(A), NULL), arrayResize(C, length(A), NULL)))" + }, + { + "print array_iif(A, B, C)", + "SELECT arrayMap(x -> multiIf(toTypeName(x.1) = 'String', NULL, toInt64(x.1) != 0, x.2, x.3), arrayZip(A, arrayResize(B, length(A), NULL), arrayResize(C, length(A), NULL)))" + }, + { + "print output = array_index_of(dynamic([1, 2, 3]), 2)", + "SELECT indexOf([1, 2, 3], 2) - 1 AS output" + }, + { + "print output = array_index_of(dynamic(['a', 'b', 'c']), 'b')", + "SELECT indexOf(['a', 'b', 'c'], 'b') - 1 AS output" + }, + { + "print output = array_index_of(dynamic(['John', 'Denver', 'Bob', 'Marley']), 'Marley')", + "SELECT indexOf(['John', 'Denver', 'Bob', 'Marley'], 'Marley') - 1 AS output" + }, + { + "print output = array_length(dynamic([1, 2, 3]))", + "SELECT length([1, 2, 3]) AS output" + }, + { + "print output = array_length(dynamic(['John', 'Denver', 'Bob', 'Marley']))", + "SELECT length(['John', 'Denver', 'Bob', 'Marley']) AS output" + }, + { + "print array_reverse(A)", + "SELECT if(throwIf(NOT startsWith(toTypeName(A), 'Array'), 'Only arrays are supported'), [], reverse(A))" + }, + { + "print array_rotate_left(A, B)", + "SELECT arrayMap(x -> (A[moduloOrZero((x + length(A)) + moduloOrZero(B, toInt64(length(A))), length(A)) + 1]), range(0, length(A)))" + }, + { + "print array_rotate_right(A, B)", + "SELECT arrayMap(x -> (A[moduloOrZero((x + length(A)) + moduloOrZero(-1 * B, toInt64(length(A))), length(A)) + 1]), range(0, length(A)))" + }, + { + "print output = array_sum(dynamic([2, 5, 3]))", + "SELECT arraySum([2, 5, 3]) AS output" + }, + { + "print output = array_sum(dynamic([2.5, 5.5, 3]))", + "SELECT arraySum([2.5, 5.5, 3]) AS output" + }, + { + "print jaccard_index(A, B)", + "SELECT length(arrayIntersect(A, B)) / length(arrayDistinct(arrayConcat(A, B)))" + }, + { + "print pack_array(A, B, C, D)", + "SELECT [A, B, C, D]" + }, + { + "print set_difference(A, B)", + "SELECT arrayFilter(x -> (NOT has(arrayDistinct(arrayConcat(B)), x)), arrayDistinct(A))" + }, + { + "print set_difference(A, B, C)", + "SELECT arrayFilter(x -> (NOT has(arrayDistinct(arrayConcat(B, C)), x)), arrayDistinct(A))" + }, + { + "print set_has_element(A, B)", + "SELECT has(A, B)" + }, + { + "print set_intersect(A, B)", + "SELECT arrayIntersect(A, B)" + }, + { + "print set_intersect(A, B, C)", + "SELECT arrayIntersect(A, B, C)" + }, + { + "print set_union(A, B)", + "SELECT arrayDistinct(arrayConcat(A, B))" + }, + { + "print set_union(A, B, C)", + "SELECT arrayDistinct(arrayConcat(A, B, C))" + } +}))); + +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_DynamicRegex, ParserRegexTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "print array_shift_left(A, B)", + R"(SELECT arrayResize\(if\(B > 0, arraySlice\(A, B \+ 1\), arrayConcat\(arrayWithConstant\(abs\(B\), fill_value_\d+\), A\)\), length\(A\), if\(\(NULL IS NULL\) AND \(\(extract\(toTypeName\(A\), 'Array\\\\\(\(\.\*\)\\\\\)*'\) AS element_type_\d+\) = 'String'\), defaultValueOfTypeName\(if\(element_type_\d+ = 'Nothing', 'Nullable\(Nothing\)', element_type_\d+\)\), NULL\) AS fill_value_\d+\))" + }, + { + "print array_shift_left(A, B, C)", + R"(SELECT arrayResize\(if\(B > 0, arraySlice\(A, B \+ 1\), arrayConcat\(arrayWithConstant\(abs\(B\), fill_value_\d+\), A\)\), length\(A\), if\(\(C IS NULL\) AND \(\(extract\(toTypeName\(A\), 'Array\\\\\(\(\.\*\)\\\\\)'\) AS element_type_\d+\) = 'String'\), defaultValueOfTypeName\(if\(element_type_\d+ = 'Nothing', 'Nullable\(Nothing\)', element_type_\d+\)\), C\) AS fill_value_\d+\))" + }, + { + "print array_shift_right(A, B)", + R"(SELECT arrayResize\(if\(\(-1 \* B\) > 0, arraySlice\(A, \(-1 \* B\) \+ 1\), arrayConcat\(arrayWithConstant\(abs\(-1 \* B\), fill_value_\d+\), A\)\), length\(A\), if\(\(NULL IS NULL\) AND \(\(extract\(toTypeName\(A\), 'Array\\\\\(\(\.\*\)\\\\\)'\) AS element_type_\d+\) = 'String'\), defaultValueOfTypeName\(if\(element_type_\d+ = 'Nothing', 'Nullable\(Nothing\)', element_type_\d+\)\), NULL\) AS fill_value_\d+\))" + }, + { + "print array_shift_right(A, B, C)", + R"(SELECT arrayResize\(if\(\(-1 \* B\) > 0, arraySlice\(A, \(-1 \* B\) \+ 1\), arrayConcat\(arrayWithConstant\(abs\(-1 \* B\), fill_value_\d+\), A\)\), length\(A\), if\(\(C IS NULL\) AND \(\(extract\(toTypeName\(A\), 'Array\\\\\(\(\.\*\)\\\\\)'\) AS element_type_\d+\) = 'String'\), defaultValueOfTypeName\(if\(element_type_\d+ = 'Nothing', 'Nullable\(Nothing\)', element_type_\d+\)\), C\) AS fill_value_\d+\))" + }, + { + "print array_slice(A, B, C)", + R"(SELECT arraySlice\(A, 1 \+ if\(B >= 0, B, arrayMax\(\[-length\(A\), B\]\) \+ length\(A\)\) AS offset_\d+, \(\(1 \+ if\(C >= 0, C, arrayMax\(\[-length\(A\), C\]\) \+ length\(A\)\)\) - offset_\d+\) \+ 1\))" + }, + { + "print array_split(A, B)", + R"(SELECT if\(empty\(arrayMap\(x -> if\(x >= 0, x, arrayMax\(\[0, x \+ CAST\(length\(A\), 'Int\d+'\)\]\)\), flatten\(\[B\]\)\) AS indices_\d+\), \[A\], arrayConcat\(\[arraySlice\(A, 1, indices_\d+\[1\]\)\], arrayMap\(i -> arraySlice\(A, \(indices_\d+\[i\]\) \+ 1, if\(i = length\(indices_\d+\), CAST\(length\(A\), 'Int\d+'\), CAST\(indices_\d+\[i \+ 1\], 'Int\d+'\)\) - \(indices_\d+\[i\]\)\), range\(1, length\(indices_\d+\) \+ 1\)\)\)\))" + }, + { + "print zip(A, B)", + R"(SELECT arrayMap\(t -> \[untuple\(t\)\], arrayZip\(arrayResize\(arg0_\d+, arrayMax\(\[length\(if\(match\(toTypeName\(A\), 'Array\\\\\(Nullable\\\\\(\.\*\\\\\)\\\\\)'\), A, CAST\(A, concat\('Array\(Nullable\(', extract\(toTypeName\(A\), 'Array\\\\\(\(\.\*\)\\\\\)'\), '\)\)'\)\)\) AS arg0_\d+\), length\(if\(match\(toTypeName\(B\), 'Array\\\\\(Nullable\\\\\(\.\*\\\\\)\\\\\)'\), B, CAST\(B, concat\('Array\(Nullable\(', extract\(toTypeName\(B\), 'Array\\\\\(\(\.\*\)\\\\\)'\), '\)\)'\)\)\) AS arg1_\d+\)\]\) AS max_length_\d+, NULL\), arrayResize\(arg1_\d+, max_length_\d+, NULL\)\)\))" + }, + { + "print zip(A, B, C)", + R"(SELECT arrayMap\(t -> \[untuple\(t\)\], arrayZip\(arrayResize\(arg0_\d+, arrayMax\(\[length\(if\(match\(toTypeName\(A\), 'Array\\\\\(Nullable\\\\\(\.\*\\\\\)\\\\\)'\), A, CAST\(A, concat\('Array\(Nullable\(', extract\(toTypeName\(A\), 'Array\\\\\(\(\.\*\)\\\\\)'\), '\)\)'\)\)\) AS arg0_\d+\), length\(if\(match\(toTypeName\(B\), 'Array\\\\\(Nullable\\\\\(\.\*\\\\\)\\\\\)'\), B, CAST\(B, concat\('Array\(Nullable\(', extract\(toTypeName\(B\), 'Array\\\\\(\(\.\*\)\\\\\)'\), '\)\)'\)\)\) AS arg1_\d+\), length\(if\(match\(toTypeName\(C\), 'Array\\\\\(Nullable\\\\\(\.\*\\\\\)\\\\\)'\), C, CAST\(C, concat\('Array\(Nullable\(', extract\(toTypeName\(C\), 'Array\\\\\(\(\.\*\)\\\\\)'\), '\)\)'\)\)\) AS arg2_\d+\)\]\) AS max_length_\d+, NULL\), arrayResize\(arg1_\d+, max_length_\d+, NULL\), arrayResize\(arg2_\d+, max_length_\d+, NULL\)\)\))" + } +}))); diff --git a/src/Parsers/tests/KQL/gtest_KQL_IP.cpp b/src/Parsers/tests/KQL/gtest_KQL_IP.cpp new file mode 100644 index 00000000000..5cc08ee89b4 --- /dev/null +++ b/src/Parsers/tests/KQL/gtest_KQL_IP.cpp @@ -0,0 +1,85 @@ +#include + +#include + +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_IP, ParserRegexTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "print format_ipv4(A)", + R"(SELECT ifNull\(if\(\(\(\(toUInt32OrNull\(toString\(A\)\) AS param_as_uint32_\d+\) IS NOT NULL\) AND \(toTypeName\(A\) = 'String'\)\) OR \(32 < 0\) OR \(\(ifNull\(param_as_uint32_\d+, multiIf\(length\(splitByChar\('/', ifNull\(toString\(A\), ''\)\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\).1\)\), NULL\)\) AS ip_as_number_\d+\) IS NULL\), NULL, IPv4NumToString\(bitAnd\(ip_as_number_\d+, bitNot\(toUInt32\(intExp2\(toInt32\(32 - 32\)\) - 1\)\)\)\)\), ''\))" + }, + { + "print format_ipv4(A, B)", + R"(SELECT ifNull\(if\(\(\(\(toUInt32OrNull\(toString\(A\)\) AS param_as_uint32_\d+\) IS NOT NULL\) AND \(toTypeName\(A\) = 'String'\)\) OR \(B < 0\) OR \(\(ifNull\(param_as_uint32_\d+, multiIf\(length\(splitByChar\('/', ifNull\(toString\(A\), ''\)\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\).1\)\), NULL\)\) AS ip_as_number_\d+\) IS NULL\), NULL, IPv4NumToString\(bitAnd\(ip_as_number_\d+, bitNot\(toUInt32\(intExp2\(toInt32\(32 - B\)\) - 1\)\)\)\)\), ''\))" + }, + { + "print format_ipv4_mask(A)", + R"(SELECT if\(empty\(ifNull\(if\(\(\(\(toUInt32OrNull\(toString\(A\)\) AS param_as_uint32_\d+\) IS NOT NULL\) AND \(toTypeName\(A\) = 'String'\)\) OR \(32 < 0\) OR \(\(ifNull\(param_as_uint32_\d+, multiIf\(length\(splitByChar\('/', ifNull\(toString\(A\), ''\)\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\).1\)\), NULL\)\) AS ip_as_number_\d+\) IS NULL\), NULL, IPv4NumToString\(bitAnd\(ip_as_number_\d+, bitNot\(toUInt32\(intExp2\(toInt32\(32 - 32\)\) - 1\)\)\)\)\), ''\) AS formatted_ip_\d+\) OR \(position\(toTypeName\(32\), 'Int'\) = 0\) OR \(NOT \(\(32 >= 0\) AND \(32 <= 32\)\)\), '', concat\(formatted_ip_\d+, '/', toString\(toInt64\(min2\(32, ifNull\(multiIf\(\(length\(splitByChar\('/', ifNull\(toString\(A\), ''\)\) AS tokens_\d+\) > 2\) OR \(NOT isIPv4String\(tokens_\d+\[1\]\)\), NULL, length\(tokens_\d+\) = 1, 32, \(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL, NULL, toUInt8\(min2\(mask_\d+, 32\)\)\) AS suffix_\d+, 32\)\)\)\)\)\))" + }, + { + "print format_ipv4_mask(A, B)", + R"(SELECT if\(empty\(ifNull\(if\(\(\(\(toUInt\d+OrNull\(toString\(A\)\) AS param_as_uint\d+_\d+\) IS NOT NULL\) AND \(toTypeName\(A\) = 'String'\)\) OR \(B < 0\) OR \(\(ifNull\(param_as_uint\d+_\d+, multiIf\(length\(splitByChar\('/', ifNull\(toString\(A\), ''\)\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\)\.1\)\), NULL\)\) AS ip_as_number_\d+\) IS NULL\), NULL, IPv4NumToString\(bitAnd\(ip_as_number_\d+, bitNot\(toUInt\d+\(intExp2\(toInt32\(\d+ - B\)\) - 1\)\)\)\)\), ''\) AS formatted_ip_\d+\) OR \(position\(toTypeName\(B\), 'Int'\) = 0\) OR \(NOT \(\(B >= 0\) AND \(B <= \d+\)\)\), '', concat\(formatted_ip_\d+, '/', toString\(toInt\d+\(min2\(B, ifNull\(multiIf\(\(length\(splitByChar\('/', ifNull\(toString\(A\), ''\)\) AS tokens_\d+\) > 2\) OR \(NOT isIPv4String\(tokens_\d+\[1\]\)\), NULL, length\(tokens_\d+\) = 1, \d+, \(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL, NULL, toUInt8\(min2\(mask_\d+, \d+\)\)\) AS suffix_\d+, \d+\)\)\)\)\)\))" + }, + { + "print ipv4_compare(A, B)", + R"(SELECT if\(\(\(multiIf\(length\(splitByChar\('/', A\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\)\.1\)\), NULL\) AS lhs_ip_\d+\) IS NULL\) OR \(\(multiIf\(\(length\(splitByChar\('/', A\) AS tokens_\d+\) > 2\) OR \(NOT isIPv4String\(tokens_\d+\[1\]\)\), NULL, length\(tokens_\d+\) = 1, \d+, \(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL, NULL, toUInt8\(min2\(mask_\d+, \d+\)\)\) AS lhs_mask_\d+\) IS NULL\) OR \(\(multiIf\(length\(splitByChar\('/', B\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\)\.1\)\), NULL\) AS rhs_ip_\d+\) IS NULL\) OR \(\(multiIf\(\(length\(splitByChar\('/', B\) AS tokens_\d+\) > 2\) OR \(NOT isIPv4String\(tokens_\d+\[1\]\)\), NULL, length\(tokens_\d+\) = 1, \d+, \(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL, NULL, toUInt8\(min2\(mask_\d+, \d+\)\)\) AS rhs_mask_\d+\) IS NULL\), NULL, sign\(IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(assumeNotNull\(lhs_ip_\d+\), toUInt8\(min2\(\d+, min2\(assumeNotNull\(lhs_mask_\d+\), assumeNotNull\(rhs_mask_\d+\)\)\)\) AS mask_\d+\)\.1\)\) - IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(assumeNotNull\(rhs_ip_\d+\), mask_\d+\)\.1\)\)\)\))" + }, + { + "print ipv4_compare(A, B, C)", + R"(SELECT if\(\(\(multiIf\(length\(splitByChar\('/', A\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\)\.1\)\), NULL\) AS lhs_ip_\d+\) IS NULL\) OR \(\(multiIf\(\(length\(splitByChar\('/', A\) AS tokens_\d+\) > 2\) OR \(NOT isIPv4String\(tokens_\d+\[1\]\)\), NULL, length\(tokens_\d+\) = 1, \d+, \(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL, NULL, toUInt8\(min2\(mask_\d+, \d+\)\)\) AS lhs_mask_\d+\) IS NULL\) OR \(\(multiIf\(length\(splitByChar\('/', B\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\)\.1\)\), NULL\) AS rhs_ip_\d+\) IS NULL\) OR \(\(multiIf\(\(length\(splitByChar\('/', B\) AS tokens_\d+\) > 2\) OR \(NOT isIPv4String\(tokens_\d+\[1\]\)\), NULL, length\(tokens_\d+\) = 1, \d+, \(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL, NULL, toUInt8\(min2\(mask_\d+, \d+\)\)\) AS rhs_mask_\d+\) IS NULL\), NULL, sign\(IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(assumeNotNull\(lhs_ip_\d+\), toUInt8\(min2\(C, min2\(assumeNotNull\(lhs_mask_\d+\), assumeNotNull\(rhs_mask_\d+\)\)\)\) AS mask_\d+\)\.1\)\) - IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(assumeNotNull\(rhs_ip_\d+\), mask_\d+\)\.1\)\)\)\))" + }, + { + "print ipv6_compare(A, B)", + R"(SELECT if\(\(length\(splitByChar\('/', A\) AS lhs_tokens_\d+\) > 2\) OR \(length\(splitByChar\('/', B\) AS rhs_tokens_\d+\) > 2\) OR \(\(IPv6StringToNumOrNull\(lhs_tokens_\d+\[1\]\) AS lhs_ipv6_\d+\) IS NULL\) OR \(\(length\(lhs_tokens_\d+\) = 2\) AND \(\(\(if\(isIPv4String\(lhs_tokens_\d+\[1\]\), 96, 0\) \+ toUInt8OrNull\(lhs_tokens_\d+\[-1\]\)\) AS lhs_suffix_\d+\) IS NULL\)\) OR \(\(IPv6StringToNumOrNull\(rhs_tokens_\d+\[1\]\) AS rhs_ipv6_\d+\) IS NULL\) OR \(\(length\(rhs_tokens_\d+\) = 2\) AND \(\(\(if\(isIPv4String\(rhs_tokens_\d+\[1\]\), 96, 0\) \+ toUInt8OrNull\(rhs_tokens_\d+\[-1\]\)\) AS rhs_suffix_\d+\) IS NULL\)\) OR \(\(toUInt8\(min2\(128, min2\(ifNull\(lhs_suffix_\d+, 128\), ifNull\(rhs_suffix_\d+, 128\)\)\)\) AS suffix_\d+\) IS NULL\) OR \(\(bitShiftLeft\(bitShiftRight\(bitNot\(reinterpretAsFixedString\(CAST\('0', 'UInt128'\)\)\), 128 - suffix_\d+ AS zeroes_\d+\), zeroes_\d+\) AS mask_\d+\) IS NULL\) OR \(\(bitAnd\(lhs_ipv6_\d+, mask_\d+\) AS lhs_base_\d+\) IS NULL\) OR \(\(bitAnd\(rhs_ipv6_\d+, mask_\d+\) AS rhs_base_\d+\) IS NULL\), NULL, multiIf\(lhs_base_\d+ < rhs_base_\d+, -1, lhs_base_\d+ > rhs_base_\d+, 1, 0\)\))" + }, + { + "print ipv6_compare(A, B, C)", + R"(SELECT if\(\(length\(splitByChar\('/', A\) AS lhs_tokens_\d+\) > 2\) OR \(length\(splitByChar\('/', B\) AS rhs_tokens_\d+\) > 2\) OR \(\(IPv6StringToNumOrNull\(lhs_tokens_\d+\[1\]\) AS lhs_ipv6_\d+\) IS NULL\) OR \(\(length\(lhs_tokens_\d+\) = 2\) AND \(\(\(if\(isIPv4String\(lhs_tokens_\d+\[1\]\), 96, 0\) \+ toUInt8OrNull\(lhs_tokens_\d+\[-1\]\)\) AS lhs_suffix_\d+\) IS NULL\)\) OR \(\(IPv6StringToNumOrNull\(rhs_tokens_\d+\[1\]\) AS rhs_ipv6_\d+\) IS NULL\) OR \(\(length\(rhs_tokens_\d+\) = 2\) AND \(\(\(if\(isIPv4String\(rhs_tokens_\d+\[1\]\), 96, 0\) \+ toUInt8OrNull\(rhs_tokens_\d+\[-1\]\)\) AS rhs_suffix_\d+\) IS NULL\)\) OR \(\(toUInt8\(min2\(C, min2\(ifNull\(lhs_suffix_\d+, 128\), ifNull\(rhs_suffix_\d+, 128\)\)\)\) AS suffix_\d+\) IS NULL\) OR \(\(bitShiftLeft\(bitShiftRight\(bitNot\(reinterpretAsFixedString\(CAST\('0', 'UInt128'\)\)\), 128 - suffix_\d+ AS zeroes_\d+\), zeroes_\d+\) AS mask_\d+\) IS NULL\) OR \(\(bitAnd\(lhs_ipv6_\d+, mask_\d+\) AS lhs_base_\d+\) IS NULL\) OR \(\(bitAnd\(rhs_ipv6_\d+, mask_\d+\) AS rhs_base_\d+\) IS NULL\), NULL, multiIf\(lhs_base_\d+ < rhs_base_\d+, -1, lhs_base_\d+ > rhs_base_\d+, 1, 0\)\))" + }, + { + "print ipv4_is_in_range(A, B)", + R"(SELECT if\(\(\(IPv4StringToNumOrNull\(A\) AS ip_\d+\) IS NULL\) OR \(\(multiIf\(length\(splitByChar\('/', B\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\)\.1\)\), NULL\) AS range_start_ip_\d+\) IS NULL\) OR \(\(multiIf\(\(length\(splitByChar\('/', B\) AS tokens_\d+\) > 2\) OR \(NOT isIPv4String\(tokens_\d+\[1\]\)\), NULL, length\(tokens_\d+\) = 1, 32, \(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL, NULL, toUInt8\(min2\(mask_\d+, 32\)\)\) AS range_mask_\d+\) IS NULL\), NULL, bitXor\(range_start_ip_\d+, bitAnd\(ip_\d+, bitNot\(toUInt32\(intExp2\(toInt32\(32 - range_mask_\d+\)\) - 1\)\)\)\) = 0\))" + }, + { + "print ipv4_is_match(A, B)", + R"(SELECT if\(\(\(multiIf\(length\(splitByChar\('/', A\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\)\.1\)\), NULL\) AS lhs_ip_\d+\) IS NULL\) OR \(\(multiIf\(\(length\(splitByChar\('/', A\) AS tokens_\d+\) > 2\) OR \(NOT isIPv4String\(tokens_\d+\[1\]\)\), NULL, length\(tokens_\d+\) = 1, 32, \(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL, NULL, toUInt8\(min2\(mask_\d+, 32\)\)\) AS lhs_mask_\d+\) IS NULL\) OR \(\(multiIf\(length\(splitByChar\('/', B\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\)\.1\)\), NULL\) AS rhs_ip_\d+\) IS NULL\) OR \(\(multiIf\(\(length\(splitByChar\('/', B\) AS tokens_\d+\) > 2\) OR \(NOT isIPv4String\(tokens_\d+\[1\]\)\), NULL, length\(tokens_\d+\) = 1, 32, \(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL, NULL, toUInt8\(min2\(mask_\d+, 32\)\)\) AS rhs_mask_\d+\) IS NULL\), NULL, sign\(IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(assumeNotNull\(lhs_ip_\d+\), toUInt8\(min2\(32, min2\(assumeNotNull\(lhs_mask_\d+\), assumeNotNull\(rhs_mask_\d+\)\)\)\) AS mask_\d+\)\.1\)\) - IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(assumeNotNull\(rhs_ip_\d+\), mask_\d+\)\.1\)\)\)\) = 0)" + }, + { + "print ipv4_is_match(A, B, C)", + R"(SELECT if\(\(\(multiIf\(length\(splitByChar\('/', A\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\)\.1\)\), NULL\) AS lhs_ip_\d+\) IS NULL\) OR \(\(multiIf\(\(length\(splitByChar\('/', A\) AS tokens_\d+\) > 2\) OR \(NOT isIPv4String\(tokens_\d+\[1\]\)\), NULL, length\(tokens_\d+\) = 1, 32, \(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL, NULL, toUInt8\(min2\(mask_\d+, 32\)\)\) AS lhs_mask_\d+\) IS NULL\) OR \(\(multiIf\(length\(splitByChar\('/', B\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\)\.1\)\), NULL\) AS rhs_ip_\d+\) IS NULL\) OR \(\(multiIf\(\(length\(splitByChar\('/', B\) AS tokens_\d+\) > 2\) OR \(NOT isIPv4String\(tokens_\d+\[1\]\)\), NULL, length\(tokens_\d+\) = 1, 32, \(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL, NULL, toUInt8\(min2\(mask_\d+, 32\)\)\) AS rhs_mask_\d+\) IS NULL\), NULL, sign\(IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(assumeNotNull\(lhs_ip_\d+\), toUInt8\(min2\(C, min2\(assumeNotNull\(lhs_mask_\d+\), assumeNotNull\(rhs_mask_\d+\)\)\)\) AS mask_\d+\)\.1\)\) - IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(assumeNotNull\(rhs_ip_\d+\), mask_\d+\)\.1\)\)\)\) = 0)" + }, + { + "print ipv6_is_match(A, B)", + R"(SELECT if\(\(length\(splitByChar\('/', A\) AS lhs_tokens_\d+\) > 2\) OR \(length\(splitByChar\('/', B\) AS rhs_tokens_\d+\) > 2\) OR \(\(IPv6StringToNumOrNull\(lhs_tokens_\d+\[1\]\) AS lhs_ipv6_\d+\) IS NULL\) OR \(\(length\(lhs_tokens_\d+\) = 2\) AND \(\(\(if\(isIPv4String\(lhs_tokens_\d+\[1\]\), 96, 0\) \+ toUInt8OrNull\(lhs_tokens_\d+\[-1\]\)\) AS lhs_suffix_\d+\) IS NULL\)\) OR \(\(IPv6StringToNumOrNull\(rhs_tokens_\d+\[1\]\) AS rhs_ipv6_\d+\) IS NULL\) OR \(\(length\(rhs_tokens_\d+\) = 2\) AND \(\(\(if\(isIPv4String\(rhs_tokens_\d+\[1\]\), 96, 0\) \+ toUInt8OrNull\(rhs_tokens_\d+\[-1\]\)\) AS rhs_suffix_\d+\) IS NULL\)\) OR \(\(toUInt8\(min2\(128, min2\(ifNull\(lhs_suffix_\d+, 128\), ifNull\(rhs_suffix_\d+, 128\)\)\)\) AS suffix_\d+\) IS NULL\) OR \(\(bitShiftLeft\(bitShiftRight\(bitNot\(reinterpretAsFixedString\(CAST\('0', 'UInt128'\)\)\), 128 - suffix_\d+ AS zeroes_\d+\), zeroes_\d+\) AS mask_\d+\) IS NULL\) OR \(\(bitAnd\(lhs_ipv6_\d+, mask_\d+\) AS lhs_base_\d+\) IS NULL\) OR \(\(bitAnd\(rhs_ipv6_\d+, mask_\d+\) AS rhs_base_\d+\) IS NULL\), NULL, multiIf\(lhs_base_\d+ < rhs_base_\d+, -1, lhs_base_\d+ > rhs_base_\d+, 1, 0\)\) = 0)" + }, + { + "print ipv6_is_match(A, B, C)", + R"(SELECT if\(\(length\(splitByChar\('/', A\) AS lhs_tokens_\d+\) > 2\) OR \(length\(splitByChar\('/', B\) AS rhs_tokens_\d+\) > 2\) OR \(\(IPv6StringToNumOrNull\(lhs_tokens_\d+\[1\]\) AS lhs_ipv6_\d+\) IS NULL\) OR \(\(length\(lhs_tokens_\d+\) = 2\) AND \(\(\(if\(isIPv4String\(lhs_tokens_\d+\[1\]\), 96, 0\) \+ toUInt8OrNull\(lhs_tokens_\d+\[-1\]\)\) AS lhs_suffix_\d+\) IS NULL\)\) OR \(\(IPv6StringToNumOrNull\(rhs_tokens_\d+\[1\]\) AS rhs_ipv6_\d+\) IS NULL\) OR \(\(length\(rhs_tokens_\d+\) = 2\) AND \(\(\(if\(isIPv4String\(rhs_tokens_\d+\[1\]\), 96, 0\) \+ toUInt8OrNull\(rhs_tokens_\d+\[-1\]\)\) AS rhs_suffix_\d+\) IS NULL\)\) OR \(\(toUInt8\(min2\(C, min2\(ifNull\(lhs_suffix_\d+, 128\), ifNull\(rhs_suffix_\d+, 128\)\)\)\) AS suffix_\d+\) IS NULL\) OR \(\(bitShiftLeft\(bitShiftRight\(bitNot\(reinterpretAsFixedString\(CAST\('0', 'UInt128'\)\)\), 128 - suffix_\d+ AS zeroes_\d+\), zeroes_\d+\) AS mask_\d+\) IS NULL\) OR \(\(bitAnd\(lhs_ipv6_\d+, mask_\d+\) AS lhs_base_\d+\) IS NULL\) OR \(\(bitAnd\(rhs_ipv6_\d+, mask_\d+\) AS rhs_base_\d+\) IS NULL\), NULL, multiIf\(lhs_base_\d+ < rhs_base_\d+, -1, lhs_base_\d+ > rhs_base_\d+, 1, 0\)\) = 0)" + }, + { + "print ipv4_is_private(A)", + R"(SELECT multiIf\(\(length\(splitByChar\('/', A\) AS tokens_\d+\) > 2\) OR \(\(toIPv4OrNull\(tokens_\d+\[1\]\) AS nullable_ip_\d+\) IS NULL\) OR \(\(length\(tokens_\d+\) = 2\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL\)\), NULL, ignore\(assumeNotNull\(nullable_ip_\d+\) AS ip_\d+, IPv4CIDRToRange\(ip_\d+, assumeNotNull\(mask_\d+\)\) AS range_\d+, IPv4NumToString\(range_\d+.1\) AS begin_\d+, IPv4NumToString\(range_\d+.2\) AS end_\d+\), NULL, \(\(length\(tokens_\d+\) = 1\) AND isIPAddressInRange\(IPv4NumToString\(ip_\d+\), '10.0.0.0/8'\)\) OR \(\(length\(tokens_\d+\) = 2\) AND isIPAddressInRange\(begin_\d+, '10.0.0.0/8'\) AND isIPAddressInRange\(end_\d+, '10.0.0.0/8'\)\) OR \(\(length\(tokens_\d+\) = 1\) AND isIPAddressInRange\(IPv4NumToString\(ip_\d+\), '172.16.0.0/12'\)\) OR \(\(length\(tokens_\d+\) = 2\) AND isIPAddressInRange\(begin_\d+, '172.16.0.0/12'\) AND isIPAddressInRange\(end_\d+, '172.16.0.0/12'\)\) OR \(\(length\(tokens_\d+\) = 1\) AND isIPAddressInRange\(IPv4NumToString\(ip_\d+\), '192.168.0.0/16'\)\) OR \(\(length\(tokens_\d+\) = 2\) AND isIPAddressInRange\(begin_\d+, '192.168.0.0/16'\) AND isIPAddressInRange\(end_\d+, '192.168.0.0/16'\)\)\))" + }, + { + "print ipv4_netmask_suffix(A)", + R"(SELECT multiIf\(\(length\(splitByChar\('/', A\) AS tokens_\d+\) > 2\) OR \(NOT isIPv4String\(tokens_\d+\[1\]\)\), NULL, length\(tokens_\d+\) = 1, 32, \(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL, NULL, toUInt8\(min2\(mask_\d+, 32\)\)\))" + }, + { + "print parse_ipv4(A)", + R"(SELECT multiIf\(length\(splitByChar\('/', A\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\)\.1\)\), NULL\))" + }, + { + "print parse_ipv4_mask(A, B)", + R"(SELECT if\(\(\(toIPv4OrNull\(A\) AS ip_\d+\) IS NULL\) OR \(\(toUInt8OrNull\(toString\(B\)\) AS mask_\d+\) IS NULL\), NULL, toUInt32\(IPv4CIDRToRange\(assumeNotNull\(ip_\d+\), arrayMax\(\[0, arrayMin\(\[32, assumeNotNull\(mask_\d+\)\]\)\]\)\).1\)\))" + }, + { + "print parse_ipv6(A)", + R"(SELECT if\(\(length\(splitByChar\('/', assumeNotNull\(A\)\) AS tokens_\d+\) > 2\) OR \(\(IPv6StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+\) IS NULL\) OR \(\(length\(tokens_\d+\) = 2\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL\)\), NULL, arrayStringConcat\(flatten\(extractAllGroups\(lower\(hex\(IPv6CIDRToRange\(assumeNotNull\(ip_\d+\), toUInt8\(ifNull\(mask_\d+ \+ if\(isIPv4String\(tokens_\d+\[1\]\), 96, 0\), 128\)\)\)\.1\)\), '\(\[\\\\da-f\]\{4\}\)'\)\), ':'\)\))" + }, + { + "print parse_ipv6_mask(A, B)", + R"(SELECT if\(empty\(ifNull\(if\(\(\(\(toUInt32OrNull\(toString\(replaceRegexpOne\(A, concat\('\^', '::'\), ''\)\)\) AS param_as_uint32_\d+\) IS NOT NULL\) AND \(toTypeName\(replaceRegexpOne\(A, concat\('\^', '::'\), ''\)\) = 'String'\)\) OR \(\(B - 96\) < 0\) OR \(\(ifNull\(param_as_uint32_\d+, multiIf\(length\(splitByChar\('/', ifNull\(toString\(replaceRegexpOne\(A, concat\('\^', '::'\), ''\)\), ''\)\) AS tokens_\d+\) = 1, IPv4StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+, \(length\(tokens_\d+\) = 2\) AND \(ip_\d+ IS NOT NULL\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NOT NULL\), IPv4StringToNumOrNull\(toString\(IPv4CIDRToRange\(toIPv4\(assumeNotNull\(ip_\d+\)\), assumeNotNull\(mask_\d+\)\)\.1\)\), NULL\)\) AS ip_as_number_\d+\) IS NULL\), NULL, IPv4NumToString\(bitAnd\(ip_as_number_\d+, bitNot\(toUInt32\(intExp2\(toInt32\(32 - \(B - 96\)\)\) - 1\)\)\)\)\), ''\) AS ipv4_\d+\), if\(\(length\(splitByChar\('/', assumeNotNull\(concat\(ifNull\(toString\(if\(\(length\(splitByChar\('/', assumeNotNull\(A\)\) AS tokens_\d+\) > 2\) OR \(\(IPv6StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+\) IS NULL\) OR \(\(length\(tokens_\d+\) = 2\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL\)\), NULL, arrayStringConcat\(flatten\(extractAllGroups\(lower\(hex\(IPv6CIDRToRange\(assumeNotNull\(ip_\d+\), toUInt8\(ifNull\(mask_\d+ \+ if\(isIPv4String\(tokens_\d+\[1\]\), 96, 0\), 128\)\)\)\.1\)\), '\(\[\\\\da-f\]\{4\}\)'\)\), ':'\)\)\), ''\), '/', ifNull\(toString\(B\), ''\)\)\)\) AS tokens_\d+\) > 2\) OR \(\(IPv6StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+\) IS NULL\) OR \(\(length\(tokens_\d+\) = 2\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL\)\), NULL, arrayStringConcat\(flatten\(extractAllGroups\(lower\(hex\(IPv6CIDRToRange\(assumeNotNull\(ip_\d+\), toUInt8\(ifNull\(mask_\d+ \+ if\(isIPv4String\(tokens_\d+\[1\]\), 96, 0\), 128\)\)\)\.1\)\), '\(\[\\\\da-f\]\{4\}\)'\)\), ':'\)\), if\(\(length\(splitByChar\('/', assumeNotNull\(ipv4_\d+\)\) AS tokens_\d+\) > 2\) OR \(\(IPv6StringToNumOrNull\(tokens_\d+\[1\]\) AS ip_\d+\) IS NULL\) OR \(\(length\(tokens_\d+\) = 2\) AND \(\(toUInt8OrNull\(tokens_\d+\[-1\]\) AS mask_\d+\) IS NULL\)\), NULL, arrayStringConcat\(flatten\(extractAllGroups\(lower\(hex\(IPv6CIDRToRange\(assumeNotNull\(ip_\d+\), toUInt8\(ifNull\(mask_\d+ \+ if\(isIPv4String\(tokens_\d+\[1\]\), 96, 0\), 128\)\)\)\.1\)\), '\(\[\\\\da-f\]\{4\}\)'\)\), ':'\)\)\))" + } +}))); diff --git a/src/Parsers/tests/KQL/gtest_KQL_MVExpand.cpp b/src/Parsers/tests/KQL/gtest_KQL_MVExpand.cpp new file mode 100644 index 00000000000..b90a700c620 --- /dev/null +++ b/src/Parsers/tests/KQL/gtest_KQL_MVExpand.cpp @@ -0,0 +1,45 @@ +#include + +#include + +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_MVExpand, ParserKQLTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "T | mv-expand c", + "SELECT *\nFROM T\nARRAY JOIN c\nSETTINGS enable_unaligned_array_join = 1" + }, + { + "T | mv-expand c, d", + "SELECT *\nFROM T\nARRAY JOIN\n c,\n d\nSETTINGS enable_unaligned_array_join = 1" + }, + { + "T | mv-expand c to typeof(bool)", + "SELECT\n * EXCEPT c_ali,\n c_ali AS c\nFROM\n(\n SELECT\n * EXCEPT c,\n accurateCastOrNull(toInt64OrNull(toString(c)), 'Boolean') AS c_ali\n FROM\n (\n SELECT *\n FROM T\n ARRAY JOIN c\n )\n)\nSETTINGS enable_unaligned_array_join = 1" + }, + { + "T | mv-expand b | mv-expand c", + "SELECT *\nFROM\n(\n SELECT *\n FROM T\n ARRAY JOIN b\n SETTINGS enable_unaligned_array_join = 1\n)\nARRAY JOIN c\nSETTINGS enable_unaligned_array_join = 1" + }, + { + "T | mv-expand with_itemindex=index b, c, d", + "SELECT\n index,\n *\nFROM T\nARRAY JOIN\n b,\n c,\n d,\n range(0, arrayMax([length(b), length(c), length(d)])) AS index\nSETTINGS enable_unaligned_array_join = 1" + }, + { + "T | mv-expand array_concat(c,d)", + "SELECT\n *,\n array_concat_\nFROM T\nARRAY JOIN arrayConcat(c, d) AS array_concat_\nSETTINGS enable_unaligned_array_join = 1" + }, + { + "T | mv-expand x = c, y = d", + "SELECT\n *,\n x,\n y\nFROM T\nARRAY JOIN\n c AS x,\n d AS y\nSETTINGS enable_unaligned_array_join = 1" + }, + { + "T | mv-expand xy = array_concat(c, d)", + "SELECT\n *,\n xy\nFROM T\nARRAY JOIN arrayConcat(c, d) AS xy\nSETTINGS enable_unaligned_array_join = 1" + }, + { + "T | mv-expand with_itemindex=index c,d to typeof(bool)", + "SELECT\n * EXCEPT d_ali,\n d_ali AS d\nFROM\n(\n SELECT\n * EXCEPT d,\n accurateCastOrNull(toInt64OrNull(toString(d)), 'Boolean') AS d_ali\n FROM\n (\n SELECT\n index,\n *\n FROM T\n ARRAY JOIN\n c,\n d,\n range(0, arrayMax([length(c), length(d)])) AS index\n )\n)\nSETTINGS enable_unaligned_array_join = 1" + } +}))); diff --git a/src/Parsers/tests/KQL/gtest_KQL_MakeSeries.cpp b/src/Parsers/tests/KQL/gtest_KQL_MakeSeries.cpp new file mode 100644 index 00000000000..533b037190d --- /dev/null +++ b/src/Parsers/tests/KQL/gtest_KQL_MakeSeries.cpp @@ -0,0 +1,25 @@ +#include + +#include + +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_MakeSeries, ParserKQLTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "T | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) to datetime(2016-09-13) step 1d by Supplier, Fruit", + "SELECT\n Supplier,\n Fruit,\n zipped.1 AS Purchase,\n zipped.2 AS PriceAvg\nFROM\n(\n SELECT\n toUInt64(min(Purchase_ali)) AS low,\n toUInt64(max(Purchase_ali)) + 86400 AS high,\n arraySort(arrayZip(Purchase, PriceAvg)) AS zipped,\n Supplier,\n Fruit,\n arrayConcat(groupArray(PriceAvg_ali) AS ga, arrayMap(x -> 0, range(0, toUInt32(if((length(range(toUInt64(toDateTime64('2016-09-10', 9, 'UTC')), toUInt64(toDateTime64('2016-09-13', 9, 'UTC')), toUInt64(86400))) - length(ga)) < 0, 0, length(range(toUInt64(toDateTime64('2016-09-10', 9, 'UTC')), toUInt64(toDateTime64('2016-09-13', 9, 'UTC')), toUInt64(86400))) - length(ga))), 1))) AS PriceAvg,\n arrayDistinct(arrayConcat(groupArray(toDateTime64(Purchase_ali - 0, 9, 'UTC')), arrayMap(x -> toDateTime64(x - 0, 9, 'UTC'), range(toUInt64(toDateTime64('2016-09-10', 9, 'UTC')), toUInt64(toDateTime64('2016-09-13', 9, 'UTC')), toUInt64(86400))))) AS Purchase\n FROM\n (\n SELECT\n Supplier,\n Fruit,\n avg(Price) AS PriceAvg_ali,\n toFloat64(toDateTime64('2016-09-10', 9, 'UTC')) + (toInt64((toFloat64(toDateTime64(Purchase, 9, 'UTC')) - toFloat64(toDateTime64('2016-09-10', 9, 'UTC'))) / 86400) * 86400) AS Purchase_ali\n FROM T\n WHERE (toInt64(toFloat64(toDateTime64(Purchase, 9, 'UTC'))) >= toUInt64(toDateTime64('2016-09-10', 9, 'UTC'))) AND (toInt64(toFloat64(toDateTime64(Purchase, 9, 'UTC'))) < toUInt64(toDateTime64('2016-09-13', 9, 'UTC')))\n GROUP BY\n Supplier,\n Fruit,\n Purchase_ali\n ORDER BY Purchase_ali ASC\n )\n GROUP BY\n Supplier,\n Fruit\n)" + }, + { + "T2 | make-series PriceAvg=avg(Price) default=0 on Purchase from 10 to 15 step 1.0 by Supplier, Fruit", + "SELECT\n Supplier,\n Fruit,\n zipped.1 AS Purchase,\n zipped.2 AS PriceAvg\nFROM\n(\n SELECT\n toUInt64(min(Purchase_ali)) AS low,\n toUInt64(max(Purchase_ali)) + 1 AS high,\n arraySort(arrayZip(Purchase, PriceAvg)) AS zipped,\n Supplier,\n Fruit,\n arrayConcat(groupArray(PriceAvg_ali) AS ga, arrayMap(x -> 0, range(0, toUInt32(if((length(range(toUInt64(10), toUInt64(15), toUInt64(1))) - length(ga)) < 0, 0, length(range(toUInt64(10), toUInt64(15), toUInt64(1))) - length(ga))), 1))) AS PriceAvg,\n arrayDistinct(arrayConcat(groupArray(Purchase_ali), arrayMap(x -> toFloat64(x), range(toUInt64(10), toUInt64(15), toUInt64(1))))) AS Purchase\n FROM\n (\n SELECT\n Supplier,\n Fruit,\n avg(Price) AS PriceAvg_ali,\n toFloat64(10) + (toInt64((toFloat64(Purchase) - toFloat64(10)) / 1) * 1) AS Purchase_ali\n FROM T2\n WHERE (toInt64(toFloat64(Purchase)) >= toUInt64(10)) AND (toInt64(toFloat64(Purchase)) < toUInt64(15))\n GROUP BY\n Supplier,\n Fruit,\n Purchase_ali\n ORDER BY Purchase_ali ASC\n )\n GROUP BY\n Supplier,\n Fruit\n)" + }, + { + "T | make-series PriceAvg = avg(Price) default=0 on Purchase step 1d by Supplier, Fruit", + "SELECT\n Supplier,\n Fruit,\n zipped.1 AS Purchase,\n zipped.2 AS PriceAvg\nFROM\n(\n SELECT\n toUInt64(min(Purchase_ali)) AS low,\n toUInt64(max(Purchase_ali)) + 86400 AS high,\n arraySort(arrayZip(Purchase, PriceAvg)) AS zipped,\n Supplier,\n Fruit,\n arrayConcat(groupArray(PriceAvg_ali) AS ga, arrayMap(x -> 0, range(0, toUInt32(if((length(range(low, high, toUInt64(86400))) - length(ga)) < 0, 0, length(range(low, high, toUInt64(86400))) - length(ga))), 1))) AS PriceAvg,\n arrayDistinct(arrayConcat(groupArray(toDateTime64(Purchase_ali - 62135596800, 9, 'UTC')), arrayMap(x -> toDateTime64(x - 62135596800, 9, 'UTC'), range(low, high, toUInt64(86400))))) AS Purchase\n FROM\n (\n SELECT\n Supplier,\n Fruit,\n avg(Price) AS PriceAvg_ali,\n toFloat64(toInt64((toFloat64(toDateTime64(Purchase, 9, 'UTC')) + 62135596800) / 86400) * 86400) AS Purchase_ali\n FROM T\n GROUP BY\n Supplier,\n Fruit,\n Purchase_ali\n ORDER BY Purchase_ali ASC\n )\n GROUP BY\n Supplier,\n Fruit\n)" + }, + { + "T2 | make-series PriceAvg=avg(Price) default=0 on Purchase step 1.0 by Supplier, Fruit", + "SELECT\n Supplier,\n Fruit,\n zipped.1 AS Purchase,\n zipped.2 AS PriceAvg\nFROM\n(\n SELECT\n toUInt64(min(Purchase_ali)) AS low,\n toUInt64(max(Purchase_ali)) + 1 AS high,\n arraySort(arrayZip(Purchase, PriceAvg)) AS zipped,\n Supplier,\n Fruit,\n arrayConcat(groupArray(PriceAvg_ali) AS ga, arrayMap(x -> 0, range(0, toUInt32(if((length(range(low, high, toUInt64(1))) - length(ga)) < 0, 0, length(range(low, high, toUInt64(1))) - length(ga))), 1))) AS PriceAvg,\n arrayDistinct(arrayConcat(groupArray(Purchase_ali), arrayMap(x -> toFloat64(x), range(low, high, toUInt64(1))))) AS Purchase\n FROM\n (\n SELECT\n Supplier,\n Fruit,\n avg(Price) AS PriceAvg_ali,\n toFloat64(toInt64((toFloat64(Purchase) + 0) / 1) * 1) AS Purchase_ali\n FROM T2\n GROUP BY\n Supplier,\n Fruit,\n Purchase_ali\n ORDER BY Purchase_ali ASC\n )\n GROUP BY\n Supplier,\n Fruit\n)" + } +}))); diff --git a/src/Parsers/tests/KQL/gtest_KQL_StringFunctions.cpp b/src/Parsers/tests/KQL/gtest_KQL_StringFunctions.cpp new file mode 100644 index 00000000000..40a6922210f --- /dev/null +++ b/src/Parsers/tests/KQL/gtest_KQL_StringFunctions.cpp @@ -0,0 +1,225 @@ +#include + +#include + +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_String, ParserKQLTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "print base64_encode_fromguid(A)", + "SELECT if(toTypeName(A) NOT IN ['UUID', 'Nullable(UUID)'], toString(throwIf(true, 'Expected guid as argument')), base64Encode(UUIDStringToNum(toString(A), 2)))" + }, + { + "print base64_decode_toguid(A)", + "SELECT toUUIDOrNull(UUIDNumToString(toFixedString(base64Decode(A), 16), 2))" + }, + { + "print base64_decode_toarray('S3VzdG8=')", + "SELECT arrayMap(x -> reinterpretAsUInt8(x), splitByRegexp('', base64Decode('S3VzdG8=')))" + }, + { + "print replace_regex('Hello, World!', '.', '\\0\\0')", + "SELECT replaceRegexpAll('Hello, World!', '.', '\\0\\0')" + }, + { + "print idx = has_any_index('this is an example', dynamic(['this', 'example'])) ", + "SELECT if(empty(['this', 'example']), -1, indexOf(arrayMap(x -> (x IN splitByChar(' ', 'this is an example')), if(empty(['this', 'example']), [''], arrayMap(x -> toString(x), ['this', 'example']))), 1) - 1) AS idx" + }, + { + "print idx = has_any_index('this is an example', dynamic([]))", + "SELECT if(empty([]), -1, indexOf(arrayMap(x -> (x IN splitByChar(' ', 'this is an example')), if(empty([]), [''], arrayMap(x -> toString(x), []))), 1) - 1) AS idx" + }, + { + "print translate('krasp', 'otsku', 'spark')", + "SELECT if(length('otsku') = 0, '', translate('spark', 'krasp', multiIf(length('otsku') = 0, 'krasp', (length('krasp') - length('otsku')) > 0, concat('otsku', repeat(substr('otsku', length('otsku'), 1), toUInt16(length('krasp') - length('otsku')))), (length('krasp') - length('otsku')) < 0, substr('otsku', 1, length('krasp')), 'otsku')))" + }, + { + "print trim_start('[^\\w]+', strcat('- ','Te st1','// $'))", + "SELECT replaceRegexpOne(concat('- ', 'Te st1', '// $'), concat('^', '[^\\\\w]+'), '')" + }, + { + "print trim_end('.com', 'bing.com')", + "SELECT replaceRegexpOne('bing.com', concat('.com', '$'), '')" + }, + { + "print trim('--', '--https://bing.com--')", + "SELECT replaceRegexpOne(replaceRegexpOne('--https://bing.com--', concat('--', '$'), ''), concat('^', '--'), '')" + }, + { + "print bool(1)", + "SELECT toBool(1)" + }, + { + "print datetime(2015-12-31 23:59:59.9)", + "SELECT parseDateTime64BestEffortOrNull('2015-12-31 23:59:59.9', 9, 'UTC')" + }, + { + "print datetime(\"2015-12-31 23:59:59.9\")", + "SELECT parseDateTime64BestEffortOrNull('2015-12-31 23:59:59.9', 9, 'UTC')" + }, + { + "print datetime('2015-12-31 23:59:59.9')", + "SELECT parseDateTime64BestEffortOrNull('2015-12-31 23:59:59.9', 9, 'UTC')" + }, + { + "print guid(74be27de-1e4e-49d9-b579-fe0b331d3642)", + "SELECT toUUIDOrNull('74be27de-1e4e-49d9-b579-fe0b331d3642')" + }, + { + "print guid('74be27de-1e4e-49d9-b579-fe0b331d3642')", + "SELECT toUUIDOrNull('74be27de-1e4e-49d9-b579-fe0b331d3642')" + }, + { + "print guid('74be27de1e4e49d9b579fe0b331d3642')", + "SELECT toUUIDOrNull('74be27de1e4e49d9b579fe0b331d3642')" + }, + { + "print int(32.5)", + "SELECT toInt32(32.5)" + }, + { + "print long(32.5)", + "SELECT toInt64(32.5)" + }, + { + "print real(32.5)", + "SELECT toFloat64(32.5)" + }, + { + "print time('1.22:34:8.128')", + "SELECT CAST('167648.128', 'Float64')" + }, + { + "print time('1d')", + "SELECT CAST('86400', 'Float64')" + }, + { + "print time('1.5d')", + "SELECT CAST('129600', 'Float64')" + }, + { + "print timespan('1.5d')", + "SELECT CAST('129600', 'Float64')" + }, + { + "print res = bin_at(6.5, 2.5, 7)", + "SELECT toFloat64(7) + (toInt64(((toFloat64(6.5) - toFloat64(7)) / 2.5) + -1) * 2.5) AS res" + }, + { + "print res = bin_at(1h, 1d, 12h)", + "SELECT concat(toString(toInt32(((toFloat64(43200) + (toInt64(((toFloat64(3600) - toFloat64(43200)) / 86400) + -1) * 86400)) AS x) / 3600)), ':', toString(toInt32((x % 3600) / 60)), ':', toString(toInt32((x % 3600) % 60))) AS res" + }, + { + "print res = bin_at(datetime(2017-05-15 10:20:00.0), 1d, datetime(1970-01-01 12:00:00.0))", + "SELECT toDateTime64(toFloat64(parseDateTime64BestEffortOrNull('1970-01-01 12:00:00.0', 9, 'UTC')) + (toInt64(((toFloat64(parseDateTime64BestEffortOrNull('2017-05-15 10:20:00.0', 9, 'UTC')) - toFloat64(parseDateTime64BestEffortOrNull('1970-01-01 12:00:00.0', 9, 'UTC'))) / 86400) + 0) * 86400), 9, 'UTC') AS res" + }, + { + "print bin(4.5, 1)", + "SELECT toInt64(toFloat64(4.5) / 1) * 1" + }, + { + "print bin(4.5, -1)", + "SELECT toInt64(toFloat64(4.5) / -1) * -1" + }, + { + "print bin(time(16d), 7d)", + "SELECT concat(toString(toInt32(((toInt64(toFloat64(CAST('1382400', 'Float64')) / 604800) * 604800) AS x) / 3600)), ':', toString(toInt32((x % 3600) / 60)), ':', toString(toInt32((x % 3600) % 60)))" + }, + { + "print bin(datetime(1970-05-11 13:45:07), 1d)", + "SELECT toDateTime64(toInt64(toFloat64(parseDateTime64BestEffortOrNull('1970-05-11 13:45:07', 9, 'UTC')) / 86400) * 86400, 9, 'UTC')" + }, + { + "print extract('x=([0-9.]+)', 1, 'hello x=456|wo' , typeof(bool));", + "SELECT accurateCastOrNull(toInt64OrNull(extract('hello x=456|wo', '[0-9.]+')), 'Boolean')" + }, + { + "print extract('x=([0-9.]+)', 1, 'hello x=456|wo' , typeof(date));", + "SELECT accurateCastOrNull(extract('hello x=456|wo', '[0-9.]+'), 'DateTime')" + }, + { + "print extract('x=([0-9.]+)', 1, 'hello x=456|wo' , typeof(guid));", + "SELECT accurateCastOrNull(extract('hello x=456|wo', '[0-9.]+'), 'UUID')" + }, + { + "print extract('x=([0-9.]+)', 1, 'hello x=456|wo' , typeof(int));", + "SELECT accurateCastOrNull(extract('hello x=456|wo', '[0-9.]+'), 'Int32')" + }, + { + "print extract('x=([0-9.]+)', 1, 'hello x=456|wo' , typeof(long));", + "SELECT accurateCastOrNull(extract('hello x=456|wo', '[0-9.]+'), 'Int64')" + }, + { + "print extract('x=([0-9.]+)', 1, 'hello x=456|wo' , typeof(real));", + "SELECT accurateCastOrNull(extract('hello x=456|wo', '[0-9.]+'), 'Float64')" + }, + { + "print extract('x=([0-9.]+)', 1, 'hello x=456|wo' , typeof(decimal));", + "SELECT toDecimal128OrNull(if(countSubstrings(extract('hello x=456|wo', '[0-9.]+'), '.') > 1, NULL, extract('hello x=456|wo', '[0-9.]+')), length(substr(extract('hello x=456|wo', '[0-9.]+'), position(extract('hello x=456|wo', '[0-9.]+'), '.') + 1)))" + }, + { + "print bin(datetime(1970-05-11 13:45:07.456345672), 1ms)", + "SELECT toDateTime64(toInt64(toFloat64(parseDateTime64BestEffortOrNull('1970-05-11 13:45:07.456345672', 9, 'UTC')) / 0.001) * 0.001, 9, 'UTC')" + }, + { + "print bin(datetime(1970-05-11 13:45:07.456345672), 1microseconds)", + "SELECT toDateTime64(toInt64(toFloat64(parseDateTime64BestEffortOrNull('1970-05-11 13:45:07.456345672', 9, 'UTC')) / 0.000001) * 0.000001, 9, 'UTC')" + }, + { + "print parse_version('1.2.3.40')", + "SELECT if((length(splitByChar('.', '1.2.3.40')) > 4) OR (length(splitByChar('.', '1.2.3.40')) < 1) OR (match('1.2.3.40', '.*[a-zA-Z]+.*') = 1), toDecimal128OrNull('NULL', 0), toDecimal128OrNull(substring(arrayStringConcat(arrayMap(x -> leftPad(x, 8, '0'), arrayMap(x -> if(empty(x), '0', x), arrayResize(splitByChar('.', '1.2.3.40'), 4)))), 8), 0))" + }, + { + "print parse_version('1')", + "SELECT if((length(splitByChar('.', '1')) > 4) OR (length(splitByChar('.', '1')) < 1) OR (match('1', '.*[a-zA-Z]+.*') = 1), toDecimal128OrNull('NULL', 0), toDecimal128OrNull(substring(arrayStringConcat(arrayMap(x -> leftPad(x, 8, '0'), arrayMap(x -> if(empty(x), '0', x), arrayResize(splitByChar('.', '1'), 4)))), 8), 0))" + }, + { + "print parse_json( dynamic([1, 2, 3]))", + "SELECT [1, 2, 3]" + }, + { + "print parse_json('{\"a\":123.5, \"b\":\"{\\\"c\\\":456}\"}')", + "SELECT if(isValidJSON('{\"a\":123.5, \"b\":\"{\"c\":456}\"}'), JSON_QUERY('{\"a\":123.5, \"b\":\"{\"c\":456}\"}', '$'), toJSONString('{\"a\":123.5, \"b\":\"{\"c\":456}\"}'))" + }, + { + "print extract_json( '$.a' , '{\"a\":123, \"b\":\"{\"c\":456}\"}' , typeof(long))", + "SELECT accurateCastOrNull(JSON_VALUE('{\"a\":123, \"b\":\"{\"c\":456}\"}', '$.a'), 'Int64')" + }, + { + "print bin(datetime(1970-05-11 13:45:07.456345672), 1ms)", + "SELECT toDateTime64(toInt64(toFloat64(parseDateTime64BestEffortOrNull('1970-05-11 13:45:07.456345672', 9, 'UTC')) / 0.001) * 0.001, 9, 'UTC')" + }, + { + "print bin(datetime(1970-05-11 13:45:07.456345672), 1microseconds)", + "SELECT toDateTime64(toInt64(toFloat64(parseDateTime64BestEffortOrNull('1970-05-11 13:45:07.456345672', 9, 'UTC')) / 0.000001) * 0.000001, 9, 'UTC')" + }, + { + "print parse_command_line('echo \"hello world!\" print$?', 'windows')", + "SELECT if(empty('echo \"hello world!\" print$?') OR hasAll(splitByChar(' ', 'echo \"hello world!\" print$?'), ['']), arrayMap(x -> NULL, splitByChar(' ', '')), splitByChar(' ', 'echo \"hello world!\" print$?'))" + }, + { + "print reverse(123)", + "SELECT reverse(accurateCastOrNull(123, 'String'))" + }, + { + "print reverse(123.34)", + "SELECT reverse(accurateCastOrNull(123.34, 'String'))" + }, + { + "print reverse('clickhouse')", + "SELECT reverse(accurateCastOrNull('clickhouse', 'String'))" + }, + { + "print result=parse_csv('aa,b,cc')", + "SELECT if(CAST(position('aa,b,cc', '\\n'), 'UInt8'), splitByChar(',', substring('aa,b,cc', 1, position('aa,b,cc', '\\n') - 1)), splitByChar(',', substring('aa,b,cc', 1, length('aa,b,cc')))) AS result" + }, + { + "print result_multi_record=parse_csv('record1,a,b,c\nrecord2,x,y,z')", + "SELECT if(CAST(position('record1,a,b,c\\nrecord2,x,y,z', '\\n'), 'UInt8'), splitByChar(',', substring('record1,a,b,c\\nrecord2,x,y,z', 1, position('record1,a,b,c\\nrecord2,x,y,z', '\\n') - 1)), splitByChar(',', substring('record1,a,b,c\\nrecord2,x,y,z', 1, length('record1,a,b,c\\nrecord2,x,y,z')))) AS result_multi_record" + }, + { + "Customers | project name_abbr = strcat(substring(FirstName,0,3), ' ', substring(LastName,2))| order by LastName", + "SELECT concat(if(toInt64(length(FirstName)) <= 0, '', substr(FirstName, (((0 % toInt64(length(FirstName))) + toInt64(length(FirstName))) % toInt64(length(FirstName))) + 1, 3)), ' ', if(toInt64(length(LastName)) <= 0, '', substr(LastName, (((2 % toInt64(length(LastName))) + toInt64(length(LastName))) % toInt64(length(LastName))) + 1))) AS name_abbr\nFROM Customers\nORDER BY LastName DESC" + } +}))); diff --git a/src/Parsers/tests/KQL/gtest_KQL_dateTimeFunctions.cpp b/src/Parsers/tests/KQL/gtest_KQL_dateTimeFunctions.cpp new file mode 100644 index 00000000000..ed38411c1b7 --- /dev/null +++ b/src/Parsers/tests/KQL/gtest_KQL_dateTimeFunctions.cpp @@ -0,0 +1,218 @@ +#include + +#include + +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_Datetime, ParserKQLTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "print week_of_year(datetime(2020-12-31))", + "SELECT toWeek(parseDateTime64BestEffortOrNull('2020-12-31', 9, 'UTC'), 3, 'UTC')" + }, + { + "print startofweek(datetime(2017-01-01 10:10:17), -1)", + "SELECT parseDateTime64BestEffortOrNull(toString(toStartOfWeek(parseDateTime64BestEffortOrNull('2017-01-01 10:10:17', 9, 'UTC'))), 9, 'UTC') + toIntervalWeek(-1)" + }, + { + "print startofmonth(datetime(2017-01-01 10:10:17), -1)", + "SELECT parseDateTime64BestEffortOrNull(toString(toStartOfMonth(parseDateTime64BestEffortOrNull('2017-01-01 10:10:17', 9, 'UTC'))), 9, 'UTC') + toIntervalMonth(-1)" + }, + { + "print startofday(datetime(2017-01-01 10:10:17), -1)", + "SELECT parseDateTime64BestEffortOrNull(toString(toStartOfDay(parseDateTime64BestEffortOrNull('2017-01-01 10:10:17', 9, 'UTC'))), 9, 'UTC') + toIntervalDay(-1)" + }, + { + "print startofyear(datetime(2017-01-01 10:10:17), -1)", + "SELECT parseDateTime64BestEffortOrNull(toString(toStartOfYear(parseDateTime64BestEffortOrNull('2017-01-01 10:10:17', 9, 'UTC'), 'UTC')), 9, 'UTC') + toIntervalYear(-1)" + }, + { + "print monthofyear(datetime(2015-12-14))", + "SELECT toMonth(parseDateTime64BestEffortOrNull('2015-12-14', 9, 'UTC'))" + }, + { + "print hourofday(datetime(2015-12-14 10:54:00))", + "SELECT toHour(parseDateTime64BestEffortOrNull('2015-12-14 10:54:00', 9, 'UTC'))" + }, + { + "print getyear(datetime(2015-10-12))", + "SELECT toYear(parseDateTime64BestEffortOrNull('2015-10-12', 9, 'UTC'))" + }, + { + "print getmonth(datetime(2015-10-12))", + "SELECT toMonth(parseDateTime64BestEffortOrNull('2015-10-12', 9, 'UTC'))" + }, + { + "print dayofyear(datetime(2015-10-12))", + "SELECT toDayOfYear(parseDateTime64BestEffortOrNull('2015-10-12', 9, 'UTC'))" + }, + { + "print dayofmonth(datetime(2015-10-12))", + "SELECT toDayOfMonth(parseDateTime64BestEffortOrNull('2015-10-12', 9, 'UTC'))" + }, + { + "print unixtime_seconds_todatetime(1546300899)", + "SELECT if((toTypeName(1546300899) = 'Int64') OR (toTypeName(1546300899) = 'Int32') OR (toTypeName(1546300899) = 'Float64') OR (toTypeName(1546300899) = 'UInt32') OR (toTypeName(1546300899) = 'UInt64'), toDateTime64(1546300899, 9, 'UTC'), toDateTime64(throwIf(true, 'unixtime_seconds_todatetime only accepts Int, Long and double type of arguments'), 9, 'UTC'))" + }, + { + "print dayofweek(datetime(2015-12-20))", + "SELECT concat(CAST(toDayOfWeek(parseDateTime64BestEffortOrNull('2015-12-20', 9, 'UTC')) % 7, 'String'), '.00:00:00')" + }, + { + "print now()", + "SELECT now64(9, 'UTC')" + }, + { + "print now(1d)", + "SELECT now64(9, 'UTC') + 86400" + }, + { + "print ago(2d)", + "SELECT now64(9, 'UTC') - 172800" + }, + { + "print endofday(datetime(2017-01-01 10:10:17), -1)", + "SELECT (toDateTime(toStartOfDay(parseDateTime64BestEffortOrNull('2017-01-01 10:10:17', 9, 'UTC')), 9, 'UTC') + toIntervalDay(-1 + 1)) - toIntervalMicrosecond(1)" + }, + { + "print endofday(datetime(2017-01-01 10:10:17), 1)", + "SELECT (toDateTime(toStartOfDay(parseDateTime64BestEffortOrNull('2017-01-01 10:10:17', 9, 'UTC')), 9, 'UTC') + toIntervalDay(1 + 1)) - toIntervalMicrosecond(1)" + }, + { + "print endofmonth(datetime(2017-01-01 10:10:17), -1)", + "SELECT (((toDateTime(toLastDayOfMonth(toDateTime(parseDateTime64BestEffortOrNull('2017-01-01 10:10:17', 9, 'UTC'), 9, 'UTC') + toIntervalMonth(-1)), 9, 'UTC') + toIntervalHour(23)) + toIntervalMinute(59)) + toIntervalSecond(60)) - toIntervalMicrosecond(1)" + }, + { + "print endofmonth(datetime(2017-01-01 10:10:17), 1)", + "SELECT (((toDateTime(toLastDayOfMonth(toDateTime(parseDateTime64BestEffortOrNull('2017-01-01 10:10:17', 9, 'UTC'), 9, 'UTC') + toIntervalMonth(1)), 9, 'UTC') + toIntervalHour(23)) + toIntervalMinute(59)) + toIntervalSecond(60)) - toIntervalMicrosecond(1)" + }, + { + "print endofweek(datetime(2017-01-01 10:10:17), -1)", + "SELECT (toDateTime(toStartOfDay(parseDateTime64BestEffortOrNull('2017-01-01 10:10:17', 9, 'UTC')), 9, 'UTC') + toIntervalWeek(-1 + 1)) - toIntervalMicrosecond(1)" + }, + { + "print endofweek(datetime(2017-01-01 10:10:17), 1)", + "SELECT (toDateTime(toStartOfDay(parseDateTime64BestEffortOrNull('2017-01-01 10:10:17', 9, 'UTC')), 9, 'UTC') + toIntervalWeek(1 + 1)) - toIntervalMicrosecond(1)" + }, + { + "print endofyear(datetime(2017-01-01 10:10:17), -1) ", + "SELECT (((toDateTime(toString(toLastDayOfMonth((toDateTime(parseDateTime64BestEffortOrNull('2017-01-01 10:10:17', 9, 'UTC'), 9, 'UTC') + toIntervalYear(-1)) + toIntervalMonth(12 - toInt8(substring(toString(toDateTime(parseDateTime64BestEffortOrNull('2017-01-01 10:10:17', 9, 'UTC'), 9, 'UTC')), 6, 2))))), 9, 'UTC') + toIntervalHour(23)) + toIntervalMinute(59)) + toIntervalSecond(60)) - toIntervalMicrosecond(1)" + }, + { + "print endofyear(datetime(2017-01-01 10:10:17), 1)" , + "SELECT (((toDateTime(toString(toLastDayOfMonth((toDateTime(parseDateTime64BestEffortOrNull('2017-01-01 10:10:17', 9, 'UTC'), 9, 'UTC') + toIntervalYear(1)) + toIntervalMonth(12 - toInt8(substring(toString(toDateTime(parseDateTime64BestEffortOrNull('2017-01-01 10:10:17', 9, 'UTC'), 9, 'UTC')), 6, 2))))), 9, 'UTC') + toIntervalHour(23)) + toIntervalMinute(59)) + toIntervalSecond(60)) - toIntervalMicrosecond(1)" + }, + { + "print make_datetime(2017,10,01)", + "SELECT makeDateTime64(2017, 10, 1, 0, 0, 0, 0, 7, 'UTC')" + }, + { + "print make_datetime(2017,10,01,12,10)", + "SELECT makeDateTime64(2017, 10, 1, 12, 10, 0, 0, 7, 'UTC')" + }, + { + "print make_datetime(2017,10,01,12,11,0.1234567)", + "SELECT makeDateTime64(2017, 10, 1, 12, 11, 0.1234567, 0, 7, 'UTC')" + }, + { + "print unixtime_microseconds_todatetime(1546300800000000)", + "SELECT fromUnixTimestamp64Micro(1546300800000000, 'UTC')" + }, + { + "print unixtime_milliseconds_todatetime(1546300800000)", + "SELECT fromUnixTimestamp64Milli(1546300800000, 'UTC')" + }, + { + "print unixtime_nanoseconds_todatetime(1546300800000000000)", + "SELECT fromUnixTimestamp64Nano(1546300800000000000, 'UTC')" + }, + { + "print datetime_diff('year',datetime(2017-01-01),datetime(2000-12-31))", + "SELECT dateDiff('year', parseDateTime64BestEffortOrNull('2017-01-01', 9, 'UTC'), parseDateTime64BestEffortOrNull('2000-12-31', 9, 'UTC')) * -1" + }, + { + "print datetime_diff('minute',datetime(2017-10-30 23:05:01),datetime(2017-10-30 23:00:59))", + "SELECT dateDiff('minute', parseDateTime64BestEffortOrNull('2017-10-30 23:05:01', 9, 'UTC'), parseDateTime64BestEffortOrNull('2017-10-30 23:00:59', 9, 'UTC')) * -1" + }, + { + "print datetime(null)", + "SELECT NULL" + }, + { + "print datetime('2014-05-25T08:20:03.123456Z')", + "SELECT parseDateTime64BestEffortOrNull('2014-05-25T08:20:03.123456Z', 9, 'UTC')" + }, + { + "print datetime(2015-12-14 18:54)", + "SELECT parseDateTime64BestEffortOrNull('2015-12-14 18:54', 9, 'UTC')" + }, + { + "print make_timespan(67,12,30,59.9799)", + "SELECT CONCAT('67.', toString(substring(toString(toTime(parseDateTime64BestEffortOrNull('0000-00-00 12:30:59.9799', 9, 'UTC'))), 12)))" + }, + { + "print todatetime('2014-05-25T08:20:03.123456Z')", + "SELECT parseDateTime64BestEffortOrNull(toString('2014-05-25T08:20:03.123456Z'), 9, 'UTC')" + }, + { + "print format_datetime(todatetime('2009-06-15T13:45:30.6175425'), 'yy-M-dd [H:mm:ss.fff]')", + "SELECT concat(substring(toString(formatDateTime(parseDateTime64BestEffortOrNull(toString('2009-06-15T13:45:30.6175425'), 9, 'UTC'), '%y-%m-%d [%H:%i:%S.]')), 1, position(toString(formatDateTime(parseDateTime64BestEffortOrNull(toString('2009-06-15T13:45:30.6175425'), 9, 'UTC'), '%y-%m-%d [%H:%i:%S.]')), '.')), substring(substring(toString(parseDateTime64BestEffortOrNull(toString('2009-06-15T13:45:30.6175425'), 9, 'UTC')), position(toString(parseDateTime64BestEffortOrNull(toString('2009-06-15T13:45:30.6175425'), 9, 'UTC')), '.') + 1), 1, 3), substring(toString(formatDateTime(parseDateTime64BestEffortOrNull(toString('2009-06-15T13:45:30.6175425'), 9, 'UTC'), '%y-%m-%d [%H:%i:%S.]')), position(toString(formatDateTime(parseDateTime64BestEffortOrNull(toString('2009-06-15T13:45:30.6175425'), 9, 'UTC'), '%y-%m-%d [%H:%i:%S.]')), '.') + 1, length(toString(formatDateTime(parseDateTime64BestEffortOrNull(toString('2009-06-15T13:45:30.6175425'), 9, 'UTC'), '%y-%m-%d [%H:%i:%S.]')))))" + }, + { + "print format_datetime(datetime(2015-12-14 02:03:04.12345), 'y-M-d h:m:s tt')", + "SELECT formatDateTime(parseDateTime64BestEffortOrNull('2015-12-14 02:03:04.12345', 9, 'UTC'), '%y-%m-%e %h:%i:%S %p')" + }, + { + "print format_timespan(time(1d), 'd-[hh:mm:ss]')", + "SELECT concat(leftPad('1', 1, '0'), toString(formatDateTime(toDateTime64(CAST('86400', 'Float64'), 9, 'UTC'), '-[%h:%i:%S]')))" + }, + { + "print format_timespan(time('12:30:55.123'), 'ddddd-[hh:mm:ss.ffff]')", + "SELECT concat(leftPad('0', 5, '0'), substring(toString(formatDateTime(toDateTime64(CAST('45055.123', 'Float64'), 9, 'UTC'), '-[%H:%i:%S.]')), 1, length(toString(formatDateTime(toDateTime64(CAST('45055.123', 'Float64'), 9, 'UTC'), '-[%H:%i:%S.]'))) - position(reverse(toString(formatDateTime(toDateTime64(CAST('45055.123', 'Float64'), 9, 'UTC'), '-[%H:%i:%S.]'))), ']')), substring(substring(toString(toDateTime64(CAST('45055.123', 'Float64'), 9, 'UTC')), position(toString(toDateTime64(CAST('45055.123', 'Float64'), 9, 'UTC')), '.') + 1), 1, 4), substring(toString(formatDateTime(toDateTime64(CAST('45055.123', 'Float64'), 9, 'UTC'), '-[%H:%i:%S.]')), position(toString(formatDateTime(toDateTime64(CAST('45055.123', 'Float64'), 9, 'UTC'), '-[%H:%i:%S.]')), ']'), length(toString(formatDateTime(toDateTime64(CAST('45055.123', 'Float64'), 9, 'UTC'), '-[%H:%i:%S.]')))))" + }, + { + "print v1=format_timespan(time('29.09:00:05.12345'), 'dd.hh:mm:ss:FF')", + "SELECT concat(leftPad('29', 2, '0'), substring(toString(formatDateTime(toDateTime64(CAST('2538005.12345', 'Float64'), 9, 'UTC'), '.%H:%i:%S:')), 1, (length(toString(formatDateTime(toDateTime64(CAST('2538005.12345', 'Float64'), 9, 'UTC'), '.%H:%i:%S:'))) - position(reverse(toString(formatDateTime(toDateTime64(CAST('2538005.12345', 'Float64'), 9, 'UTC'), '.%H:%i:%S:'))), ':')) + 1), substring(substring(toString(toDateTime64(CAST('2538005.12345', 'Float64'), 9, 'UTC')), position(toString(toDateTime64(CAST('2538005.12345', 'Float64'), 9, 'UTC')), '.') + 1), 1, 2)) AS v1" + }, + { + "print v2=format_timespan(time('29.09:00:05.12345'), 'ddd.h:mm:ss [fffffff]');", + "SELECT concat(leftPad('29', 3, '0'), substring(toString(formatDateTime(toDateTime64(CAST('2538005.12345', 'Float64'), 9, 'UTC'), '.%H:%i:%S []')), 1, length(toString(formatDateTime(toDateTime64(CAST('2538005.12345', 'Float64'), 9, 'UTC'), '.%H:%i:%S []'))) - position(reverse(toString(formatDateTime(toDateTime64(CAST('2538005.12345', 'Float64'), 9, 'UTC'), '.%H:%i:%S []'))), ']')), substring(substring(toString(toDateTime64(CAST('2538005.12345', 'Float64'), 9, 'UTC')), position(toString(toDateTime64(CAST('2538005.12345', 'Float64'), 9, 'UTC')), '.') + 1), 1, 7), substring(toString(formatDateTime(toDateTime64(CAST('2538005.12345', 'Float64'), 9, 'UTC'), '.%H:%i:%S []')), position(toString(formatDateTime(toDateTime64(CAST('2538005.12345', 'Float64'), 9, 'UTC'), '.%H:%i:%S []')), ']'), length(toString(formatDateTime(toDateTime64(CAST('2538005.12345', 'Float64'), 9, 'UTC'), '.%H:%i:%S []'))))) AS v2" + }, + { + "print datetime_part('day', datetime(2017-10-30 01:02:03.7654321))", + "SELECT formatDateTime(parseDateTime64BestEffortOrNull('2017-10-30 01:02:03.7654321', 9, 'UTC'), '%e')" + }, + { + "print datetime_add('day',1,datetime(2017-10-30 01:02:03.7654321))", + "SELECT parseDateTime64BestEffortOrNull('2017-10-30 01:02:03.7654321', 9, 'UTC') + toIntervalDay(1)" + }, + { + "print totimespan(time(1d))", + "SELECT CAST('86400', 'Float64')" + }, + { + "print totimespan('0.01:34:23')", + "SELECT CAST('5663', 'Float64')" + }, + { + "print totimespan(time('-1:12:34'))", + "SELECT CAST('-4354', 'Float64')" + }, + { + "print totimespan(-1d)", + "SELECT -86400" + }, + { + "print totimespan('abc')", + "SELECT NULL" + }, + { + "print time(2)", + "SELECT CAST('172800', 'Float64')" + }, + { + "hits | project bin(datetime(EventTime), 1m)", + "SELECT toDateTime64(toInt64(toFloat64(if((toTypeName(EventTime) = 'Int64') OR (toTypeName(EventTime) = 'Int32') OR (toTypeName(EventTime) = 'Float64') OR (toTypeName(EventTime) = 'UInt32') OR (toTypeName(EventTime) = 'UInt64'), toDateTime64(EventTime, 9, 'UTC'), parseDateTime64BestEffortOrNull(CAST(EventTime, 'String'), 9, 'UTC'))) / 60) * 60, 9, 'UTC')\nFROM hits" + } + +}))); diff --git a/src/Parsers/tests/KQL/gtest_KQL_dynamicFunctions.cpp b/src/Parsers/tests/KQL/gtest_KQL_dynamicFunctions.cpp new file mode 100644 index 00000000000..93df5a6c55f --- /dev/null +++ b/src/Parsers/tests/KQL/gtest_KQL_dynamicFunctions.cpp @@ -0,0 +1,140 @@ +#include +#include + +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_Dynamic, ParserKQLTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "print t = array_sort_asc(dynamic([null, 'd', 'a', 'c', 'c']))", + "SELECT kql_array_sort_asc([NULL, 'd', 'a', 'c', 'c']).1 AS t" + }, + { + "print t = array_sort_asc(dynamic([4, 1, 3, 2]))", + "SELECT kql_array_sort_asc([4, 1, 3, 2]).1 AS t" + }, + { + "print t = array_sort_asc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))", + "SELECT kql_array_sort_asc(['b', 'a', 'c'], ['q', 'p', 'r']).1 AS t" + }, + { + "print t = array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , false)", + "SELECT kql_array_sort_asc(['d', NULL, 'a', 'c', 'c'], false).1 AS t" + }, + { + "print t = array_sort_asc( dynamic([null, 'd', null, null, 'a', 'c', 'c', null, null, null]) , false)", + "SELECT kql_array_sort_asc([NULL, 'd', NULL, NULL, 'a', 'c', 'c', NULL, NULL, NULL], false).1 AS t" + }, + { + "print t = array_sort_asc( dynamic([null, 'd', null, null, 'a', 'c', 'c', null, null, null]) , true)", + "SELECT kql_array_sort_asc([NULL, 'd', NULL, NULL, 'a', 'c', 'c', NULL, NULL, NULL], true).1 AS t" + }, + { + "print t = array_sort_asc( dynamic([null, 'd', null, null, 'a', 'c', 'c', null, null, null]))", + "SELECT kql_array_sort_asc([NULL, 'd', NULL, NULL, 'a', 'c', 'c', NULL, NULL, NULL]).1 AS t" + }, + { + "print t = array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']), 1 < 2)", + "SELECT kql_array_sort_asc(['d', NULL, 'a', 'c', 'c'], 1 < 2).1 AS t" + }, + { + "print t = array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2)", + "SELECT kql_array_sort_asc(['d', NULL, 'a', 'c', 'c'], 1 > 2).1 AS t" + }, + { + "print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]), false)", + "SELECT kql_array_sort_asc([2, 1, NULL, 3], [20, 10, 40, 30], false).1 AS t" + }, + { + "print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]), 1 > 2)", + "SELECT kql_array_sort_asc([2, 1, NULL, 3], [20, 10, 40, 30], 1 > 2).1 AS t" + }, + { + "print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]), true)", + "SELECT kql_array_sort_asc([2, 1, NULL, 3], [20, 10, 40, 30], true).1 AS t" + }, + { + "print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]), 1 < 2)", + "SELECT kql_array_sort_asc([2, 1, NULL, 3], [20, 10, 40, 30], 1 < 2).1 AS t" + }, + { + "print t = array_sort_desc(dynamic([null, 'd', 'a', 'c', 'c']))", + "SELECT kql_array_sort_desc([NULL, 'd', 'a', 'c', 'c']).1 AS t" + }, + { + "print t = array_sort_desc(dynamic([4, 1, 3, 2]))", + "SELECT kql_array_sort_desc([4, 1, 3, 2]).1 AS t" + }, + { + "print t = array_sort_desc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))", + "SELECT kql_array_sort_desc(['b', 'a', 'c'], ['q', 'p', 'r']).1 AS t" + }, + { + "print array_sort_desc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))", + "SELECT kql_array_sort_desc(['b', 'a', 'c'], ['q', 'p', 'r'])" + }, + { + "print t = array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , false)", + "SELECT kql_array_sort_desc(['d', NULL, 'a', 'c', 'c'], false).1 AS t" + }, + { + "print array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]))[0]", + "SELECT tupleElement(kql_array_sort_asc([2, 1, NULL, 3], [20, 10, 40, 30]), if(0 >= 0, 0 + 1, 0))" + }, + { + "print (t) = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]))", + "SELECT kql_array_sort_asc([2, 1, NULL, 3], [20, 10, 40, 30]).1 AS t" + }, + { + "print (t,w) = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]))", + "SELECT\n kql_array_sort_asc([2, 1, NULL, 3], [20, 10, 40, 30]).1 AS t,\n kql_array_sort_asc([2, 1, NULL, 3], [20, 10, 40, 30]).2 AS w" + }, + { + "print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30])),y=5", + "SELECT\n kql_array_sort_asc([2, 1, NULL, 3], [20, 10, 40, 30]).1 AS t,\n 5 AS y" + }, + { + "print 5, (t) = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30])),y=5", + "SELECT\n 5,\n kql_array_sort_asc([2, 1, NULL, 3], [20, 10, 40, 30]).1 AS t,\n 5 AS y" + }, + { + "print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30])),w = array_sort_asc(dynamic([2, 1, 3]))", + "SELECT\n kql_array_sort_asc([2, 1, NULL, 3], [20, 10, 40, 30]).1 AS t,\n kql_array_sort_asc([2, 1, 3]).1 AS w" + }, + { + "print A[0]", + "SELECT A[if(0 >= 0, 0 + 1, 0)]" + }, + { + "print A[0][1]", + "SELECT (A[if(0 >= 0, 0 + 1, 0)])[if(1 >= 0, 1 + 1, 1)]" + }, + { + "print dynamic([[1,2,3,4,5],[20,30]])[0]", + "SELECT [[1, 2, 3, 4, 5], [20, 30]][if(0 >= 0, 0 + 1, 0)]" + }, + { + "print dynamic([[1,2,3,4,5],[20,30]])[1][1]", + "SELECT ([[1, 2, 3, 4, 5], [20, 30]][if(1 >= 0, 1 + 1, 1)])[if(1 >= 0, 1 + 1, 1)]" + }, + { + "print A[B[1]]", + "SELECT A[if((B[if(1 >= 0, 1 + 1, 1)]) >= 0, (B[if(1 >= 0, 1 + 1, 1)]) + 1, B[if(1 >= 0, 1 + 1, 1)])]" + }, + { + "print A[strlen('a')-1]", + "SELECT A[if((lengthUTF8('a') - 1) >= 0, (lengthUTF8('a') - 1) + 1, lengthUTF8('a') - 1)]" + }, + { + "print strlen(A[0])", + "SELECT lengthUTF8(A[if(0 >= 0, 0 + 1, 0)])" + }, + { + "print repeat(1, 3)", + "SELECT if(3 < 0, [NULL], arrayWithConstant(abs(3), 1))" + }, + { + "print repeat(1, -3)", + "SELECT if(-3 < 0, [NULL], arrayWithConstant(abs(-3), 1))" + } + }))); diff --git a/src/Parsers/tests/KQL/gtest_KQL_operator_in_sql.cpp b/src/Parsers/tests/KQL/gtest_KQL_operator_in_sql.cpp new file mode 100644 index 00000000000..a35287f471b --- /dev/null +++ b/src/Parsers/tests/KQL/gtest_KQL_operator_in_sql.cpp @@ -0,0 +1,66 @@ +#include + +#include +#include + +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_operator_in_sql, ParserKQLTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "select * from kql($$Customers | where FirstName !in ('Peter', 'Latoya')$$)", + "SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE FirstName NOT IN ('Peter', 'Latoya')\n)" + }, + { + "select * from kql($$Customers | where FirstName !contains 'Pet'$$);", + "SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE NOT (FirstName ILIKE '%Pet%')\n)" + }, + { + "select * from kql($$Customers | where FirstName !contains_cs 'Pet'$$);", + "SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE NOT (FirstName LIKE '%Pet%')\n)" + }, + { + "select * from kql($$Customers | where FirstName !endswith 'ter'$$);", + "SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE NOT (FirstName ILIKE '%ter')\n)" + }, + { + "select * from kql($$Customers | where FirstName !endswith_cs 'ter'$$);" + "SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE NOT endsWith(FirstName, 'ter')\n)" + }, + { + "select * from kql($$Customers | where FirstName != 'Peter'$$);", + "SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE FirstName != 'Peter'\n)" + }, + { + "select * from kql($$Customers | where FirstName !has 'Peter'$$);", + "SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE NOT hasTokenCaseInsensitive(FirstName, 'Peter')\n)" + }, + { + "select * from kql($$Customers | where FirstName !has_cs 'peter'$$);", + "SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE NOT hasToken(FirstName, 'peter')\n)" + }, + { + "select * from kql($$Customers | where FirstName !hasprefix 'Peter'$$);", + "SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE (NOT (FirstName ILIKE 'Peter%')) AND (NOT (FirstName ILIKE '% Peter%'))\n)" + }, + { + "select * from kql($$Customers | where FirstName !hasprefix_cs 'Peter'$$);", + "SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE (NOT startsWith(FirstName, 'Peter')) AND (NOT (FirstName LIKE '% Peter%'))\n)" + }, + { + "select * from kql($$Customers | where FirstName !hassuffix 'Peter'$$);", + "SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE (NOT (FirstName ILIKE '%Peter')) AND (NOT (FirstName ILIKE '%Peter %'))\n)" + }, + { + "select * from kql($$Customers | where FirstName !hassuffix_cs 'Peter'$$);", + "SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE (NOT endsWith(FirstName, 'Peter')) AND (NOT (FirstName LIKE '%Peter %'))\n)" + }, + { + "select * from kql($$Customers | where FirstName !startswith 'Peter'$$);", + "SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE NOT (FirstName ILIKE 'Peter%')\n)" + }, + { + "select * from kql($$Customers | where FirstName !startswith_cs 'Peter'$$);", + "SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE NOT startsWith(FirstName, 'Peter')\n)" + } +}))); diff --git a/src/Parsers/tests/KQL/gtest_KQL_test_subquery.cpp b/src/Parsers/tests/KQL/gtest_KQL_test_subquery.cpp new file mode 100644 index 00000000000..619f634670a --- /dev/null +++ b/src/Parsers/tests/KQL/gtest_KQL_test_subquery.cpp @@ -0,0 +1,109 @@ +#include + +#include + +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_TestSubquery, ParserKQLTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !in ('Peter', 'Latoya')));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE FirstName NOT IN ('Peter', 'Latoya')\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName, Age | where Age !in (28, 29)));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE Age NOT IN (28, 29)\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !contains 'ste'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE NOT (FirstName ILIKE '%ste%')\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !contains_cs 'Ste'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE NOT (FirstName LIKE '%Ste%')\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !contains_cs 'ste'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE NOT (FirstName LIKE '%ste%')\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !endswith 'ore'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE NOT (FirstName ILIKE '%ore')\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !endswith_cs 'Ore'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE NOT endsWith(FirstName, 'Ore')\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName != 'Theodore'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE FirstName != 'Theodore'\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !~ 'theodore'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE lower(FirstName) != lower('theodore')\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !has 'Peter'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE NOT hasTokenCaseInsensitive(FirstName, 'Peter')\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !has_cs 'Peter'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE NOT hasToken(FirstName, 'Peter')\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !hasprefix 'Peter'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE (NOT (FirstName ILIKE 'Peter%')) AND (NOT (FirstName ILIKE '% Peter%'))\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !hasprefix_cs 'Peter'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE (NOT startsWith(FirstName, 'Peter')) AND (NOT (FirstName LIKE '% Peter%'))\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !hassuffix 'Peter'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE (NOT (FirstName ILIKE '%Peter')) AND (NOT (FirstName ILIKE '%Peter %'))\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !hassuffix_cs 'Peter'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE (NOT endsWith(FirstName, 'Peter')) AND (NOT (FirstName LIKE '%Peter %'))\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !startswith 'Peter'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE NOT (FirstName ILIKE 'Peter%')\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName !startswith_cs 'Peter'));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE NOT startsWith(FirstName, 'Peter')\n)" + }, + { + "Customers | where FirstName !in~ ((Customers | project FirstName | where FirstName !in~ ('peter', 'apple')));", + "SELECT *\nFROM Customers\nWHERE lower(FirstName) NOT IN (\n SELECT lower(FirstName)\n FROM Customers\n WHERE lower(FirstName) NOT IN (lower('peter'), lower('apple'))\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName | where FirstName in~ ('peter', 'apple')));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n WHERE lower(FirstName) IN (lower('peter'), lower('apple'))\n)" + }, + { + "Customers | where substring(FirstName,0,3) in~ ((Customers | project substring(FirstName,0,3) | where FirstName in~ ('peter', 'apple')));", + "SELECT *\nFROM Customers\nWHERE lower(if(toInt64(length(FirstName)) <= 0, '', substr(FirstName, (((0 % toInt64(length(FirstName))) + toInt64(length(FirstName))) % toInt64(length(FirstName))) + 1, 3))) IN (\n SELECT lower(if(toInt64(length(FirstName)) <= 0, '', substr(FirstName, (((0 % toInt64(length(FirstName))) + toInt64(length(FirstName))) % toInt64(length(FirstName))) + 1, 3)))\n FROM Customers\n WHERE lower(FirstName) IN (lower('peter'), lower('apple'))\n)" + }, + { + "Customers | where FirstName in~ ((Customers | where FirstName !in~ ('peter', 'apple')| project FirstName));", + "SELECT *\nFROM Customers\nWHERE lower(FirstName) IN (\n SELECT lower(FirstName)\n FROM Customers\n WHERE lower(FirstName) NOT IN (lower('peter'), lower('apple'))\n)" + }, + { + "Customers | where FirstName in ((Customers | project FirstName, LastName, Age));", + "SELECT *\nFROM Customers\nWHERE FirstName IN (\n SELECT FirstName\n FROM Customers\n)" + }, + { + "Customers | where FirstName in~ ((Customers | project FirstName, LastName, Age|where Age <30));", + "SELECT *\nFROM Customers\nWHERE lower(FirstName) IN (\n SELECT lower(FirstName)\n FROM Customers\n WHERE Age < 30\n)" + }, + { + "Customers | where FirstName !in ((Customers | project FirstName, LastName, Age |where Age <30 ));", + "SELECT *\nFROM Customers\nWHERE FirstName NOT IN (\n SELECT FirstName\n FROM Customers\n WHERE Age < 30\n)" + }, + { + "Customers | where FirstName !in~ ((Customers | project FirstName, LastName, Age |where Age <30));", + "SELECT *\nFROM Customers\nWHERE lower(FirstName) NOT IN (\n SELECT lower(FirstName)\n FROM Customers\n WHERE Age < 30\n)" + } +}))); diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 3a90256f262..556743e70ec 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -18,6 +18,7 @@ #include #include #include +#include "gtest_common.h" #include namespace @@ -26,13 +27,6 @@ using namespace DB; using namespace std::literals; } - -struct ParserTestCase -{ - const std::string_view input_text; - const char * expected_ast = nullptr; -}; - std::ostream & operator<<(std::ostream & ostr, const std::shared_ptr parser) { return ostr << "Parser: " << parser->getName(); @@ -47,9 +41,6 @@ std::ostream & operator<<(std::ostream & ostr, const ParserTestCase & test_case) return ostr << "ParserTestCase input: " << input_text; } -class ParserTest : public ::testing::TestWithParam, ParserTestCase>> -{}; - TEST_P(ParserTest, parseQuery) { const auto & parser = std::get<0>(GetParam()); @@ -307,7 +298,7 @@ INSTANTIATE_TEST_SUITE_P(ParserAttachUserQuery, ParserTest, } }))); -INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserTest, +INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserKQLTest, ::testing::Combine( ::testing::Values(std::make_shared()), ::testing::ValuesIn(std::initializer_list{ @@ -339,10 +330,6 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserTest, "Customers | project FirstName,LastName,Occupation | take 3 | project FirstName,LastName", "SELECT\n FirstName,\n LastName\nFROM\n(\n SELECT\n FirstName,\n LastName,\n Occupation\n FROM Customers\n LIMIT 3\n)" }, - { - "Customers | project FirstName,LastName,Occupation | take 3 | project FirstName,LastName,Education", - "SELECT\n FirstName,\n LastName,\n Education\nFROM\n(\n SELECT\n FirstName,\n LastName,\n Occupation\n FROM Customers\n LIMIT 3\n)" - }, { "Customers | sort by FirstName desc", "SELECT *\nFROM Customers\nORDER BY FirstName DESC" @@ -425,23 +412,23 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserTest, }, { "Customers |summarize count() by Occupation", - "SELECT\n count(),\n Occupation\nFROM Customers\nGROUP BY Occupation" + "SELECT\n Occupation,\n count() AS count_\nFROM Customers\nGROUP BY Occupation" }, { "Customers|summarize sum(Age) by Occupation", - "SELECT\n sum(Age),\n Occupation\nFROM Customers\nGROUP BY Occupation" + "SELECT\n Occupation,\n sum(Age) AS sum_Age\nFROM Customers\nGROUP BY Occupation" }, { "Customers|summarize avg(Age) by Occupation", - "SELECT\n avg(Age),\n Occupation\nFROM Customers\nGROUP BY Occupation" + "SELECT\n Occupation,\n avg(Age) AS avg_Age\nFROM Customers\nGROUP BY Occupation" }, { "Customers|summarize min(Age) by Occupation", - "SELECT\n min(Age),\n Occupation\nFROM Customers\nGROUP BY Occupation" + "SELECT\n Occupation,\n min(Age) AS min_Age\nFROM Customers\nGROUP BY Occupation" }, { "Customers |summarize max(Age) by Occupation", - "SELECT\n max(Age),\n Occupation\nFROM Customers\nGROUP BY Occupation" + "SELECT\n Occupation,\n max(Age) AS max_Age\nFROM Customers\nGROUP BY Occupation" }, { "Customers | where FirstName contains 'pet'", @@ -486,6 +473,146 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserTest, { "Customers | where FirstName !startswith 'pet'", "SELECT *\nFROM Customers\nWHERE NOT (FirstName ILIKE 'pet%')" + }, + { + "Customers | where Age in ((Customers|project Age|where Age < 30))", + "SELECT *\nFROM Customers\nWHERE Age IN (\n SELECT Age\n FROM Customers\n WHERE Age < 30\n)" + }, + { + "Customers|where Occupation has_any ('Skilled','abcd')", + "SELECT *\nFROM Customers\nWHERE hasTokenCaseInsensitive(Occupation, 'Skilled') OR hasTokenCaseInsensitive(Occupation, 'abcd')" + }, + { + "Customers|where Occupation has_all ('Skilled','abcd')", + "SELECT *\nFROM Customers\nWHERE hasTokenCaseInsensitive(Occupation, 'Skilled') AND hasTokenCaseInsensitive(Occupation, 'abcd')" + }, + { + "Customers|where Occupation has_all (strcat('Skill','ed'),'Manual')", + "SELECT *\nFROM Customers\nWHERE hasTokenCaseInsensitive(Occupation, concat('Skill', 'ed')) AND hasTokenCaseInsensitive(Occupation, 'Manual')" + }, + { + "Customers | where Occupation == strcat('Pro','fessional') | take 1", + "SELECT *\nFROM Customers\nWHERE Occupation = concat('Pro', 'fessional')\nLIMIT 1" + }, + { + "Customers | project countof('The cat sat on the mat', 'at')", + "SELECT countSubstrings('The cat sat on the mat', 'at')\nFROM Customers" + }, + { + "Customers | project countof('The cat sat on the mat', 'at', 'normal')", + "SELECT countSubstrings('The cat sat on the mat', 'at')\nFROM Customers" + }, + { + "Customers | project countof('The cat sat on the mat', 'at', 'regex')", + "SELECT countMatches('The cat sat on the mat', 'at')\nFROM Customers" + }, + { + "Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 0, 'The price of PINEAPPLE ice cream is 10')", + "SELECT extract('The price of PINEAPPLE ice cream is 10', '\\b[A-Z]+\\b.+\\b\\\\d+')\nFROM Customers" + }, + { + "Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 1, 'The price of PINEAPPLE ice cream is 20')", + "SELECT extract('The price of PINEAPPLE ice cream is 20', '\\b[A-Z]+\\b')\nFROM Customers" + }, + { + "Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 30')", + "SELECT extract('The price of PINEAPPLE ice cream is 30', '\\b\\\\d+')\nFROM Customers" + }, + { + "Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 40', typeof(int))", + "SELECT accurateCastOrNull(extract('The price of PINEAPPLE ice cream is 40', '\\b\\\\d+'), 'Int32')\nFROM Customers" + }, + { + "Customers | project extract_all('(\\w)(\\w+)(\\w)','The price of PINEAPPLE ice cream is 50')", + "SELECT extractAllGroups('The price of PINEAPPLE ice cream is 50', '(\\\\w)(\\\\w+)(\\\\w)')\nFROM Customers" + }, + { + " Customers | project split('aa_bb', '_')", + "SELECT if(empty('_'), splitByString(' ', 'aa_bb'), splitByString('_', 'aa_bb'))\nFROM Customers" + }, + { + "Customers | project split('aaa_bbb_ccc', '_', 1)", + "SELECT multiIf((length(if(empty('_'), splitByString(' ', 'aaa_bbb_ccc'), splitByString('_', 'aaa_bbb_ccc'))) >= 2) AND (2 > 0), arrayPushBack([], if(empty('_'), splitByString(' ', 'aaa_bbb_ccc'), splitByString('_', 'aaa_bbb_ccc'))[2]), 2 = 0, if(empty('_'), splitByString(' ', 'aaa_bbb_ccc'), splitByString('_', 'aaa_bbb_ccc')), arrayPushBack([], NULL[1]))\nFROM Customers" + }, + { + "Customers | project strcat_delim('-', '1', '2', 'A')", + "SELECT concat('1', '-', '2', '-', 'A')\nFROM Customers" + }, + { + "Customers | project indexof('abcdefg','cde')", + "SELECT position('abcdefg', 'cde', 1) - 1\nFROM Customers" + }, + { + "Customers | project indexof('abcdefg','cde', 2) ", + "SELECT position('abcdefg', 'cde', 3) - 1\nFROM Customers" + }, + { + "print x=1, s=strcat('Hello', ', ', 'World!')", + "SELECT\n 1 AS x,\n concat('Hello', ', ', 'World!') AS s" + }, + { + "print parse_urlquery('https://john:123@google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment')", + "SELECT concat('{', concat('\"Query Parameters\":', concat('{\"', replace(replace(if(position('https://john:123@google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment', '?') > 0, queryString('https://john:123@google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment'), 'https://john:123@google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment'), '=', '\":\"'), '&', '\",\"'), '\"}')), '}')" + }, + { + "print strcmp('a','b')", + "SELECT multiIf('a' = 'b', 0, 'a' < 'b', -1, 1)" + }, + { + "print parse_url('https://john:123@google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment')", + "SELECT concat('{', concat('\"Scheme\":\"', protocol('https://john:123@google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment'), '\"'), ',', concat('\"Host\":\"', domain('https://john:123@google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment'), '\"'), ',', concat('\"Port\":\"', toString(port('https://john:123@google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment')), '\"'), ',', concat('\"Path\":\"', path('https://john:123@google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment'), '\"'), ',', concat('\"Username\":\"', splitByChar(':', splitByChar('@', netloc('https://john:123@google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment'))[1])[1], '\"'), ',', concat('\"Password\":\"', splitByChar(':', splitByChar('@', netloc('https://john:123@google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment'))[1])[2], '\"'), ',', concat('\"Query Parameters\":', concat('{\"', replace(replace(queryString('https://john:123@google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment'), '=', '\":\"'), '&', '\",\"'), '\"}')), ',', concat('\"Fragment\":\"', fragment('https://john:123@google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment'), '\"'), '}')" + }, + { + "Customers | summarize t = make_list(FirstName) by FirstName", + "SELECT\n FirstName,\n groupArrayIf(FirstName, FirstName IS NOT NULL) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "Customers | summarize t = make_list(FirstName, 10) by FirstName", + "SELECT\n FirstName,\n groupArrayIf(10)(FirstName, FirstName IS NOT NULL) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "Customers | summarize t = make_list_if(FirstName, Age > 10) by FirstName", + "SELECT\n FirstName,\n groupArrayIf(FirstName, Age > 10) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "Customers | summarize t = make_list_if(FirstName, Age > 10, 10) by FirstName", + "SELECT\n FirstName,\n groupArrayIf(10)(FirstName, Age > 10) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "Customers | summarize t = make_set(FirstName) by FirstName", + "SELECT\n FirstName,\n groupUniqArray(FirstName) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "Customers | summarize t = make_set(FirstName, 10) by FirstName", + "SELECT\n FirstName,\n groupUniqArray(10)(FirstName) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "Customers | summarize t = make_set_if(FirstName, Age > 10) by FirstName", + "SELECT\n FirstName,\n groupUniqArrayIf(FirstName, Age > 10) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "Customers | summarize t = make_set_if(FirstName, Age > 10, 10) by FirstName", + "SELECT\n FirstName,\n groupUniqArrayIf(10)(FirstName, Age > 10) AS t\nFROM Customers\nGROUP BY FirstName" + }, + { + "print output = dynamic([1, 2, 3])", + "SELECT [1, 2, 3] AS output" + }, + { + "print output = dynamic(['a', 'b', 'c'])", + "SELECT ['a', 'b', 'c'] AS output" + }, + { + "T | extend duration = endTime - startTime", + "SELECT\n * EXCEPT duration,\n endTime - startTime AS duration\nFROM T" + }, + { + "T |project endTime, startTime | extend duration = endTime - startTime", + "SELECT\n * EXCEPT duration,\n endTime - startTime AS duration\nFROM\n(\n SELECT\n endTime,\n startTime\n FROM T\n)" + }, + { + "T | extend c =c*2, b-a, d = a + b, a*b", + "SELECT\n * EXCEPT c EXCEPT d,\n c * 2 AS c,\n b - a AS Column1,\n a + b AS d,\n a * b AS Column2\nFROM T" } }))); diff --git a/src/Parsers/tests/gtest_common.cpp b/src/Parsers/tests/gtest_common.cpp new file mode 100644 index 00000000000..7710df94644 --- /dev/null +++ b/src/Parsers/tests/gtest_common.cpp @@ -0,0 +1,81 @@ +#include "gtest_common.h" + +#include +#include +#include +#include +#include + +#include + +#include + +namespace +{ +using namespace DB; +using namespace std::literals; +} + + +TEST_P(ParserRegexTest, parseQuery) +{ + const auto & parser = std::get<0>(GetParam()); + const auto & [input_text, expected_ast] = std::get<1>(GetParam()); + + ASSERT_TRUE(parser); + ASSERT_TRUE(expected_ast); + + DB::ASTPtr ast; + ASSERT_NO_THROW(ast = parseQuery(*parser, input_text.begin(), input_text.end(), 0, 0)); + DB::WriteBufferFromOwnString buf; + formatAST(*ast->clone(), buf, false, false); + EXPECT_THAT(buf.str(), ::testing::MatchesRegex(expected_ast)); +} + +TEST_P(ParserKQLTest, parseKQLQuery) +{ + const auto & parser = std::get<0>(GetParam()); + const auto & [input_text, expected_ast] = std::get<1>(GetParam()); + + ASSERT_NE(nullptr, parser); + + if (expected_ast) + { + if (std::string(expected_ast).starts_with("throws")) + { + EXPECT_THROW(parseKQLQuery(*parser, input_text.begin(), input_text.end(), 0, 0), DB::Exception); + } + else + { + DB::ASTPtr ast; + ASSERT_NO_THROW(ast = parseKQLQuery(*parser, input_text.begin(), input_text.end(), 0, 0)); + if (std::string("CREATE USER or ALTER USER query") != parser->getName() + && std::string("ATTACH access entity query") != parser->getName()) + { + DB::WriteBufferFromOwnString buf; + formatAST(*ast->clone(), buf, false, false); + String formatted_ast = buf.str(); + EXPECT_EQ(expected_ast, formatted_ast); + } + else + { + if (input_text.starts_with("ATTACH")) + { + auto salt = (dynamic_cast(ast.get())->auth_data)->getSalt().value_or(""); + EXPECT_TRUE(std::regex_match(salt, std::regex(expected_ast))); + } + else + { + DB::WriteBufferFromOwnString buf; + formatAST(*ast->clone(), buf, false, false); + String formatted_ast = buf.str(); + EXPECT_TRUE(std::regex_match(formatted_ast, std::regex(expected_ast))); + } + } + } + } + else + { + ASSERT_THROW(parseKQLQuery(*parser, input_text.begin(), input_text.end(), 0, 0), DB::Exception); + } +} diff --git a/src/Parsers/tests/gtest_common.h b/src/Parsers/tests/gtest_common.h new file mode 100644 index 00000000000..0c6b21217fb --- /dev/null +++ b/src/Parsers/tests/gtest_common.h @@ -0,0 +1,20 @@ +#pragma once + +#include + +#include + +#include + +struct ParserTestCase +{ + const std::string_view input_text; + const char * expected_ast = nullptr; +}; + +class ParserTest : public ::testing::TestWithParam, ParserTestCase>> +{}; +class ParserKQLTest : public ::testing::TestWithParam, ParserTestCase>> +{}; +class ParserRegexTest : public ::testing::TestWithParam, ParserTestCase>> +{}; diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index e011fb8ecbe..b36449ab568 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -17,7 +17,7 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const settings.remove_redundant_sorting = from.query_plan_remove_redundant_sorting; settings.aggregate_partitions_independently = from.allow_aggregate_partitions_independently; settings.remove_redundant_distinct = from.query_plan_remove_redundant_distinct; - settings.optimize_projection = from.optimize_use_projections && from.query_plan_optimize_projection; + settings.optimize_projection = from.optimize_use_projections; settings.force_use_projection = settings.optimize_projection && from.force_optimize_projection; settings.optimize_use_implicit_projections = settings.optimize_projection && from.optimize_use_implicit_projections; return settings; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a1fb4d0d2d1..25a7fb5737f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6324,93 +6324,6 @@ bool MergeTreeData::mayBenefitFromIndexForIn( using PartitionIdToMaxBlock = std::unordered_map; -static void selectBestProjection( - const MergeTreeDataSelectExecutor & reader, - const StorageSnapshotPtr & storage_snapshot, - const SelectQueryInfo & query_info, - const ActionDAGNodes & added_filter_nodes, - const Names & required_columns, - ProjectionCandidate & candidate, - ContextPtr query_context, - std::shared_ptr max_added_blocks, - const Settings & settings, - const MergeTreeData::DataPartsVector & parts, - ProjectionCandidate *& selected_candidate, - size_t & min_sum_marks) -{ - MergeTreeData::DataPartsVector projection_parts; - MergeTreeData::DataPartsVector normal_parts; - for (const auto & part : parts) - { - const auto & projections = part->getProjectionParts(); - auto it = projections.find(candidate.desc->name); - if (it != projections.end()) - projection_parts.push_back(it->second); - else - normal_parts.push_back(part); - } - - if (projection_parts.empty()) - return; - - auto projection_result_ptr = reader.estimateNumMarksToRead( - projection_parts, - candidate.prewhere_info, - candidate.required_columns, - storage_snapshot->metadata, - candidate.desc->metadata, - query_info, - added_filter_nodes, - query_context, - settings.max_threads, - max_added_blocks); - - if (projection_result_ptr->error()) - return; - - auto sum_marks = projection_result_ptr->marks(); - if (normal_parts.empty()) - { - // All parts are projection parts which allows us to use in_order_optimization. - // TODO It might be better to use a complete projection even with more marks to read. - candidate.complete = true; - } - else - { - auto normal_result_ptr = reader.estimateNumMarksToRead( - normal_parts, - query_info.prewhere_info, - required_columns, - storage_snapshot->metadata, - storage_snapshot->metadata, - query_info, // TODO syntax_analysis_result set in index - added_filter_nodes, - query_context, - settings.max_threads, - max_added_blocks); - - if (normal_result_ptr->error()) - return; - - if (normal_result_ptr->marks() == 0) - candidate.complete = true; - else - { - sum_marks += normal_result_ptr->marks(); - candidate.merge_tree_normal_select_result_ptr = normal_result_ptr; - } - } - candidate.merge_tree_projection_select_result_ptr = projection_result_ptr; - - // We choose the projection with least sum_marks to read. - if (sum_marks < min_sum_marks) - { - selected_candidate = &candidate; - min_sum_marks = sum_marks; - } -} - - Block MergeTreeData::getMinMaxCountProjectionBlock( const StorageMetadataPtr & metadata_snapshot, const Names & required_columns, @@ -6672,483 +6585,10 @@ ActionDAGNodes MergeTreeData::getFiltersForPrimaryKeyAnalysis(const InterpreterS return filter_nodes; } - -std::optional MergeTreeData::getQueryProcessingStageWithAggregateProjection( - ContextPtr query_context, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info) const -{ - const auto & metadata_snapshot = storage_snapshot->metadata; - const auto & settings = query_context->getSettingsRef(); - - if (settings.query_plan_optimize_projection) - return std::nullopt; - - /// TODO: Analyzer syntax analyzer result - if (!query_info.syntax_analyzer_result) - return std::nullopt; - - if (!settings.optimize_use_projections || query_info.ignore_projections || query_info.is_projection_query - || settings.aggregate_functions_null_for_empty /* projections don't work correctly with this setting */) - return std::nullopt; - - // Currently projections don't support parallel replicas reading yet. - if (settings.parallel_replicas_count > 1 || settings.max_parallel_replicas > 1) - return std::nullopt; - - /// Cannot use projections in case of additional filter. - if (query_info.additional_filter_ast) - return std::nullopt; - - auto query_ptr = query_info.query; - auto original_query_ptr = query_info.original_query; - - auto * select_query = query_ptr->as(); - auto * original_select_query = original_query_ptr->as(); - - if (!original_select_query || !select_query) - return std::nullopt; - - // Currently projections don't support final yet. - if (select_query->final() || original_select_query->final()) - return std::nullopt; - - // Currently projections don't support sample yet. - if (original_select_query->sampleSize()) - return std::nullopt; - - // Currently projection don't support deduplication when moving parts between shards. - if (settings.allow_experimental_query_deduplication) - return std::nullopt; - - // Currently projections don't support ARRAY JOIN yet. - if (original_select_query->arrayJoinExpressionList().first) - return std::nullopt; - - // In order to properly analyze joins, aliases should be recognized. However, aliases get lost during projection analysis. - // Let's disable projection if there are any JOIN clauses. - // TODO: We need a better identifier resolution mechanism for projection analysis. - if (original_select_query->hasJoin()) - return std::nullopt; - - // INTERPOLATE expressions may include aliases, so aliases should be preserved - if (original_select_query->interpolate() && !original_select_query->interpolate()->children.empty()) - return std::nullopt; - - // Projections don't support grouping sets yet. - if (original_select_query->group_by_with_grouping_sets - || original_select_query->group_by_with_totals - || original_select_query->group_by_with_rollup - || original_select_query->group_by_with_cube) - return std::nullopt; - - auto query_options = SelectQueryOptions( - QueryProcessingStage::WithMergeableState, - /* depth */ 1, - /* is_subquery_= */ true - ).ignoreProjections().ignoreAlias(); - - InterpreterSelectQuery select( - original_query_ptr, - query_context, - query_options, - query_info.prepared_sets); - - const auto & analysis_result = select.getAnalysisResult(); - query_info.prepared_sets = select.getQueryAnalyzer()->getPreparedSets(); - - /// For PK analysis - ActionDAGNodes added_filter_nodes = MergeTreeData::getFiltersForPrimaryKeyAnalysis(select); - - bool can_use_aggregate_projection = true; - /// If the first stage of the query pipeline is more complex than Aggregating - Expression - Filter - ReadFromStorage, - /// we cannot use aggregate projection. - if (analysis_result.join != nullptr || analysis_result.array_join != nullptr) - can_use_aggregate_projection = false; - - /// Check if all needed columns can be provided by some aggregate projection. Here we also try - /// to find expression matches. For example, suppose an aggregate projection contains a column - /// named sum(x) and the given query also has an expression called sum(x), it's a match. This is - /// why we need to ignore all aliases during projection creation and the above query planning. - /// It's also worth noting that, sqrt(sum(x)) will also work because we can treat sum(x) as a - /// required column. - - /// The ownership of ProjectionDescription is hold in metadata_snapshot which lives along with - /// InterpreterSelect, thus we can store the raw pointer here. - std::vector candidates; - NameSet keys; - std::unordered_map key_name_pos_map; - size_t pos = 0; - for (const auto & desc : select.getQueryAnalyzer()->aggregationKeys()) - { - keys.insert(desc.name); - key_name_pos_map.insert({desc.name, pos++}); - } - auto actions_settings = ExpressionActionsSettings::fromSettings(settings, CompileExpressions::yes); - - // All required columns should be provided by either current projection or previous actions - // Let's traverse backward to finish the check. - // TODO what if there is a column with name sum(x) and an aggregate sum(x)? - auto rewrite_before_where = - [&](ProjectionCandidate & candidate, const ProjectionDescription & projection, - NameSet & required_columns, const Block & source_block, const Block & aggregates) - { - if (analysis_result.before_where) - { - candidate.where_column_name = analysis_result.where_column_name; - candidate.remove_where_filter = !required_columns.contains(analysis_result.where_column_name); - candidate.before_where = analysis_result.before_where->clone(); - - auto new_required_columns = candidate.before_where->foldActionsByProjection( - required_columns, - projection.sample_block_for_keys, - candidate.where_column_name); - if (new_required_columns.empty() && !required_columns.empty()) - return false; - required_columns = std::move(new_required_columns); - candidate.before_where->addAggregatesViaProjection(aggregates); - } - - if (analysis_result.prewhere_info) - { - candidate.prewhere_info = analysis_result.prewhere_info->clone(); - - auto prewhere_actions = candidate.prewhere_info->prewhere_actions->clone(); - auto prewhere_required_columns = required_columns; - // required_columns should not contain columns generated by prewhere - for (const auto & column : prewhere_actions->getResultColumns()) - required_columns.erase(column.name); - - { - // prewhere_action should not add missing keys. - auto new_prewhere_required_columns = prewhere_actions->foldActionsByProjection( - prewhere_required_columns, projection.sample_block_for_keys, candidate.prewhere_info->prewhere_column_name, false); - if (new_prewhere_required_columns.empty() && !prewhere_required_columns.empty()) - return false; - prewhere_required_columns = std::move(new_prewhere_required_columns); - candidate.prewhere_info->prewhere_actions = prewhere_actions; - } - - if (candidate.prewhere_info->row_level_filter) - { - auto row_level_filter_actions = candidate.prewhere_info->row_level_filter->clone(); - // row_level_filter_action should not add missing keys. - auto new_prewhere_required_columns = row_level_filter_actions->foldActionsByProjection( - prewhere_required_columns, projection.sample_block_for_keys, candidate.prewhere_info->row_level_column_name, false); - if (new_prewhere_required_columns.empty() && !prewhere_required_columns.empty()) - return false; - prewhere_required_columns = std::move(new_prewhere_required_columns); - candidate.prewhere_info->row_level_filter = row_level_filter_actions; - } - - required_columns.insert(prewhere_required_columns.begin(), prewhere_required_columns.end()); - } - - bool match = true; - for (const auto & column : required_columns) - { - /// There are still missing columns, fail to match - if (!source_block.has(column)) - { - match = false; - break; - } - } - return match; - }; - - auto virtual_block = getSampleBlockWithVirtualColumns(); - auto add_projection_candidate = [&](const ProjectionDescription & projection, bool minmax_count_projection = false) - { - ProjectionCandidate candidate{}; - candidate.desc = &projection; - candidate.context = select.getContext(); - - auto sample_block = projection.sample_block; - auto sample_block_for_keys = projection.sample_block_for_keys; - for (const auto & column : virtual_block) - { - sample_block.insertUnique(column); - sample_block_for_keys.insertUnique(column); - } - - // If optimize_aggregation_in_order = true, we need additional information to transform the projection's pipeline. - auto attach_aggregation_in_order_info = [&]() - { - for (const auto & desc : select.getQueryAnalyzer()->aggregationKeys()) - { - const String & key = desc.name; - auto actions_dag = analysis_result.before_aggregation->clone(); - actions_dag->foldActionsByProjection({key}, sample_block_for_keys); - candidate.group_by_elements_actions.emplace_back(std::make_shared(actions_dag, actions_settings)); - candidate.group_by_elements_order_descr.emplace_back(key, 1, 1); - } - }; - - if (projection.type == ProjectionDescription::Type::Aggregate && analysis_result.need_aggregate && can_use_aggregate_projection) - { - Block aggregates; - // Let's first check if all aggregates are provided by current projection - for (const auto & aggregate : select.getQueryAnalyzer()->aggregates()) - { - if (const auto * column = sample_block.findByName(aggregate.column_name)) - { - aggregates.insert(*column); - continue; - } - - // We can treat every count_not_null_column as count() when selecting minmax_count_projection - if (minmax_count_projection && dynamic_cast(aggregate.function.get())) - { - const auto * count_column = sample_block.findByName("count()"); - if (!count_column) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "`count()` column is missing when minmax_count_projection == true. It is a bug"); - aggregates.insert({count_column->column, count_column->type, aggregate.column_name}); - continue; - } - - // No match - return; - } - - // Check if all aggregation keys can be either provided by some action, or by current - // projection directly. Reshape the `before_aggregation` action DAG so that it only - // needs to provide aggregation keys, and the DAG of certain child might be substituted - // by some keys in projection. - candidate.before_aggregation = analysis_result.before_aggregation->clone(); - auto required_columns = candidate.before_aggregation->foldActionsByProjection(keys, sample_block_for_keys); - - // TODO Let's find out the exact required_columns for keys. - if (required_columns.empty() && (!keys.empty() && !candidate.before_aggregation->getRequiredColumns().empty())) - return; - - if (analysis_result.optimize_aggregation_in_order) - attach_aggregation_in_order_info(); - - // Reorder aggregation keys and attach aggregates - candidate.before_aggregation->reorderAggregationKeysForProjection(key_name_pos_map); - candidate.before_aggregation->addAggregatesViaProjection(aggregates); - - if (rewrite_before_where(candidate, projection, required_columns, sample_block_for_keys, aggregates)) - { - candidate.required_columns = {required_columns.begin(), required_columns.end()}; - for (const auto & aggregate : aggregates) - candidate.required_columns.push_back(aggregate.name); - candidates.push_back(std::move(candidate)); - } - } - else if (projection.type == ProjectionDescription::Type::Normal) - { - if (analysis_result.before_aggregation && analysis_result.optimize_aggregation_in_order) - attach_aggregation_in_order_info(); - - if (analysis_result.hasWhere() || analysis_result.hasPrewhere()) - { - const auto & actions - = analysis_result.before_aggregation ? analysis_result.before_aggregation : analysis_result.before_order_by; - NameSet required_columns; - for (const auto & column : actions->getRequiredColumns()) - required_columns.insert(column.name); - - if (rewrite_before_where(candidate, projection, required_columns, sample_block, {})) - { - candidate.required_columns = {required_columns.begin(), required_columns.end()}; - candidates.push_back(std::move(candidate)); - } - } - } - }; - - ProjectionCandidate * selected_candidate = nullptr; - size_t min_sum_marks = std::numeric_limits::max(); - if (settings.optimize_use_implicit_projections && metadata_snapshot->minmax_count_projection - && !has_lightweight_delete_parts.load(std::memory_order_relaxed)) /// Disable ReadFromStorage for parts with lightweight. - add_projection_candidate(*metadata_snapshot->minmax_count_projection, true); - std::optional minmax_count_projection_candidate; - if (!candidates.empty()) - { - minmax_count_projection_candidate.emplace(std::move(candidates.front())); - candidates.clear(); - } - MergeTreeDataSelectExecutor reader(*this); - std::shared_ptr max_added_blocks; - if (settings.select_sequential_consistency) - { - if (const StorageReplicatedMergeTree * replicated = dynamic_cast(this)) - max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); - } - - const auto & snapshot_data = assert_cast(*storage_snapshot->data); - const auto & parts = snapshot_data.parts; - - auto prepare_min_max_count_projection = [&]() - { - DataPartsVector normal_parts; - query_info.minmax_count_projection_block = getMinMaxCountProjectionBlock( - metadata_snapshot, - minmax_count_projection_candidate->required_columns, - !query_info.filter_asts.empty() || analysis_result.prewhere_info || analysis_result.before_where, - query_info, - parts, - normal_parts, - max_added_blocks.get(), - query_context); - - // minmax_count_projection cannot be used used when there is no data to process, because - // it will produce incorrect result during constant aggregation. - // See https://github.com/ClickHouse/ClickHouse/issues/36728 - if (!query_info.minmax_count_projection_block) - return; - - if (minmax_count_projection_candidate->prewhere_info) - { - const auto & prewhere_info = minmax_count_projection_candidate->prewhere_info; - - if (prewhere_info->row_level_filter) - { - ExpressionActions(prewhere_info->row_level_filter, actions_settings).execute(query_info.minmax_count_projection_block); - query_info.minmax_count_projection_block.erase(prewhere_info->row_level_column_name); - } - - if (prewhere_info->prewhere_actions) - ExpressionActions(prewhere_info->prewhere_actions, actions_settings).execute(query_info.minmax_count_projection_block); - - if (prewhere_info->remove_prewhere_column) - query_info.minmax_count_projection_block.erase(prewhere_info->prewhere_column_name); - } - - if (normal_parts.empty()) - { - selected_candidate = &*minmax_count_projection_candidate; - selected_candidate->complete = true; - min_sum_marks = query_info.minmax_count_projection_block.rows(); - } - else if (normal_parts.size() < parts.size()) - { - auto normal_result_ptr = reader.estimateNumMarksToRead( - normal_parts, - query_info.prewhere_info, - analysis_result.required_columns, - metadata_snapshot, - metadata_snapshot, - query_info, - added_filter_nodes, - query_context, - settings.max_threads, - max_added_blocks); - - if (!normal_result_ptr->error()) - { - selected_candidate = &*minmax_count_projection_candidate; - selected_candidate->merge_tree_normal_select_result_ptr = normal_result_ptr; - min_sum_marks = query_info.minmax_count_projection_block.rows() + normal_result_ptr->marks(); - } - } - }; - - // If minmax_count_projection is a valid candidate, prepare it and check its completeness. - if (minmax_count_projection_candidate) - prepare_min_max_count_projection(); - - // We cannot find a complete match of minmax_count_projection, add more projections to check. - if (!selected_candidate || !selected_candidate->complete) - for (const auto & projection : metadata_snapshot->projections) - add_projection_candidate(projection); - - // Let's select the best projection to execute the query. - if (!candidates.empty()) - { - query_info.merge_tree_select_result_ptr = reader.estimateNumMarksToRead( - parts, - query_info.prewhere_info, - analysis_result.required_columns, - metadata_snapshot, - metadata_snapshot, - query_info, - added_filter_nodes, - query_context, - settings.max_threads, - max_added_blocks); - - if (!query_info.merge_tree_select_result_ptr->error()) - { - // Add 1 to base sum_marks so that we prefer projections even when they have equal number of marks to read. - // NOTE: It is not clear if we need it. E.g. projections do not support skip index for now. - auto sum_marks = query_info.merge_tree_select_result_ptr->marks() + 1; - if (sum_marks < min_sum_marks) - { - selected_candidate = nullptr; - min_sum_marks = sum_marks; - } - } - - /// Favor aggregate projections - for (auto & candidate : candidates) - { - if (candidate.desc->type == ProjectionDescription::Type::Aggregate) - { - selectBestProjection( - reader, - storage_snapshot, - query_info, - added_filter_nodes, - analysis_result.required_columns, - candidate, - query_context, - max_added_blocks, - settings, - parts, - selected_candidate, - min_sum_marks); - } - } - - /// Select the best normal projection. - for (auto & candidate : candidates) - { - if (candidate.desc->type == ProjectionDescription::Type::Normal) - { - selectBestProjection( - reader, - storage_snapshot, - query_info, - added_filter_nodes, - analysis_result.required_columns, - candidate, - query_context, - max_added_blocks, - settings, - parts, - selected_candidate, - min_sum_marks); - } - } - } - - if (!selected_candidate) - return std::nullopt; - else if (min_sum_marks == 0) - { - /// If selected_projection indicated an empty result set. Remember it in query_info but - /// don't use projection to run the query, because projection pipeline with empty result - /// set will not work correctly with empty_result_for_aggregation_by_empty_set. - query_info.merge_tree_empty_result = true; - return std::nullopt; - } - - if (selected_candidate->desc->type == ProjectionDescription::Type::Aggregate) - { - selected_candidate->aggregation_keys = select.getQueryAnalyzer()->aggregationKeys(); - selected_candidate->aggregate_descriptions = select.getQueryAnalyzer()->aggregates(); - } - - return *selected_candidate; -} - - QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( ContextPtr query_context, QueryProcessingStage::Enum to_stage, - const StorageSnapshotPtr & storage_snapshot, + const StorageSnapshotPtr &, SelectQueryInfo & query_info) const { if (query_context->getClientInfo().collaborate_with_initiator) @@ -7168,14 +6608,7 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( if (to_stage >= QueryProcessingStage::Enum::WithMergeableState) { - if (auto projection = getQueryProcessingStageWithAggregateProjection(query_context, storage_snapshot, query_info)) - { - query_info.projection = std::move(projection); - if (query_info.projection->desc->type == ProjectionDescription::Type::Aggregate) - return QueryProcessingStage::Enum::WithMergeableState; - } - else - query_info.projection = std::nullopt; + query_info.projection = std::nullopt; } return QueryProcessingStage::Enum::FetchColumns; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 7d97eaa295c..10d2d970155 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -413,13 +413,10 @@ public: const PartitionIdToMaxBlock * max_block_numbers_to_read, ContextPtr query_context) const; - std::optional getQueryProcessingStageWithAggregateProjection( - ContextPtr query_context, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info) const; - QueryProcessingStage::Enum getQueryProcessingStage( ContextPtr query_context, QueryProcessingStage::Enum to_stage, - const StorageSnapshotPtr & storage_snapshot, + const StorageSnapshotPtr &, SelectQueryInfo & info) const override; ReservationPtr reserveSpace(UInt64 expected_size, VolumePtr & volume) const; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 92b98a0af0c..7e028f8c65c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -69,7 +69,6 @@ namespace ErrorCodes extern const int TOO_MANY_PARTITIONS; extern const int DUPLICATED_PART_UUIDS; extern const int NO_SUCH_COLUMN_IN_TABLE; - extern const int PROJECTION_NOT_USED; } @@ -179,11 +178,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( query_info.merge_tree_select_result_ptr, enable_parallel_reading); - if (!step && settings.optimize_use_projections && settings.force_optimize_projection - && !metadata_for_reading->projections.empty() && !settings.query_plan_optimize_projection) - throw Exception(ErrorCodes::PROJECTION_NOT_USED, - "No projection is used when optimize_use_projections = 1 and force_optimize_projection = 1"); - auto plan = std::make_unique(); if (step) plan->addStep(std::move(step)); diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index 2edc05c4d7e..7695d235425 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -142,6 +142,9 @@ def test_usage(cluster, node_name): ) ) + # to check right handling of paths in disk web + node2.query("SELECT count() FROM system.remote_data_paths") + node2.query("DROP TABLE test{} SYNC".format(i)) print(f"Ok {i}") diff --git a/tests/integration/test_format_schema_on_server/test.py b/tests/integration/test_format_schema_on_server/test.py index 0b7d8837ad3..4d62dc976ce 100644 --- a/tests/integration/test_format_schema_on_server/test.py +++ b/tests/integration/test_format_schema_on_server/test.py @@ -1,5 +1,7 @@ import pytest +import os from helpers.cluster import ClickHouseCluster +from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) instance = cluster.add_instance("instance", clickhouse_path_dir="clickhouse_path") @@ -44,3 +46,69 @@ def test_protobuf_format_output(started_cluster): ) == "\x07\x08\x01\x12\x03abc\x07\x08\x02\x12\x03def" ) + + +def test_drop_cache_protobuf_format(started_cluster): + create_simple_table() + instance.query("INSERT INTO test.simple VALUES (1, 'abc'), (2, 'def')") + + schema = """ +syntax = "proto3"; + +message MessageTmp { + uint64 key = 1; + string value = 2; +} +""" + + protobuf_schema_path_name = "message_tmp.proto" + + database_path = os.path.abspath(os.path.join(instance.path, "database")) + with open( + os.path.join(database_path, "format_schemas", protobuf_schema_path_name), "w" + ) as file: + file.write(schema) + assert ( + instance.http_query( + "SELECT * FROM test.simple FORMAT Protobuf SETTINGS format_schema='message_tmp:MessageTmp'" + ) + == "\x07\x08\x01\x12\x03abc\x07\x08\x02\x12\x03def" + ) + # Replace simple.proto with a new Protobuf schema + new_schema = """ +syntax = "proto3"; + +message MessageTmp { + uint64 key2 = 1; + string value2 = 2; +} +""" + with open( + os.path.join(database_path, "format_schemas", protobuf_schema_path_name), "w" + ) as file: + file.write(new_schema) + + instance.query("DROP TABLE IF EXISTS test.new_simple") + instance.query( + """ + CREATE TABLE test.new_simple (key2 UInt64, value2 String) + ENGINE = MergeTree ORDER BY tuple(); + """ + ) + instance.query("INSERT INTO test.new_simple VALUES (1, 'abc'), (2, 'def')") + + instance.query("SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf") + + # Tets works with new scheme + assert ( + instance.http_query( + "SELECT * FROM test.new_simple FORMAT Protobuf SETTINGS format_schema='message_tmp:MessageTmp'" + ) + == "\x07\x08\x01\x12\x03abc\x07\x08\x02\x12\x03def" + ) + # Tests that stop working with old scheme + with pytest.raises(Exception) as exc: + instance.http_query( + "SELECT * FROM test.simple FORMAT Protobuf SETTINGS format_schema='message_tmp:MessageTmp'" + ) + assert "NO_COLUMNS_SERIALIZED_TO_PROTOBUF_FIELDS)" in str(exc.value) diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index fdc23a6203c..45ee876aa1d 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -115,10 +115,9 @@ def test_create_alter_user(): node.query("DROP USER u1, u2") -def check_secrets_for_tables(tables, table_name_prefix, password): - for i, table in enumerate(tables): - table_name = table_name_prefix + str(i) - if password in table: +def check_secrets_for_tables(test_cases, password): + for table_name, query, error in test_cases: + if (not error) and (password in query): assert password in node.query( f"SHOW CREATE TABLE {table_name} {show_secrets}=1" ) @@ -146,10 +145,31 @@ def test_create_table(): f"PostgreSQL(named_collection_4, host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user', password = '{password}')", f"MongoDB(named_collection_5, host = 'mongo1', port = 5432, db = 'mongo_db', collection = 'mongo_col', user = 'mongo_user', password = '{password}')", f"S3(named_collection_6, url = 'http://minio1:9001/root/data/test8.csv', access_key_id = 'minio', secret_access_key = '{password}', format = 'CSV')", + f"S3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV', 'gzip')", + f"S3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '{password}')", + ( + f"DeltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", + "DNS_ERROR", + ), ] - for i, table_engine in enumerate(table_engines): - node.query(f"CREATE TABLE table{i} (x int) ENGINE = {table_engine}") + def make_test_case(i): + table_name = f"table{i}" + table_engine = table_engines[i] + error = None + if isinstance(table_engine, tuple): + table_engine, error = table_engine + query = f"CREATE TABLE {table_name} (x int) ENGINE = {table_engine}" + return table_name, query, error + + # Generate test cases as a list of tuples (table_name, query, error). + test_cases = [make_test_case(i) for i in range(len(table_engines))] + + for table_name, query, error in test_cases: + if error: + assert error in node.query_and_get_error(query) + else: + node.query(query) for toggle, secret in enumerate(["[HIDDEN]", password]): assert ( @@ -187,43 +207,69 @@ def test_create_table(): "CREATE TABLE table11 (`x` int) ENGINE = PostgreSQL(named_collection_4, host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user', password = '[HIDDEN]')", "CREATE TABLE table12 (`x` int) ENGINE = MongoDB(named_collection_5, host = 'mongo1', port = 5432, db = 'mongo_db', collection = 'mongo_col', user = 'mongo_user', password = '[HIDDEN]'", "CREATE TABLE table13 (`x` int) ENGINE = S3(named_collection_6, url = 'http://minio1:9001/root/data/test8.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]', format = 'CSV')", + "CREATE TABLE table14 (x int) ENGINE = S3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV', 'gzip')", + "CREATE TABLE table15 (`x` int) ENGINE = S3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '[HIDDEN]')", + "CREATE TABLE table16 (`x` int) ENGINE = DeltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", ], must_not_contain=[password], ) - check_secrets_for_tables(table_engines, "table", password) + check_secrets_for_tables(test_cases, password) - for i in range(len(table_engines)): - node.query(f"DROP TABLE table{i}") + for table_name, query, error in test_cases: + if not error: + node.query(f"DROP TABLE {table_name}") def test_create_database(): password = new_password() database_engines = [ - f"MySQL('localhost:3306', 'mysql_db', 'mysql_user', '{password}') SETTINGS connect_timeout=1, connection_max_tries=1", - f"MySQL(named_collection_1, host = 'localhost', port = 3306, database = 'mysql_db', user = 'mysql_user', password = '{password}') SETTINGS connect_timeout=1, connection_max_tries=1", + ( + f"MySQL('localhost:3306', 'mysql_db', 'mysql_user', '{password}') SETTINGS connect_timeout=1, connection_max_tries=1", + "ALL_CONNECTION_TRIES_FAILED", + ), + ( + f"MySQL(named_collection_1, host = 'localhost', port = 3306, database = 'mysql_db', user = 'mysql_user', password = '{password}') SETTINGS connect_timeout=1, connection_max_tries=1", + "ALL_CONNECTION_TRIES_FAILED", + ), + f"S3('http://minio1:9001/root/data', 'minio', '{password}')", + f"S3(named_collection_2, secret_access_key = '{password}', access_key_id = 'minio')", # f"PostgreSQL('localhost:5432', 'postgres_db', 'postgres_user', '{password}')", ] - for i, database_engine in enumerate(database_engines): - # query_and_get_answer_with_error() is used here because we don't want to stop on error "Cannot connect to MySQL server". - # We test logging here and not actual work with MySQL server. - node.query_and_get_answer_with_error( - f"CREATE DATABASE database{i} ENGINE = {database_engine}" - ) + def make_test_case(i): + database_name = f"database{i}" + database_engine = database_engines[i] + error = None + if isinstance(database_engine, tuple): + database_engine, error = database_engine + query = f"CREATE DATABASE {database_name} ENGINE = {database_engine}" + return database_name, query, error + + # Generate test cases as a list of tuples (database_name, query, error). + test_cases = [make_test_case(i) for i in range(len(database_engines))] + + for database_name, query, error in test_cases: + if error: + assert error in node.query_and_get_error(query) + else: + node.query(query) check_logs( must_contain=[ "CREATE DATABASE database0 ENGINE = MySQL('localhost:3306', 'mysql_db', 'mysql_user', '[HIDDEN]')", "CREATE DATABASE database1 ENGINE = MySQL(named_collection_1, host = 'localhost', port = 3306, database = 'mysql_db', user = 'mysql_user', password = '[HIDDEN]')", - # "CREATE DATABASE database2 ENGINE = PostgreSQL('localhost:5432', 'postgres_db', 'postgres_user', '[HIDDEN]')", + "CREATE DATABASE database2 ENGINE = S3('http://minio1:9001/root/data', 'minio', '[HIDDEN]')", + "CREATE DATABASE database3 ENGINE = S3(named_collection_2, secret_access_key = '[HIDDEN]', access_key_id = 'minio')", + # "CREATE DATABASE database4 ENGINE = PostgreSQL('localhost:5432', 'postgres_db', 'postgres_user', '[HIDDEN]')", ], must_not_contain=[password], ) - for i in range(len(database_engines)): - node.query(f"DROP DATABASE IF EXISTS database{i}") + for database_name, query, error in test_cases: + if not error: + node.query(f"DROP DATABASE {database_name}") def test_table_functions(): @@ -260,10 +306,28 @@ def test_table_functions(): f"s3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '{password}')", f"remote(named_collection_6, addresses_expr = '127.{{2..11}}', database = 'default', table = 'remote_table', user = 'remote_user', password = '{password}', sharding_key = rand())", f"remoteSecure(named_collection_6, addresses_expr = '127.{{2..11}}', database = 'default', table = 'remote_table', user = 'remote_user', password = '{password}')", + f"s3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV')", + f"s3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '{password}')", + ( + f"deltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", + "DNS_ERROR", + ), ] - for i, table_function in enumerate(table_functions): - node.query(f"CREATE TABLE tablefunc{i} (x int) AS {table_function}") + def make_test_case(i): + table_name = f"tablefunc{i}" + table_function = table_functions[i] + error = None + if isinstance(table_function, tuple): + table_function, error = table_function + query = f"CREATE TABLE {table_name} (x int) AS {table_function}" + return table_name, query, error + + # Generate test cases as a list of tuples (table_name, query, error). + test_cases = [make_test_case(i) for i in range(len(table_functions))] + + for table_name, query, error in test_cases: + node.query(query) for toggle, secret in enumerate(["[HIDDEN]", password]): assert ( @@ -318,14 +382,18 @@ def test_table_functions(): "CREATE TABLE tablefunc27 (`x` int) AS s3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", "CREATE TABLE tablefunc28 (`x` int) AS remote(named_collection_6, addresses_expr = '127.{2..11}', database = 'default', table = 'remote_table', user = 'remote_user', password = '[HIDDEN]', sharding_key = rand())", "CREATE TABLE tablefunc29 (`x` int) AS remoteSecure(named_collection_6, addresses_expr = '127.{2..11}', database = 'default', table = 'remote_table', user = 'remote_user', password = '[HIDDEN]')", + "CREATE TABLE tablefunc30 (x int) AS s3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV')", + "CREATE TABLE tablefunc31 (`x` int) AS s3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '[HIDDEN]')", + "CREATE TABLE tablefunc32 (`x` int) AS deltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", ], must_not_contain=[password], ) - check_secrets_for_tables(table_functions, "tablefunc", password) + check_secrets_for_tables(test_cases, password) - for i in range(len(table_functions)): - node.query(f"DROP TABLE tablefunc{i}") + for table_name, query, error in test_cases: + if not error: + node.query(f"DROP TABLE {table_name}") def test_table_function_ways_to_call(): diff --git a/tests/integration/test_mysql_protocol/prepared_statements.reference b/tests/integration/test_mysql_protocol/prepared_statements.reference index 2cb6e7f5339..c4d252c3c1b 100644 --- a/tests/integration/test_mysql_protocol/prepared_statements.reference +++ b/tests/integration/test_mysql_protocol/prepared_statements.reference @@ -14,7 +14,7 @@ ui128 type is CHAR, value: 15324355 ui256 type is CHAR, value: 41345135123432 f32 type is FLOAT, value: -0.796896 f64 type is DOUBLE, value: -0.113259 -b type is BIT, value: true +b type is TINYINT, value: true Row #2 i8 type is TINYINT, value: 127 i16 type is SMALLINT, value: 32767 @@ -30,7 +30,7 @@ ui128 type is CHAR, value: 340282366920938463463374607431768211455 ui256 type is CHAR, value: 115792089237316195423570985008687907853269984665640564039457584007913129639935 f32 type is FLOAT, value: 1.234000 f64 type is DOUBLE, value: 3.352451 -b type is BIT, value: false +b type is TINYINT, value: false ### testStringTypes Row #1 diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 59e1565337b..f131005ed15 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -577,6 +577,37 @@ def test_mysql_set_variables(started_cluster): assert code == 0 +def test_mysql_boolean_format(started_cluster): + node.query( + """ + CREATE OR REPLACE TABLE mysql_boolean_format_test + ( + `a` Bool, + `b` Nullable(Bool), + `c` LowCardinality(Nullable(Bool)) + ) ENGINE MergeTree ORDER BY a; + """, + settings={"password": "123", "allow_suspicious_low_cardinality_types": 1}, + ) + node.query( + "INSERT INTO mysql_boolean_format_test VALUES (false, true, false), (true, false, true);", + settings={"password": "123"}, + ) + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run( + """ + mysql --protocol tcp -h {host} -P {port} default -u user_with_double_sha1 --password=abacaba + -e "SELECT * FROM mysql_boolean_format_test;" + """.format( + host=started_cluster.get_instance_ip("node"), port=server_port + ), + demux=True, + ) + logging.debug( + f"test_mysql_boolean_format code:{code} stdout:{stdout}, stderr:{stderr}" + ) + assert stdout.decode() == "a\tb\tc\n" + "0\t1\t0\n" + "1\t0\t1\n" + + def test_python_client(started_cluster): client = pymysql.connections.Connection( host=started_cluster.get_instance_ip("node"), diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 690944d9984..15e7109a1e9 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2747,10 +2747,10 @@ def test_rabbitmq_random_detach(rabbitmq_cluster): i[0] += 1 mes_id = str(i) channel.basic_publish( - exchange="test_sharding", + exchange="random", routing_key="", properties=pika.BasicProperties(message_id=mes_id), - body=message, + body=messages[-1], ) connection.close() diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 0fb941cca96..9289030331f 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -106,6 +106,7 @@ SYSTEM DROP COMPILED EXPRESSION CACHE ['SYSTEM DROP COMPILED EXPRESSION','DROP C SYSTEM DROP FILESYSTEM CACHE ['SYSTEM DROP FILESYSTEM CACHE','DROP FILESYSTEM CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM SYNC FILESYSTEM CACHE ['SYSTEM REPAIR FILESYSTEM CACHE','REPAIR FILESYSTEM CACHE','SYNC FILESYSTEM CACHE'] GLOBAL SYSTEM SYSTEM DROP SCHEMA CACHE ['SYSTEM DROP SCHEMA CACHE','DROP SCHEMA CACHE'] GLOBAL SYSTEM DROP CACHE +SYSTEM DROP FORMAT SCHEMA CACHE ['SYSTEM DROP FORMAT SCHEMA CACHE','DROP FORMAT SCHEMA CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP S3 CLIENT CACHE ['SYSTEM DROP S3 CLIENT','DROP S3 CLIENT CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP CACHE ['DROP CACHE'] \N SYSTEM SYSTEM RELOAD CONFIG ['RELOAD CONFIG'] GLOBAL SYSTEM RELOAD diff --git a/tests/queries/0_stateless/02366_kql_create_table.reference b/tests/queries/0_stateless/02366_kql_create_table.reference new file mode 100644 index 00000000000..35136b5ff42 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_create_table.reference @@ -0,0 +1,4 @@ +-- 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 new file mode 100644 index 00000000000..b266679b06a --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_create_table.sql @@ -0,0 +1,29 @@ +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 new file mode 100644 index 00000000000..fe666f3734c --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_datatype.reference @@ -0,0 +1,105 @@ +-- 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 new file mode 100644 index 00000000000..ecd29504298 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_datatype.sql @@ -0,0 +1,117 @@ +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 new file mode 100644 index 00000000000..2100f44f18c --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_distinct.reference @@ -0,0 +1,27 @@ +-- 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 new file mode 100644 index 00000000000..3c997eb4865 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_distinct.sql @@ -0,0 +1,28 @@ +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 new file mode 100644 index 00000000000..2936c9ea19c --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_extend.reference @@ -0,0 +1,32 @@ +-- 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 new file mode 100644 index 00000000000..3de489b0815 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_extend.sql @@ -0,0 +1,58 @@ +-- 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'); + +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 new file mode 100644 index 00000000000..6276cd6d867 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_func_binary.reference @@ -0,0 +1,7 @@ + -- 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 new file mode 100644 index 00000000000..824022b564c --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_func_binary.sql @@ -0,0 +1,8 @@ +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 new file mode 100644 index 00000000000..40d8d7e19ac --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_func_datetime.reference @@ -0,0 +1,76 @@ +-- 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 new file mode 100644 index 00000000000..b1fba4166a9 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_func_datetime.sql @@ -0,0 +1,86 @@ +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 new file mode 100644 index 00000000000..564f1eebc4b --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_func_dynamic.reference @@ -0,0 +1,152 @@ +-- 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 new file mode 100644 index 00000000000..b0956f032d0 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_func_dynamic.sql @@ -0,0 +1,161 @@ +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 new file mode 100644 index 00000000000..2a0bbf53fff --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_func_ip.reference @@ -0,0 +1,123 @@ +-- 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 new file mode 100644 index 00000000000..c9b335f203a --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_func_ip.sql @@ -0,0 +1,131 @@ +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 new file mode 100644 index 00000000000..92f283abcb6 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_func_math.reference @@ -0,0 +1,4 @@ +-- 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 new file mode 100644 index 00000000000..4e83622eb6b --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_func_math.sql @@ -0,0 +1,7 @@ +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 new file mode 100644 index 00000000000..b7fa62c5d43 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_func_scalar.reference @@ -0,0 +1,16 @@ +-- 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 new file mode 100644 index 00000000000..d7e94cfd9d1 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_func_scalar.sql @@ -0,0 +1,26 @@ +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 new file mode 100644 index 00000000000..9bdd38ca5db --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_func_string.reference @@ -0,0 +1,360 @@ +-- 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 new file mode 100644 index 00000000000..d251b04e08b --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_func_string.sql @@ -0,0 +1,313 @@ +-- 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 new file mode 100644 index 00000000000..8e7fde997bf --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_makeseries.reference @@ -0,0 +1,60 @@ +-- 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 new file mode 100644 index 00000000000..ecf2ef43cc4 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_makeseries.sql @@ -0,0 +1,74 @@ +-- 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); + +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 new file mode 100644 index 00000000000..25be070eb0b --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_mvexpand.reference @@ -0,0 +1,65 @@ +-- 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 new file mode 100644 index 00000000000..e7798609646 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_mvexpand.sql @@ -0,0 +1,35 @@ +-- 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_operator_in_sql.reference b/tests/queries/0_stateless/02366_kql_operator_in_sql.reference new file mode 100644 index 00000000000..4e0987aa5c3 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_operator_in_sql.reference @@ -0,0 +1,60 @@ +-- #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 new file mode 100644 index 00000000000..0b02faa0680 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_operator_in_sql.sql @@ -0,0 +1,42 @@ +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 new file mode 100644 index 00000000000..aeb42feb6be --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_summarize.reference @@ -0,0 +1,92 @@ +-- 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 new file mode 100644 index 00000000000..21a1b643d98 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_summarize.sql @@ -0,0 +1,102 @@ +-- 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); +EventLog | summarize count=count() by bin(unixtime_seconds_todatetime(Created/1000), 1s); +EventLog | summarize count=count() by time_label=bin(Created/1000, 1s); +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 new file mode 100644 index 00000000000..e70c02ce34f --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_tabular.reference @@ -0,0 +1,139 @@ +-- 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 new file mode 100644 index 00000000000..f73c4c09cca --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_tabular.sql @@ -0,0 +1,88 @@ +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 } diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql index 4f40da6c626..148ad303bd4 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql @@ -15,7 +15,7 @@ AND name NOT IN ( 'h3ToGeoBoundary', 'h3ToParent', 'h3ToString', 'h3UnidirectionalEdgeIsValid', 'h3kRing', 'stringToH3', 'geoToS2', 's2CapContains', 's2CapUnion', 's2CellsIntersect', 's2GetNeighbors', 's2RectAdd', 's2RectContains', 's2RectIntersection', 's2RectUnion', 's2ToGeo', 'normalizeUTF8NFC', 'normalizeUTF8NFD', 'normalizeUTF8NFKC', 'normalizeUTF8NFKD', - 'lemmatize', 'tokenize', 'stem', 'synonyms', + 'lemmatize', 'tokenize', 'stem', 'synonyms', 'kql_array_sort_asc', 'kql_array_sort_desc', 'detectCharset', 'detectLanguageUnknown', 'detectProgrammingLanguage', 'detectTonality' -- these functions are not enabled in fast test ) ORDER BY name; diff --git a/tests/queries/0_stateless/02801_backup_native_copy.reference b/tests/queries/0_stateless/02801_backup_native_copy.reference index f9b008cde2e..cddd0688fe7 100644 --- a/tests/queries/0_stateless/02801_backup_native_copy.reference +++ b/tests/queries/0_stateless/02801_backup_native_copy.reference @@ -1,4 +1,4 @@ -BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS allow_s3_native_copy = 1 1 -BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS allow_s3_native_copy = 0 0 -RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS allow_s3_native_copy = 1 1 -RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS allow_s3_native_copy = 0 0 +BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS allow_s3_native_copy=true 1 +BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS allow_s3_native_copy=false 0 +RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS allow_s3_native_copy=true 1 +RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS allow_s3_native_copy=false 0 diff --git a/tests/queries/0_stateless/02888_obsolete_settings.reference b/tests/queries/0_stateless/02888_obsolete_settings.reference index af4e486ffb5..90313977ab4 100644 --- a/tests/queries/0_stateless/02888_obsolete_settings.reference +++ b/tests/queries/0_stateless/02888_obsolete_settings.reference @@ -40,6 +40,7 @@ odbc_max_field_size optimize_duplicate_order_by_and_distinct optimize_fuse_sum_count_avg partial_merge_join_optimizations +query_plan_optimize_projection replication_alter_columns_timeout restore_threads temporary_live_view_timeout diff --git a/tests/queries/0_stateless/02889_system_drop_format_schema.reference b/tests/queries/0_stateless/02889_system_drop_format_schema.reference new file mode 100644 index 00000000000..ade894eeeb2 --- /dev/null +++ b/tests/queries/0_stateless/02889_system_drop_format_schema.reference @@ -0,0 +1,2 @@ +SYSTEM DROP FORMAT SCHEMA CACHE +SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf diff --git a/tests/queries/0_stateless/02889_system_drop_format_schema.sql b/tests/queries/0_stateless/02889_system_drop_format_schema.sql new file mode 100644 index 00000000000..2f17ae3d23f --- /dev/null +++ b/tests/queries/0_stateless/02889_system_drop_format_schema.sql @@ -0,0 +1,2 @@ +EXPLAIN SYNTAX SYSTEM DROP FORMAT SCHEMA CACHE; +EXPLAIN SYNTAX SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf;