Kusto-phase2: remove dialect auto option. use table function instead of subquery for kql() function fix type.

This commit is contained in:
Yong Wang 2023-04-02 18:14:09 -07:00
parent 9e82ed1d04
commit 6f6a103c5f
38 changed files with 1610 additions and 878 deletions

View File

@ -352,15 +352,6 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu
String message;
if (dialect == Dialect::kusto)
res = tryParseQuery(kql_parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth);
else if (dialect == Dialect::kusto_auto)
{
res = tryParseQuery(parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth);
if (!res)
{
pos = begin;
res = tryParseQuery(kql_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);
@ -374,18 +365,6 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu
{
if (dialect == Dialect::kusto)
res = parseQueryAndMovePosition(kql_parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth);
else if (dialect == Dialect::kusto_auto)
{
try
{
res = parseQueryAndMovePosition(parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth);
}
catch (...)
{
pos = begin;
res = parseQueryAndMovePosition(kql_parser, begin, 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);
}

View File

@ -207,7 +207,6 @@ enum class Dialect
{
clickhouse,
kusto,
kusto_auto,
prql,
};

View File

@ -106,6 +106,9 @@ list (APPEND OBJECT_LIBS $<TARGET_OBJECTS:clickhouse_functions_url>)
add_subdirectory(array)
list (APPEND OBJECT_LIBS $<TARGET_OBJECTS:clickhouse_functions_array>)
add_subdirectory(Kusto)
list (APPEND OBJECT_LIBS $<TARGET_OBJECTS:clickhouse_functions_kusto>)
if (TARGET ch_contrib::datasketches)
add_subdirectory(UniqTheta)
list (APPEND OBJECT_LIBS $<TARGET_OBJECTS:clickhouse_functions_uniqtheta>)

View File

@ -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()

View File

@ -0,0 +1,264 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeTuple.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/Kusto/KqlFunctionBase.h>
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 <typename Name, bool is_desc>
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<FunctionKqlArraySort>(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<DataTypeArray>(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<DataTypeArray>(makeNullable(nested_types[i]));
return std::make_shared<DataTypeTuple>(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<size_t> 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<ColumnArray>(holder.get());
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(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<const ColumnArray &>(*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<DataTypeArray>(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<DataTypeNullable>(std::make_shared<DataTypeInt8>());
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<DataTypeArray>(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<DataTypeArray>(result_type), "sorted")},
{DataTypeUInt8().createColumnConst(1, toField(UInt8(sorted_index + 1))), std::make_shared<DataTypeUInt8>(), ""}});
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<ColumnArray>(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<DataTypeArray>(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<DataTypeUInt64>();
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<DataTypeArray>(nested_types[i]);
ColumnsWithTypeAndName slice_args_left(
{{ColumnWithTypeAndName(tuple_columns[i], arg_type, "array")},
{DataTypeUInt8().createColumnConst(1, toField(UInt8(1))), std::make_shared<DataTypeUInt8>(), ""},
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<NameKqlArraySortAsc, false>;
using FunctionKqlArraySortDesc = FunctionKqlArraySort<NameKqlArraySortDesc, true>;
REGISTER_FUNCTION(KqlArraySort)
{
factory.registerFunction<FunctionKqlArraySortAsc>();
factory.registerFunction<FunctionKqlArraySortDesc>();
}
}

View File

@ -0,0 +1,32 @@
#pragma once
#include <Columns/ColumnArray.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeArray.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Interpreters/Context_fwd.h>
#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>(), ""},
{DataTypeUInt8().createColumnConst(1, toField(UInt8(2))), std::make_shared<DataTypeUInt8>(), ""}
});
auto if_res = FunctionFactory::instance().get("if", context)->build(if_columns)->execute(if_columns, std::make_shared<DataTypeUInt8>(), input_rows_count);
auto result = if_res->getUInt(0);
return (result == 1);
}
};
}

View File

@ -42,7 +42,6 @@
#include <Parsers/queryToString.h>
#include <Interpreters/StorageID.h>
#include <Parsers/Kusto/ParserKQLStatement.h>
namespace DB
{
@ -105,35 +104,28 @@ bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserSelectWithUnionQuery select;
ParserExplainQuery explain;
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
ASTPtr result_node = nullptr;
ParserKeyword s_kql("KQL");
if (s_kql.ignore(pos, expected))
if (ASTPtr select_node; select.parse(pos, select_node, expected))
{
if (!ParserKQLTaleFunction().parse(pos, result_node, expected))
return false;
result_node = std::move(select_node);
}
else
else if (ASTPtr explain_node; explain.parse(pos, explain_node, expected))
{
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
if (ASTPtr select_node; select.parse(pos, select_node, expected))
{
result_node = std::move(select_node);
}
else if (ASTPtr explain_node; explain.parse(pos, explain_node, expected))
{
const auto & explain_query = explain_node->as<const ASTExplainQuery &>();
const auto & explain_query = explain_node->as<const ASTExplainQuery &>();
if (explain_query.getTableFunction() || explain_query.getTableOverride())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "EXPLAIN in a subquery cannot have a table function or table override");
/// Replace subquery `(EXPLAIN <kind> <explain_settings> SELECT ...)`
/// with `(SELECT * FROM viewExplain("<kind>", "<explain_settings>", SELECT ...))`
/// Replace subquery `(EXPLAIN <kind> <explain_settings> SELECT ...)`
/// with `(SELECT * FROM viewExplain("<kind>", "<explain_settings>", SELECT ...))`
String kind_str = ASTExplainQuery::toString(explain_query.getKind());
String kind_str = ASTExplainQuery::toString(explain_query.getKind());
String settings_str;
if (ASTPtr settings_ast = explain_query.getSettings())
@ -143,32 +135,31 @@ bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
settings_str = queryToString(settings_ast);
}
const ASTPtr & explained_ast = explain_query.getExplainedQuery();
if (explained_ast)
{
auto view_explain = makeASTFunction("viewExplain",
std::make_shared<ASTLiteral>(kind_str),
std::make_shared<ASTLiteral>(settings_str),
explained_ast);
result_node = buildSelectFromTableFunction(view_explain);
}
else
{
auto view_explain = makeASTFunction("viewExplain",
std::make_shared<ASTLiteral>(kind_str),
std::make_shared<ASTLiteral>(settings_str));
result_node = buildSelectFromTableFunction(view_explain);
}
const ASTPtr & explained_ast = explain_query.getExplainedQuery();
if (explained_ast)
{
auto view_explain = makeASTFunction("viewExplain",
std::make_shared<ASTLiteral>(kind_str),
std::make_shared<ASTLiteral>(settings_str),
explained_ast);
result_node = buildSelectFromTableFunction(view_explain);
}
else
{
return false;
auto view_explain = makeASTFunction("viewExplain",
std::make_shared<ASTLiteral>(kind_str),
std::make_shared<ASTLiteral>(settings_str));
result_node = buildSelectFromTableFunction(view_explain);
}
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
}
else
{
return false;
}
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
node = std::make_shared<ASTSubquery>();
node->children.push_back(result_node);
@ -178,16 +169,6 @@ bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
/// 'kql(' is used for subuquery in Kusto, should not be treated as an identifier if kql followed by (
ParserKeyword s_kql("KQL");
if (s_kql.ignore(pos, expected))
{
if (pos->type == TokenType::OpeningRoundBracket)
{ --pos;
return false;
}
--pos;
}
/// Identifier in backquotes or in double quotes
if (pos->type == TokenType::QuotedIdentifier)
{

View File

@ -27,6 +27,7 @@
#include <Common/logger_useful.h>
#include <Parsers/queryToString.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/Kusto/ParserKQLStatement.h>
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<ASTFunction>();
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<ASTFunction>();
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))
@ -2159,6 +2180,55 @@ 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;
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<Layer> getFunctionLayer(ASTPtr identifier, bool is_table_function, bool allow_function_parameters_ = true)
{
@ -2195,6 +2265,8 @@ std::unique_ptr<Layer> getFunctionLayer(ASTPtr identifier, bool is_table_functio
return std::make_unique<ViewLayer>(false);
else if (function_name_lowercase == "viewifpermitted")
return std::make_unique<ViewLayer>(true);
else if (function_name_lowercase == "kql")
return std::make_unique<KustoLayer>();
}
if (function_name == "tuple")
@ -2448,7 +2520,7 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos
if (layers.front()->is_table_function)
{
if (typeid_cast<ViewLayer *>(layers.back().get()))
if (typeid_cast<ViewLayer *>(layers.back().get()) || typeid_cast<KustoLayer *>(layers.back().get()))
{
if (identifier_parser.parse(pos, tmp, expected)
&& ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected))
@ -2584,7 +2656,7 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos
{
layers.back()->pushOperand(std::move(tmp));
}
else if (pos->type == TokenType::OpeningRoundBracket || String(pos->begin , pos->end) == "kql")
else if (pos->type == TokenType::OpeningRoundBracket)
{
if (subquery_parser.parse(pos, tmp, expected))

View File

@ -716,7 +716,7 @@ https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/make-seriesoper
<!-- Default settings. -->
<default>
<load_balancing>random</load_balancing>
<dialect>kusto_auto</dialect>
<dialect>kusto</dialect>
</default> `
- Query can be executed with HTTP client as below once dialect is set in users.xml
@ -725,12 +725,12 @@ https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/make-seriesoper
- 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=<config-file path> `)
` <config>
<dialect>kusto_auto</dialect>
<dialect>kusto</dialect>
</config> `
OR
pass dialect setting with '--'. For example :
` clickhouse-client --dialect='kusto_auto' -q "KQL query" `
` clickhouse-client --dialect='kusto' -q "KQL query" `
- **strcmp** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/strcmpfunction)
`print strcmp('abc','ABC')`
@ -773,7 +773,6 @@ https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/make-seriesoper
`set dialect='clickhouse'`
`set dialect='kusto'`
`set dialect='kusto_auto'`
## IP functions
- [parse_ipv4](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parse-ipv4function)
@ -850,8 +849,7 @@ Please note that the functions listed below only take constant parameters for no
`set sql_dialect='clickhouse'`
- only process kql
`set sql_dialect='kusto'`
- process both kql and CH sql
`set sql_dialect='kusto_auto'`
## KQL() function
- create table

View File

@ -1,28 +1,13 @@
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/IParserBase.h>
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
#include <Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.h>
#include <Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.h>
#include <Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h>
#include <Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.h>
#include <Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.h>
#include <Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h>
#include <Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.h>
#include <Parsers/Kusto/KustoFunctions/KQLIPFunctions.h>
#include <Parsers/Kusto/KustoFunctions/KQLStringFunctions.h>
#include <Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.h>
#include <Parsers/Kusto/ParserKQLDateTypeTimespan.h>
#include "KQLFunctionFactory.h"
#include <Parsers/Kusto/ParserKQLOperators.h>
#include <Parsers/Kusto/ParserKQLQuery.h>
#include <Parsers/Kusto/ParserKQLStatement.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/Kusto/Utilities.h>
#include <Parsers/Kusto/ParserKQLDateTypeTimespan.h>
#include <boost/lexical_cast.hpp>
#include <magic_enum.hpp>
#include <pcg_random.hpp>
#include <Poco/String.h>
#include <format>
#include <numeric>
#include <stack>
namespace DB::ErrorCodes
@ -122,8 +107,11 @@ bool IParserKQLFunction::directMapping(
String IParserKQLFunction::generateUniqueIdentifier()
{
static pcg32_unique unique_random_generator;
return std::to_string(unique_random_generator());
// 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 pcg16_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)
@ -131,25 +119,52 @@ String IParserKQLFunction::getArgument(const String & function_name, DB::IParser
if (auto optional_argument = getOptionalArgument(function_name, pos, argument_state))
return std::move(*optional_argument);
throw Exception(std::format("Required argument was not provided in {}", function_name), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Required argument was not provided in {}", function_name);
}
std::vector<std::string> IParserKQLFunction::getArguments(
const String & function_name, DB::IParser::Pos & pos, const ArgumentState argument_state, const Interval & argument_count_interval)
{
std::vector<std::string> arguments;
while (auto argument = getOptionalArgument(function_name, pos, argument_state))
{
arguments.push_back(std::move(*argument));
}
if (!argument_count_interval.IsWithinBounds(static_cast<int>(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)
{
String converted_arg;
std::vector<String> tokens;
std::unique_ptr<IParserKQLFunction> fun;
int32_t round_bracket_count = 0, square_bracket_count = 0;
if (pos->type == TokenType::ClosingRoundBracket || pos->type == TokenType::ClosingSquareBracket)
return converted_arg;
return {};
if (pos->isEnd() || pos->type == TokenType::PipeMark || pos->type == TokenType::Semicolon)
throw Exception("Need more argument(s) in function: " + fn_name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Need more argument(s) in function: {}", fn_name);
std::vector<String> tokens;
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
{
String new_token;
if (!KQLOperators().convert(tokens, 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;
if (!KQLOperators::convert(tokens, pos))
{
if (pos->type == TokenType::BareWord)
{
@ -159,13 +174,19 @@ String IParserKQLFunction::getConvertedArgument(const String & fn_name, IParser:
pos->type == TokenType::Comma || pos->type == TokenType::ClosingRoundBracket
|| pos->type == TokenType::ClosingSquareBracket)
{
break;
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 = "'" + String(pos->begin + 1, pos->end - 1) + "'";
token = "'" + escapeSingleQuotes(String(pos->begin + 1, pos->end - 1)) + "'";
else if (pos->type == TokenType::OpeningSquareBracket)
{
++pos;
@ -183,12 +204,22 @@ String IParserKQLFunction::getConvertedArgument(const String & fn_name, IParser:
tokens.push_back(token);
}
}
++pos;
if (pos->type == TokenType::Comma || pos->type == TokenType::ClosingRoundBracket || pos->type == TokenType::ClosingSquareBracket)
break;
{
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;
}
}
for (auto const & token : tokens)
converted_arg = converted_arg.empty() ? token : converted_arg + " " + token;
String converted_arg;
for (const auto & token : tokens)
converted_arg.append((converted_arg.empty() ? "" : " ") + token);
return converted_arg;
}
@ -213,7 +244,7 @@ IParserKQLFunction::getOptionalArgument(const String & function_name, DB::IParse
magic_enum::enum_type_name<ArgumentState>(),
magic_enum::enum_name(argument_state));
String expression;
const auto * begin = pos->begin;
std::stack<DB::TokenType> scopes;
while (!pos->isEnd() && (!scopes.empty() || (pos->type != DB::TokenType::Comma && pos->type != DB::TokenType::ClosingRoundBracket)))
{
@ -229,24 +260,15 @@ IParserKQLFunction::getOptionalArgument(const String & function_name, DB::IParse
scopes.pop();
}
if (token_type == DB::TokenType::QuotedIdentifier)
{
expression.push_back('\'');
expression.append(pos->begin + 1, pos->end - 1);
expression.push_back('\'');
}
else
expression.append(pos->begin, pos->end);
++pos;
}
return expression;
return std::string(begin, pos->begin);
}
String IParserKQLFunction::getKQLFunctionName(IParser::Pos & pos)
{
String fn_name = String(pos->begin, pos->end);
String fn_name(pos->begin, pos->end);
++pos;
if (pos->type != TokenType::OpeningRoundBracket)
{
@ -287,20 +309,29 @@ String IParserKQLFunction::kqlCallToExpression(
void IParserKQLFunction::validateEndOfFunction(const String & fn_name, IParser::Pos & pos)
{
if (pos->type != TokenType::ClosingRoundBracket)
throw Exception("Too many arguments in function: " + fn_name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Too many arguments in function: {}", fn_name);
}
String IParserKQLFunction::getExpression(IParser::Pos & pos)
{
String arg = String(pos->begin, pos->end);
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<std::string>(time_span.toSeconds());
};
if (pos->type == TokenType::BareWord)
{
String new_arg;
auto fun = KQLFunctionFactory::get(arg);
if (fun && fun->convert(new_arg, pos))
const auto fun = KQLFunctionFactory::get(arg);
if (String new_arg; fun && fun->convert(new_arg, pos))
{
validateEndOfFunction(arg, pos);
arg = new_arg;
arg = std::move(new_arg);
}
else
{
@ -310,20 +341,18 @@ String IParserKQLFunction::getExpression(IParser::Pos & pos)
if (pos->type == TokenType::OpeningRoundBracket)
{
if (Poco::toLower(arg) != "and" && Poco::toLower(arg) != "or")
throw Exception(arg + " is not a supported kusto function", ErrorCodes::UNKNOWN_FUNCTION);
throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "{} is not a supported kusto function", arg);
}
--pos;
}
ParserKQLDateTypeTimespan time_span;
ASTPtr node;
Expected expected;
if (time_span.parse(pos, node, expected))
arg = boost::lexical_cast<std::string>(time_span.toSeconds());
parseConstTimespan();
}
}
else if (pos->type == TokenType::ErrorWrongNumber)
parseConstTimespan();
else if (pos->type == TokenType::QuotedIdentifier)
arg = "'" + String(pos->begin + 1, pos->end - 1) + "'";
arg = "'" + escapeSingleQuotes(String(pos->begin + 1, pos->end - 1)) + "'";
else if (pos->type == TokenType::OpeningSquareBracket)
{
++pos;
@ -339,122 +368,15 @@ String IParserKQLFunction::getExpression(IParser::Pos & pos)
return arg;
}
int IParserKQLFunction::getNullCounts(String arg)
String IParserKQLFunction::escapeSingleQuotes(const String & input)
{
size_t index = 0;
int null_counts = 0;
for (char & i : arg)
String output;
for (const auto & ch : input)
{
if (i == 'n')
i = 'N';
if (i == 'u')
i = 'U';
if (i == 'l')
i = 'L';
if (ch == '\'')
output += ch;
output += ch;
}
while ((index = arg.find("NULL", index)) != std::string::npos)
{
index += 4;
null_counts += 1;
}
return null_counts;
return output;
}
int IParserKQLFunction::IParserKQLFunction::getArrayLength(String arg)
{
int array_length = 0;
bool comma_found = false;
for (char i : arg)
{
if (i == ',')
{
comma_found = true;
array_length += 1;
}
}
return comma_found ? array_length + 1 : 0;
}
String IParserKQLFunction::ArraySortHelper(String & out, IParser::Pos & pos, bool ascending)
{
String fn_name = getKQLFunctionName(pos);
if (fn_name.empty())
return "false";
String reverse;
String second_arg;
String expr;
if (!ascending)
reverse = "Reverse";
++pos;
String first_arg = getConvertedArgument(fn_name, pos);
int null_count = getNullCounts(first_arg);
if (pos->type == TokenType::Comma)
++pos;
out = "array(";
if (pos->type != TokenType::ClosingRoundBracket && String(pos->begin, pos->end) != "dynamic")
{
second_arg = getConvertedArgument(fn_name, pos);
out += "if (" + second_arg + ", array" + reverse + "Sort(" + first_arg + "), concat(arraySlice(array" + reverse + "Sort("
+ first_arg + ") as as1, indexOf(as1, NULL) as len1), arraySlice(as1, 1, len1-1)))";
out += " )";
return out;
}
--pos;
std::vector<String> argument_list;
if (pos->type != TokenType::ClosingRoundBracket)
{
while (pos->type != TokenType::ClosingRoundBracket)
{
++pos;
if (String(pos->begin, pos->end) != "dynamic")
{
expr = getConvertedArgument(fn_name, pos);
break;
}
second_arg = getConvertedArgument(fn_name, pos);
argument_list.push_back(second_arg);
}
}
else
{
++pos;
out += "array" + reverse + "Sort(" + first_arg + ")";
}
if (!argument_list.empty())
{
String temp_first_arg = first_arg;
int first_arg_length = getArrayLength(temp_first_arg);
if (null_count > 0 && expr.empty())
expr = "true";
if (null_count > 0)
first_arg = "if (" + expr + ", array" + reverse + "Sort(" + first_arg + "), concat(arraySlice(array" + reverse + "Sort("
+ first_arg + ") as as1, indexOf(as1, NULL) as len1 ), arraySlice( as1, 1, len1-1) ) )";
else
first_arg = "array" + reverse + "Sort(" + first_arg + ")";
out += first_arg;
for (auto & i : argument_list)
{
out += " , ";
if (first_arg_length != getArrayLength(i))
out += "array(NULL)";
else if (null_count > 0)
out += "If (" + expr + "," + "array" + reverse + "Sort((x, y) -> y, " + i + "," + temp_first_arg
+ "), arrayConcat(arraySlice(" + "array" + reverse + "Sort((x, y) -> y, " + i + "," + temp_first_arg
+ ") , length(" + temp_first_arg + ") - " + std::to_string(null_count) + " + 1) , arraySlice(" + "array" + reverse
+ "Sort((x, y) -> y, " + i + "," + temp_first_arg + ") , 1, length(" + temp_first_arg + ") - "
+ std::to_string(null_count) + ") ) )";
else
out += "array" + reverse + "Sort((x, y) -> y, " + i + "," + temp_first_arg + ")";
}
}
out += " )";
return out;
}
}

View File

@ -1,7 +1,6 @@
#pragma once
#include <Parsers/IParserBase.h>
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
#include <span>
@ -29,6 +28,12 @@ private:
class IParserKQLFunction
{
public:
enum class ArgumentState
{
Parsed,
Raw
};
template <typename F>
ALWAYS_INLINE static bool wrapConvertImpl(IParser::Pos & pos, const F & func)
{
@ -59,31 +64,28 @@ public:
virtual const char * getName() const = 0;
virtual ~IParserKQLFunction() = default;
static String getExpression(IParser::Pos & pos);
protected:
enum class ArgumentState
{
Parsed,
Raw
};
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 String generateUniqueIdentifier();
static String getArgument(const String & function_name, DB::IParser::Pos & pos, ArgumentState argument_state = ArgumentState::Parsed);
static std::vector<std::string> 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<String>
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<const std::string_view> params, uint32_t max_depth);
static String kqlCallToExpression(std::string_view function_name, std::span<const std::string_view> 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);
static String getKQLFunctionName(IParser::Pos & pos);
static String ArraySortHelper(String & out, IParser::Pos & pos, bool ascending);
static int getNullCounts(String arg);
static int getArrayLength(String arg);
};
}

View File

@ -140,7 +140,7 @@ bool DatatypeInt::convertImpl(String & out, IParser::Pos & pos)
++pos;
if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral)
throw Exception("String is not parsed as int literal.", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "String is not parsed as int literal.");
else
{
auto arg = getConvertedArgument(fn_name, pos);
@ -162,7 +162,7 @@ bool DatatypeReal::convertImpl(String & out, IParser::Pos & pos)
++pos;
if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral)
throw Exception("String is not parsed as double literal.", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "String is not parsed as double literal.");
else
{
auto arg = getConvertedArgument(fn_name, pos);
@ -203,7 +203,7 @@ bool DatatypeTimespan::convertImpl(String & out, IParser::Pos & pos)
++pos;
}
else
throw Exception("Not a correct timespan expression: " + fn_name, ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not a correct timespan expression: {}", fn_name);
return true;
}
@ -219,7 +219,7 @@ bool DatatypeDecimal::convertImpl(String & out, IParser::Pos & pos)
int precision = 34;
if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral)
throw Exception("Failed to parse String as decimal Literal: " + fn_name, ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse String as decimal Literal: {}", fn_name);
--pos;
arg = getArgument(fn_name, pos);
@ -228,7 +228,7 @@ bool DatatypeDecimal::convertImpl(String & out, IParser::Pos & pos)
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("Failed to parse String as decimal Literal: " + fn_name, ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse String as decimal Literal: {}", fn_name);
if (std::regex_match(arg, expr))
{
@ -248,7 +248,7 @@ bool DatatypeDecimal::convertImpl(String & out, IParser::Pos & pos)
scale = std::max(precision - length, 0);
}
if (is_string)
throw Exception("Failed to parse String as decimal Literal: " + fn_name, ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse String as decimal Literal: {}", fn_name);
if (scale < 0 || Poco::toUpper(arg) == "NULL")
out = "NULL";

View File

@ -117,7 +117,7 @@ bool DatetimePart::convertImpl(String & out, IParser::Pos & pos)
else if (part == "SECOND")
format = "%S";
else
throw Exception("Unexpected argument " + part + " for " + fn_name, ErrorCodes::SYNTAX_ERROR);
throw Exception(ErrorCodes::SYNTAX_ERROR, "Unexpected argument {} for {}", part, fn_name);
out = std::format("formatDateTime({}, '{}')", date, format);
return true;
@ -178,7 +178,7 @@ bool EndOfMonth::convertImpl(String & out, IParser::Pos & pos)
++pos;
offset = getConvertedArgument(fn_name, pos);
if (offset.empty())
throw Exception("Number of arguments do not match in function:" + fn_name, ErrorCodes::SYNTAX_ERROR);
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) + "
@ -241,7 +241,7 @@ bool EndOfYear::convertImpl(String & out, IParser::Pos & pos)
const String datetime_str = getConvertedArgument(fn_name, pos);
if (datetime_str.empty())
throw Exception("Number of arguments do not match in function:" + fn_name, ErrorCodes::SYNTAX_ERROR);
throw Exception(ErrorCodes::SYNTAX_ERROR, "Number of arguments do not match in function: {}", fn_name);
String offset = "0";
if (pos->type == TokenType::Comma)
@ -249,7 +249,7 @@ bool EndOfYear::convertImpl(String & out, IParser::Pos & pos)
++pos;
offset = getConvertedArgument(fn_name, pos);
if (offset.empty())
throw Exception("Number of arguments do not match in function:" + fn_name, ErrorCodes::SYNTAX_ERROR);
throw Exception(ErrorCodes::SYNTAX_ERROR, "Number of arguments do not match in function: {}", fn_name);
offset.erase(remove(offset.begin(), offset.end(), ' '), offset.end());
}
@ -295,7 +295,7 @@ bool FormatDateTime::convertImpl(String & out, IParser::Pos & pos)
if (c == ' ' || c == '-' || c == '_' || c == '[' || c == ']' || c == '/' || c == ',' || c == '.' || c == ':')
formatspecifier = formatspecifier + c;
else
throw Exception("Invalid format delimiter in function:" + fn_name, ErrorCodes::SYNTAX_ERROR);
throw Exception(ErrorCodes::SYNTAX_ERROR, "Invalid format delimiter in function: {}", fn_name);
++i;
}
else
@ -326,7 +326,7 @@ bool FormatDateTime::convertImpl(String & out, IParser::Pos & pos)
else if (arg.starts_with('f') || arg.starts_with('F'))
decimal = arg.size();
else
throw Exception("Format specifier " + arg + " in function:" + fn_name + "is not supported", ErrorCodes::SYNTAX_ERROR);
throw Exception( ErrorCodes::SYNTAX_ERROR, "Format specifier {} in function: {} is not supported",arg, fn_name);
res.pop_back();
i = i + arg.size();
}
@ -385,7 +385,7 @@ bool FormatTimeSpan::convertImpl(String & out, IParser::Pos & pos)
if (c == ' ' || c == '-' || c == '_' || c == '[' || c == ']' || c == '/' || c == ',' || c == '.' || c == ':')
formatspecifier = formatspecifier + c;
else
throw Exception("Invalid format delimiter in function:" + fn_name, ErrorCodes::SYNTAX_ERROR);
throw Exception( ErrorCodes::SYNTAX_ERROR, "Invalid format delimiter in function: {}", fn_name);
++i;
}
else
@ -414,7 +414,7 @@ bool FormatTimeSpan::convertImpl(String & out, IParser::Pos & pos)
else if (arg.starts_with('f') || arg.starts_with('F'))
decimal = arg.size();
else
throw Exception("Format specifier " + arg + " in function:" + fn_name + "is not supported", ErrorCodes::SYNTAX_ERROR);
throw Exception(ErrorCodes::SYNTAX_ERROR, "Format specifier {} in function:{} is not supported", arg, fn_name);
res.pop_back();
i = i + arg.size();
}
@ -532,7 +532,7 @@ bool MakeTimeSpan::convertImpl(String & out, IParser::Pos & pos)
}
if (arg_count < 2 || arg_count > 4)
throw Exception("argument count out of bound in function: " + fn_name, ErrorCodes::SYNTAX_ERROR);
throw Exception(ErrorCodes::SYNTAX_ERROR, "argument count out of bound in function: {}", fn_name);
if (arg_count == 2)
{
@ -568,7 +568,7 @@ bool MakeTimeSpan::convertImpl(String & out, IParser::Pos & pos)
day = day + ".";
}
else
throw Exception("argument count out of bound in function: " + fn_name, ErrorCodes::SYNTAX_ERROR);
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;
@ -599,7 +599,7 @@ bool MakeDateTime::convertImpl(String & out, IParser::Pos & pos)
}
if (arg_count < 1 || arg_count > 7)
throw Exception("argument count out of bound in function: " + fn_name, ErrorCodes::SYNTAX_ERROR);
throw Exception(ErrorCodes::SYNTAX_ERROR, "argument count out of bound in function: {}", fn_name);
if (arg_count < 7)
{
@ -757,7 +757,7 @@ bool UnixTimeSecondsToDateTime::convertImpl(String & out, IParser::Pos & pos)
++pos;
if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral)
throw Exception(fn_name + " accepts only long, int and double type of arguments ", ErrorCodes::BAD_ARGUMENTS);
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(

View File

@ -168,18 +168,12 @@ bool ArraySlice::convertImpl(String & out, IParser::Pos & pos)
bool ArraySortAsc::convertImpl(String & out, IParser::Pos & pos)
{
out = ArraySortHelper(out, pos, true);
if (out == "false")
return false;
return true;
return directMapping(out, pos, "kql_array_sort_asc");
}
bool ArraySortDesc::convertImpl(String & out, IParser::Pos & pos)
{
out = ArraySortHelper(out, pos, false);
if (out == "false")
return false;
return true;
return directMapping(out, pos, "kql_array_sort_desc");
}
bool ArraySplit::convertImpl(String & out, IParser::Pos & pos)
@ -277,7 +271,7 @@ bool Repeat::convertImpl(String & out, IParser::Pos & pos)
count.erase(remove(count.begin(), count.end(), ' '), count.end());
if (count.empty())
throw Exception("number of arguments do not match in function: " + function_name, ErrorCodes::SYNTAX_ERROR);
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);

View File

@ -142,18 +142,18 @@ bool Extract::convertImpl(String & out, IParser::Pos & pos)
if (s_kql.ignore(pos, expected))
{
if (!open_bracket.ignore(pos, expected))
throw Exception("Syntax error near typeof", ErrorCodes::SYNTAX_ERROR);
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(type_literal + " is not a supported kusto data type for extract", ErrorCodes::UNKNOWN_TYPE);
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("Syntax error near typeof", ErrorCodes::SYNTAX_ERROR);
throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near typeof");
}
}
@ -276,17 +276,17 @@ bool ExtractJson::convertImpl(String & out, IParser::Pos & pos)
if (s_kql.ignore(pos, expected))
{
if (!open_bracket.ignore(pos, expected))
throw Exception("Syntax error near typeof", ErrorCodes::SYNTAX_ERROR);
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(datatype + " is not a supported kusto data type for " + fn_name, ErrorCodes::UNKNOWN_TYPE);
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("Syntax error near typeof", ErrorCodes::SYNTAX_ERROR);
throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near typeof");
}
}
const auto json_val = std::format("JSON_VALUE({0},{1})", json_datasource, json_datapath);
@ -402,7 +402,7 @@ bool ParseCommandLine::convertImpl(String & out, IParser::Pos & pos)
const String type = getConvertedArgument(fn_name, pos);
if (type != "'windows'")
throw Exception("Supported type argument is windows for " + fn_name, ErrorCodes::BAD_ARGUMENTS);
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}))",
@ -603,7 +603,7 @@ bool StrCatDelim::convertImpl(String & out, IParser::Pos & pos)
args += ")";
if (arg_count < 2 || arg_count > 64)
throw Exception("argument count out of bound in function: " + fn_name, ErrorCodes::SYNTAX_ERROR);
throw Exception(ErrorCodes::SYNTAX_ERROR, "argument count out of bound in function: {}", fn_name);
out = std::move(args);
return true;
@ -674,7 +674,7 @@ bool SubString::convertImpl(String & out, IParser::Pos & pos)
auto length = getConvertedArgument(fn_name, pos);
if (starting_index.empty())
throw Exception("number of arguments do not match in function: " + fn_name, ErrorCodes::SYNTAX_ERROR);
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 + ") )";

View File

@ -1,7 +1,7 @@
#include <cmath>
#include <cstdlib>
#include <format>
#include <unordered_map>
#include <cmath>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/IParserBase.h>
#include <Parsers/Kusto/ParserKQLDateTypeTimespan.h>
@ -27,7 +27,7 @@ bool ParserKQLDateTypeTimespan ::parseImpl(Pos & pos, [[maybe_unused]] ASTPtr &
return true;
}
double ParserKQLDateTypeTimespan ::toSeconds()
double ParserKQLDateTypeTimespan::toSeconds()
{
switch (time_span_unit)
{

View File

@ -94,11 +94,12 @@ bool ParserKQLMVExpand::parseColumnArrayExprs(ColumnArrayExprs & column_array_ex
to_type = String(pos->begin, pos->end);
if (type_cast.find(to_type) == type_cast.end())
throw Exception(to_type + " is not a supported kusto data type for mv-expand", ErrorCodes::UNKNOWN_TYPE);
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"

View File

@ -1,43 +1,213 @@
#include <Parsers/ASTLiteral.h>
#include "ParserKQLOperators.h"
#include <Interpreters/ITokenExtractor.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
#include <Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h>
#include <Parsers/Kusto/ParserKQLOperators.h>
#include <Parsers/Kusto/ParserKQLQuery.h>
#include <Parsers/Kusto/ParserKQLStatement.h>
#include <Parsers/formatAST.h>
#include "KustoFunctions/IParserKQLFunction.h"
#include "ParserKQLQuery.h"
#include "ParserKQLStatement.h"
namespace DB
{
#include <format>
#include <unordered_map>
namespace ErrorCodes
namespace DB::ErrorCodes
{
extern const int SYNTAX_ERROR;
extern const int LOGICAL_ERROR;
extern const int SYNTAX_ERROR;
}
String KQLOperators::genHasAnyAllOpExpr(std::vector<String> & tokens, IParser::Pos & token_pos, String kql_op, String ch_op)
namespace
{
String new_expr;
Expected expected;
ParserToken s_lparen(TokenType::OpeningRoundBracket);
enum class WildcardsPos : uint8_t
{
none,
left,
right,
both
};
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<String, KQLOperatorValue> 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<DB::ASTSelectQuery>()->select();
while (selectColumns->children.size() > 1)
selectColumns->children.pop_back();
if (useLowerCase)
{
auto args = std::make_shared<DB::ASTExpressionList>();
args->children.push_back(selectColumns->children[0]);
auto func_lower = std::make_shared<DB::ASTFunction>();
func_lower->name = "lower";
func_lower->children.push_back(selectColumns->children[0]);
func_lower->arguments = args;
if (selectColumns->children[0]->as<DB::ASTIdentifier>())
func_lower->alias = std::move(selectColumns->children[0]->as<DB::ASTIdentifier>()->alias);
else if (selectColumns->children[0]->as<DB::ASTFunction>())
func_lower->alias = std::move(selectColumns->children[0]->as<DB::ASTFunction>()->alias);
auto funcs = std::make_shared<DB::ASTExpressionList>();
funcs->children.push_back(func_lower);
selectColumns->children[0] = std::move(funcs);
}
}
std::string applyFormatString(const std::string_view format_string, const std::string & haystack, const std::string & needle)
{
return std::vformat(format_string, std::make_format_args(haystack, needle));
}
std::string constructHasOperatorTranslation(const KQLOperatorValue kql_op, const std::string & haystack, const std::string & needle)
{
if (kql_op != KQLOperatorValue::has && kql_op != KQLOperatorValue::not_has && kql_op != KQLOperatorValue::has_cs
&& kql_op != KQLOperatorValue::not_has_cs && kql_op != KQLOperatorValue::has_all && kql_op != KQLOperatorValue::has_any)
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unexpected operator: {}", magic_enum::enum_name(kql_op));
const auto tokens = std::invoke([&needle] {
std::vector<std::string_view> result;
size_t pos = 0;
size_t start = 0;
size_t length = 0;
DB::SplitTokenExtractor token_extractor;
while (pos < needle.length() && token_extractor.nextInString(needle.c_str(), needle.length(), &pos, &start, &length))
result.emplace_back(needle.c_str() + start, length);
return result;
});
const auto is_case_sensitive = kql_op == KQLOperatorValue::has_cs || kql_op == KQLOperatorValue::not_has_cs;
const auto has_token_suffix = is_case_sensitive ? "" : "CaseInsensitive";
const auto has_all_tokens
= std::accumulate(tokens.cbegin(), tokens.cend(), std::string(), [&has_token_suffix, &haystack](auto acc, const auto & token) {
return std::move(acc) + std::format("hasToken{}({}, '{}') and ", has_token_suffix, haystack, token);
});
const auto is_negation = kql_op == KQLOperatorValue::not_has || kql_op == KQLOperatorValue::not_has_cs;
return std::format(
"{4}ifNull(hasToken{3}OrNull({0}, {1}), {2} position{3}({0}, {1}) > 0)",
haystack,
needle,
has_all_tokens,
has_token_suffix,
is_negation ? "not " : "");
}
}
String genHasAnyAllOpExpr(
std::vector<std::string> & tokens,
DB::IParser::Pos & token_pos,
const std::string & kql_op,
const std::function<std::string(const std::string &, const std::string &)> & translate)
{
std::string new_expr;
DB::Expected expected;
DB::ParserToken s_lparen(DB::TokenType::OpeningRoundBracket);
++token_pos;
if (!s_lparen.ignore(token_pos, expected))
throw Exception("Syntax error near " + kql_op, ErrorCodes::SYNTAX_ERROR);
throw DB::Exception(DB::ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op);
auto haystack = tokens.back();
String logic_op = (kql_op == "has_all") ? " and " : " or ";
while (!token_pos->isEnd() && token_pos->type != TokenType::PipeMark && token_pos->type != TokenType::Semicolon)
const auto * const logic_op = (kql_op == "has_all") ? " and " : " or ";
while (!token_pos->isEnd() && token_pos->type != DB::TokenType::PipeMark && token_pos->type != DB::TokenType::Semicolon)
{
auto tmp_arg = IParserKQLFunction::getExpression(token_pos);
if (token_pos->type == TokenType::Comma)
new_expr = new_expr + logic_op;
auto tmp_arg = DB::IParserKQLFunction::getExpression(token_pos);
if (token_pos->type == DB::TokenType::Comma)
new_expr += logic_op;
else
new_expr = new_expr + ch_op + "(" + haystack + "," + tmp_arg + ")";
new_expr += translate(haystack, tmp_arg);
++token_pos;
if (token_pos->type == TokenType::ClosingRoundBracket)
if (token_pos->type == DB::TokenType::ClosingRoundBracket)
break;
}
@ -45,34 +215,145 @@ String KQLOperators::genHasAnyAllOpExpr(std::vector<String> & tokens, IParser::P
return new_expr;
}
String KQLOperators::genInOpExpr(IParser::Pos & token_pos, String kql_op, String ch_op)
String genEqOpExprCis(std::vector<String> & tokens, DB::IParser::Pos & token_pos, const DB::String & ch_op)
{
ParserKQLTaleFunction kqlfun_p;
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 genBetweenOpExpr(std::vector<std::string> & tokens, DB::IParser::Pos & token_pos, const String & ch_op)
{
DB::String new_expr;
new_expr += ch_op + "(";
new_expr += tokens.back() + ",";
tokens.pop_back();
++token_pos;
DB::BracketCount bracket_count;
bracket_count.count(token_pos);
++token_pos;
while (!token_pos->isEnd())
{
if ((token_pos->type == DB::TokenType::PipeMark || token_pos->type == DB::TokenType::Semicolon))
break;
if (token_pos->type == DB::TokenType::Dot)
break;
new_expr += DB::IParserKQLFunction::getExpression(token_pos);
++token_pos;
}
new_expr += ",";
DB::ParserToken dot_token(DB::TokenType::Dot);
if (dot_token.ignore(token_pos) && dot_token.ignore(token_pos))
{
while (!token_pos->isEnd())
{
bracket_count.count(token_pos);
if ((token_pos->type == DB::TokenType::PipeMark || token_pos->type == DB::TokenType::Semicolon) && bracket_count.isZero())
break;
new_expr += DB::IParserKQLFunction::getExpression(token_pos);
if (token_pos->type == DB::TokenType::ClosingRoundBracket && bracket_count.isZero())
{
break;
}
++token_pos;
}
}
else
throw DB::Exception(DB::ErrorCodes::SYNTAX_ERROR, "Syntax error, number of dots do not match.");
return new_expr;
}
String genInOpExprCis(std::vector<String> & 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("Syntax error near " + kql_op, ErrorCodes::SYNTAX_ERROR);
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))
{
new_expr = ch_op + " kql";
auto tmp_pos = token_pos;
while (tmp_pos != pos)
{
new_expr = new_expr + " " + String(tmp_pos->begin, tmp_pos->end);
++tmp_pos;
}
if (pos->type != TokenType::ClosingRoundBracket)
throw Exception("Syntax error near " + kql_op, ErrorCodes::SYNTAX_ERROR);
rebuildSubqueryForInOperator(select, false);
auto new_expr = ch_op + " (" + serializeAST(*select) + ")";
token_pos = pos;
return new_expr;
}
@ -82,10 +363,15 @@ String KQLOperators::genInOpExpr(IParser::Pos & token_pos, String kql_op, String
return ch_op;
}
String KQLOperators::genHaystackOpExpr(
std::vector<String> & tokens, IParser::Pos & token_pos, String kql_op, String ch_op, WildcardsPos wildcards_pos, WildcardsPos space_pos)
std::string genHaystackOpExpr(
std::vector<std::string> & tokens,
DB::IParser::Pos & token_pos,
const std::string & kql_op,
const std::function<std::string(const std::string &, const std::string &)> & translate,
WildcardsPos wildcards_pos,
WildcardsPos space_pos = WildcardsPos::none)
{
String new_expr, left_wildcards, right_wildcards, left_space, right_space;
std::string new_expr, left_wildcards, right_wildcards, left_space, right_space;
++token_pos;
if (!s_lparen.ignore(token_pos, expected))
@ -176,257 +462,275 @@ String KQLOperators::genHaystackOpExpr(std::vector<String> &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 + "')";
else if (!tokens.empty() && ((token_pos)->type == TokenType::BareWord))
if (!tokens.empty() && (token_pos->type == DB::TokenType::StringLiteral || token_pos->type == DB::TokenType::QuotedIdentifier))
new_expr = translate(
tokens.back(),
"'" + left_wildcards + left_space + DB::IParserKQLFunction::escapeSingleQuotes(String(token_pos->begin + 1, token_pos->end - 1))
+ right_space + right_wildcards + "'");
else if (!tokens.empty() && token_pos->type == DB::TokenType::BareWord)
{
auto tmp_arg = IParserKQLFunction::getExpression(token_pos);
new_expr = ch_op + "(" + tokens.back() + ", concat('" + left_wildcards + left_space + "', " + tmp_arg + ", '" + right_space
+ right_wildcards + "'))";
auto tmp_arg = DB::IParserKQLFunction::getExpression(token_pos);
new_expr = translate(
tokens.back(), "concat('" + left_wildcards + left_space + "', " + tmp_arg + ", '" + right_space + right_wildcards + "')");
}
else
throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op);
throw DB::Exception(DB::ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op);
tokens.pop_back();
return new_expr;
}
namespace DB
{
bool KQLOperators::convert(std::vector<String> & tokens, IParser::Pos & pos)
{
if (pos->isEnd() || pos->type == TokenType::PipeMark || pos->type == TokenType::Semicolon)
return false;
auto begin = pos;
auto token = String(pos->begin, pos->end);
if (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
String op = token;
if (token == "!")
{
++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);
}
else if (token == "matches")
{
KQLOperatorValue op_value = KQLOperatorValue::none;
auto token = String(pos->begin, pos->end);
String op = token;
if (token == "!")
{
++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);
}
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";
else
--pos;
}
}
else
{
op = token;
}
++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) == "regex")
op += " regex";
else
--pos;
}
}
++pos;
if (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
{
if (String(pos->begin, pos->end) == "~")
op += "~";
else
--pos;
}
else
--pos;
if (KQLOperator.find(op) == KQLOperator.end())
{
pos = begin;
return false;
}
const auto op_it = KQLOperator.find(op);
if (op_it == KQLOperator.end())
{
pos = begin;
return false;
}
op_value = KQLOperator[op];
String new_expr;
String new_expr;
if (op_value == KQLOperatorValue::none)
tokens.push_back(op);
else
{
if (tokens.empty())
throw Exception("Syntax error near " + op, ErrorCodes::SYNTAX_ERROR);
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::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::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::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::not_endswith_cs:
new_expr = genHaystackOpExpr(tokens, pos, op, "not endsWith", WildcardsPos::none);
break;
case KQLOperatorValue::equal:
break;
case KQLOperatorValue::not_equal:
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_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_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::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::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::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::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::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::not_in_cs:
new_expr = genInOpExpr(pos, op, "not in");
break;
case KQLOperatorValue::in:
break;
case KQLOperatorValue::not_in:
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::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::not_startswith_cs:
new_expr = genHaystackOpExpr(tokens, pos, op, "not startsWith", WildcardsPos::none);
break;
default:
break;
}
tokens.push_back(new_expr);
}
const auto & op_value = op_it->second;
if (op_value == KQLOperatorValue::none)
{
tokens.push_back(op);
return true;
}
pos = begin;
return false;
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, std::bind_front(&applyFormatString, "ilike({0}, {1})"), WildcardsPos::both);
break;
case KQLOperatorValue::not_contains:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "not ilike({0}, {1})"), WildcardsPos::both);
break;
case KQLOperatorValue::contains_cs:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "like({0}, {1})"), WildcardsPos::both);
break;
case KQLOperatorValue::not_contains_cs:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "not like({0}, {1})"), WildcardsPos::both);
break;
case KQLOperatorValue::endswith:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "ilike({0}, {1})"), WildcardsPos::left);
break;
case KQLOperatorValue::not_endswith:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "not ilike({0}, {1})"), WildcardsPos::left);
break;
case KQLOperatorValue::endswith_cs:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "endsWith({0}, {1})"), WildcardsPos::none);
break;
case KQLOperatorValue::not_endswith_cs:
new_expr
= genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "not endsWith({0}, {1})"), WildcardsPos::none);
break;
case KQLOperatorValue::equal:
new_expr = genEqOpExprCis(tokens, pos, "==");
break;
case KQLOperatorValue::not_equal:
new_expr = genEqOpExprCis(tokens, pos, "!=");
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, std::bind_front(&constructHasOperatorTranslation, op_value), WildcardsPos::none);
break;
case KQLOperatorValue::not_has:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&constructHasOperatorTranslation, op_value), WildcardsPos::none);
break;
case KQLOperatorValue::has_all:
case KQLOperatorValue::has_any:
new_expr = genHasAnyAllOpExpr(tokens, pos, op, std::bind_front(&constructHasOperatorTranslation, op_value));
break;
case KQLOperatorValue::has_cs:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&constructHasOperatorTranslation, op_value), WildcardsPos::none);
break;
case KQLOperatorValue::not_has_cs:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&constructHasOperatorTranslation, op_value), WildcardsPos::none);
break;
case KQLOperatorValue::hasprefix:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "ilike({0}, {1})"), WildcardsPos::right);
new_expr += " or ";
tokens.push_back(last_op);
new_expr += genHaystackOpExpr(
tokens, last_pos, op, std::bind_front(&applyFormatString, "ilike({0}, {1})"), WildcardsPos::both, WildcardsPos::left);
break;
case KQLOperatorValue::not_hasprefix:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "not ilike({0}, {1})"), WildcardsPos::right);
new_expr += " and ";
tokens.push_back(last_op);
new_expr += genHaystackOpExpr(
tokens, last_pos, op, std::bind_front(&applyFormatString, "not ilike({0}, {1})"), WildcardsPos::both, WildcardsPos::left);
break;
case KQLOperatorValue::hasprefix_cs:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "startsWith({0}, {1})"), WildcardsPos::none);
new_expr += " or ";
tokens.push_back(last_op);
new_expr += genHaystackOpExpr(
tokens, last_pos, op, std::bind_front(&applyFormatString, "like({0}, {1})"), WildcardsPos::both, WildcardsPos::left);
break;
case KQLOperatorValue::not_hasprefix_cs:
new_expr
= genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "not startsWith({0}, {1})"), WildcardsPos::none);
new_expr += " and ";
tokens.push_back(last_op);
new_expr += genHaystackOpExpr(
tokens, last_pos, op, std::bind_front(&applyFormatString, "not like({0}, {1})"), WildcardsPos::both, WildcardsPos::left);
break;
case KQLOperatorValue::hassuffix:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "ilike({0}, {1})"), WildcardsPos::left);
new_expr += " or ";
tokens.push_back(last_op);
new_expr += genHaystackOpExpr(
tokens, last_pos, op, std::bind_front(&applyFormatString, "ilike({0}, {1})"), WildcardsPos::both, WildcardsPos::right);
break;
case KQLOperatorValue::not_hassuffix:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "not ilike({0}, {1})"), WildcardsPos::left);
new_expr += " and ";
tokens.push_back(last_op);
new_expr += genHaystackOpExpr(
tokens, last_pos, op, std::bind_front(&applyFormatString, "not ilike({0}, {1})"), WildcardsPos::both, WildcardsPos::right);
break;
case KQLOperatorValue::hassuffix_cs:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "endsWith({0}, {1})"), WildcardsPos::none);
new_expr += " or ";
tokens.push_back(last_op);
new_expr += genHaystackOpExpr(
tokens, last_pos, op, std::bind_front(&applyFormatString, "like({0}, {1})"), WildcardsPos::both, WildcardsPos::right);
break;
case KQLOperatorValue::not_hassuffix_cs:
new_expr
= genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "not endsWith({0}, {1})"), WildcardsPos::none);
new_expr += " and ";
tokens.push_back(last_op);
new_expr += genHaystackOpExpr(
tokens, last_pos, op, std::bind_front(&applyFormatString, "not like({0}, {1})"), WildcardsPos::both, WildcardsPos::right);
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::in:
new_expr = genInOpExprCis(tokens, pos, op, "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, std::bind_front(&applyFormatString, "match({0}, {1})"), WildcardsPos::none);
break;
case KQLOperatorValue::startswith:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "ilike({0}, {1})"), WildcardsPos::right);
break;
case KQLOperatorValue::not_startswith:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "not ilike({0}, {1})"), WildcardsPos::right);
break;
case KQLOperatorValue::startswith_cs:
new_expr = genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "startsWith({0}, {1})"), WildcardsPos::none);
break;
case KQLOperatorValue::not_startswith_cs:
new_expr
= genHaystackOpExpr(tokens, pos, op, std::bind_front(&applyFormatString, "not startsWith({0}, {1})"), WildcardsPos::none);
break;
case KQLOperatorValue::between:
new_expr = genBetweenOpExpr(tokens, pos, "kql_between");
break;
case KQLOperatorValue::not_between:
new_expr = genBetweenOpExpr(tokens, pos, "not kql_between");
break;
default:
break;
}
tokens.push_back(new_expr);
return true;
}
}
}

View File

@ -1,11 +1,9 @@
#pragma once
#include <unordered_map>
#include <Parsers/IParserBase.h>
#include <Parsers/Kusto/ParserKQLQuery.h>
#include <Parsers/IParser.h>
namespace DB
{
class KQLOperators
{
public:
@ -106,6 +104,7 @@ protected:
WildcardsPos space_pos = WildcardsPos::none);
static String genInOpExpr(IParser::Pos & token_pos, String kql_op, String ch_op);
static String genHasAnyAllOpExpr(std::vector<String> & tokens, IParser::Pos & token_pos, String kql_op, String ch_op);
static bool convert(std::vector<String> & tokens, IParser::Pos & pos);
};
}

View File

@ -150,55 +150,160 @@ String ParserKQLBase::getExprFromPipe(Pos & pos)
String ParserKQLBase::getExprFromToken(Pos & pos)
{
String res;
std::vector<String> tokens;
std::unique_ptr<IParserKQLFunction> kql_function;
String alias;
std::vector<Pos> 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) != "~")
{
if (tokens.empty())
throw Exception("Syntax error near equal symbol", ErrorCodes::SYNTAX_ERROR);
alias = tokens.back();
tokens.pop_back();
if (alias[0] == '\'' || alias[0] == '\"')
throw Exception(alias + " Quoted string is not a valid alias", ErrorCodes::SYNTAX_ERROR);
}
comma_pos.push_back(pos);
--pos;
}
else if (!KQLOperators().convert(tokens, pos))
{
token = IParserKQLFunction::getExpression(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<String> 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<String> 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;
}
@ -265,9 +370,9 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
String table_name(pos->begin, pos->end);
if (table_name == "print")
operation_pos.push_back(std::make_pair(table_name, pos));
operation_pos.emplace_back(table_name, pos);
else
operation_pos.push_back(std::make_pair("table", pos));
operation_pos.emplace_back("table", pos);
++pos;
@ -364,7 +469,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")
@ -473,7 +578,7 @@ 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<ASTSubquery>();

View File

@ -41,4 +41,25 @@ protected:
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;
};
}

View File

@ -59,7 +59,7 @@ 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;

View File

@ -42,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"; }

View File

@ -0,0 +1,74 @@
#include "Utilities.h"
#include "KustoFunctions/IParserKQLFunction.h"
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
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<int>(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<ASTExpressionList>();
expression_list->children.push_back(std::make_shared<ASTAsterisk>());
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<ASTSelectWithUnionQuery>();
auto & list_of_selects = select_with_union_query->list_of_selects;
list_of_selects = std::make_shared<ASTExpressionList>();
list_of_selects->children.push_back(select_query);
select_with_union_query->children.push_back(list_of_selects);
return select_with_union_query;
}
}

View File

@ -0,0 +1,13 @@
#pragma once
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/IParser.h>
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);
}

View File

@ -213,7 +213,7 @@ Token Lexer::nextTokenImpl()
for (const char * iterator = token_begin; iterator < pos; ++iterator)
{
if (!isWordCharASCII(*iterator) && *iterator != '$' && *iterator != '.')
if (!isWordCharASCII(*iterator) && *iterator != '$')
return Token(TokenType::ErrorWrongNumber, token_begin, pos);
}

View File

@ -20,7 +20,6 @@
#include <Parsers/ParserSetQuery.h>
#include <Common/typeid_cast.h>
#include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/Kusto/ParserKQLStatement.h>
namespace DB
{
@ -656,25 +655,17 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
/// ENGINE can not be specified for table functions.
if (storage || !table_function_p.parse(pos, as_table_function, expected))
{
ParserKeyword s_kql("KQL");
if (s_kql.ignore(pos, expected))
/// AS [db.]table
if (!name_p.parse(pos, as_table, expected))
return false;
if (s_dot.ignore(pos, expected))
{
if (!ParserKQLTaleFunction().parse(pos, select, expected))
return false;
}
else
{
/// AS [db.]table
as_database = as_table;
if (!name_p.parse(pos, as_table, expected))
return false;
if (s_dot.ignore(pos, expected))
{
as_database = as_table;
if (!name_p.parse(pos, as_table, expected))
return false;
}
}
/// Optional - ENGINE can be specified.
if (!storage)
storage_p.parse(pos, storage, expected);

View File

@ -12,7 +12,6 @@
#include <Parsers/InsertQuerySettingsPushDownVisitor.h>
#include <Common/typeid_cast.h>
#include "Parsers/IAST_fwd.h"
#include <Parsers/Kusto/ParserKQLStatement.h>
namespace DB
{
@ -47,7 +46,6 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserFunction table_function_p{false};
ParserStringLiteral infile_name_p;
ParserExpressionWithOptionalAlias exp_elem_p(false);
ParserKeyword s_kql("KQL");
/// create ASTPtr variables (result of parsing will be put in them).
/// They will be used to initialize ASTInsertQuery's fields.
@ -184,11 +182,6 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserWatchQuery watch_p;
watch_p.parse(pos, watch, expected);
}
else if (!infile && s_kql.ignore(pos, expected))
{
if (!ParserKQLTaleFunction().parse(pos, select, expected))
return false;
}
else if (!infile)
{
/// If all previous conditions were false and it's not FROM INFILE, query is incorrect

View File

@ -1,7 +1,7 @@
#include <Parsers/TokenIterator.h>
#include <unordered_set>
#include <base/types.h>
#include <Parsers/Kusto/ParserKQLDateTypeTimespan.h>
namespace DB
{
@ -21,53 +21,57 @@ Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool
UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin)
{
std::unordered_set<String> valid_kql_negative_suffix
({
"contains",
"contains_cs",
"endswith",
"endswith_cs",
"~",
"=",
"has",
"has_cs",
"hasprefix",
"hasprefix_cs",
"hassuffix",
"hassuffix_cs",
"in",
"startswith",
"startswith_cs"
});
std::unordered_set<String> 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.
bool is_kql_included = false;
for (TokenIterator it = begin; !it->isEnd() ; ++it)
for (TokenIterator it = begin; !it->isEnd(); ++it)
{
//for kql function in sql like : select * from kql(Customers | where FirstName !in ("test", "test2"));
if (String(it.get().begin, it.get().end) == "kql")
if (!it.isValid()) // allow kql negative operators
{
++it;
if (it->type == TokenType::OpeningRoundBracket)
is_kql_included = true;
--it;
}
if (!it.isValid())
{
if (!is_kql_included)
break;
if (it->type == TokenType::ErrorSingleExclamationMark)
{
++it;
if (valid_kql_negative_suffix.find(String(it.get().begin, it.get().end)) == valid_kql_negative_suffix.end())
{
++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)
@ -82,7 +86,8 @@ UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin)
stack.push_back(*it);
return stack;
}
else if ((stack.back().type == TokenType::OpeningRoundBracket && it->type == TokenType::ClosingRoundBracket)
else if (
(stack.back().type == TokenType::OpeningRoundBracket && it->type == TokenType::ClosingRoundBracket)
|| (stack.back().type == TokenType::OpeningSquareBracket && it->type == TokenType::ClosingSquareBracket))
{
/// Valid match.

View File

@ -7,128 +7,100 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_Dynamic, ParserTest,
::testing::ValuesIn(std::initializer_list<ParserTestCase>{
{
"print t = array_sort_asc(dynamic([null, 'd', 'a', 'c', 'c']))",
"SELECT [arraySort([NULL, 'd', 'a', 'c', 'c'])] AS t"
"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 t = array_sort_asc(dynamic([4, 1, 3, 2]))",
"SELECT [arraySort([4, 1, 3, 2])] AS t"
},
{
"print t = array_sort_asc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))",
"SELECT [arraySort(['b', 'a', 'c']), arraySort((x, y) -> y, ['q', 'p', 'r'], ['b', 'a', 'c'])] AS t"
},
{
"print t = array_sort_asc(dynamic(['q', 'p', 'r']), dynamic(['clickhouse','hello', 'world']))",
"SELECT [arraySort(['q', 'p', 'r']), arraySort((x, y) -> y, ['clickhouse', 'hello', 'world'], ['q', 'p', 'r'])] AS t"
},
{
"print t = array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , false)",
"SELECT [if(false, arraySort(['d', NULL, 'a', 'c', 'c']), concat(arraySlice(arraySort(['d', NULL, 'a', 'c', 'c']) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1)))] AS t"
},
{
"print t = array_sort_asc( dynamic([null, 'd', null, null, 'a', 'c', 'c', null, null, null]) , false)",
"SELECT [if(false, arraySort([NULL, 'd', NULL, NULL, 'a', 'c', 'c', NULL, NULL, NULL]), concat(arraySlice(arraySort([NULL, 'd', NULL, NULL, 'a', 'c', 'c', NULL, NULL, NULL]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1)))] AS t"
},
{
"print t = array_sort_asc( dynamic([null, 'd', null, null, 'a', 'c', 'c', null, null, null]) , true)",
"SELECT [if(true, arraySort([NULL, 'd', NULL, NULL, 'a', 'c', 'c', NULL, NULL, NULL]), concat(arraySlice(arraySort([NULL, 'd', NULL, NULL, 'a', 'c', 'c', NULL, NULL, NULL]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1)))] AS t"
},
{
"print t = array_sort_asc( dynamic([null, 'd', null, null, 'a', 'c', 'c', null, null, null]))",
"SELECT [arraySort([NULL, 'd', NULL, NULL, 'a', 'c', 'c', NULL, NULL, NULL])] AS t"
},
{
"print t = array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , 1 < 2)",
"SELECT [if(1 < 2, arraySort(['d', NULL, 'a', 'c', 'c']), concat(arraySlice(arraySort(['d', NULL, 'a', 'c', 'c']) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1)))] AS t"
},
{
"print t = array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2)",
"SELECT [if(1 > 2, arraySort(['d', NULL, 'a', 'c', 'c']), concat(arraySlice(arraySort(['d', NULL, 'a', 'c', 'c']) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1)))] AS t"
},
{
"print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]), false)",
"SELECT [if(false, arraySort([2, 1, NULL, 3]), concat(arraySlice(arraySort([2, 1, NULL, 3]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1))), If(false, arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), arrayConcat(arraySlice(arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), (length([2, 1, NULL, 3]) - 1) + 1), arraySlice(arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), 1, length([2, 1, NULL, 3]) - 1)))] AS t"
},
{
"print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]), 1 > 2)",
"SELECT [if(1 > 2, arraySort([2, 1, NULL, 3]), concat(arraySlice(arraySort([2, 1, NULL, 3]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1))), If(1 > 2, arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), arrayConcat(arraySlice(arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), (length([2, 1, NULL, 3]) - 1) + 1), arraySlice(arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), 1, length([2, 1, NULL, 3]) - 1)))] AS t"
},
{
"print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]), true)",
"SELECT [if(true, arraySort([2, 1, NULL, 3]), concat(arraySlice(arraySort([2, 1, NULL, 3]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1))), If(true, arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), arrayConcat(arraySlice(arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), (length([2, 1, NULL, 3]) - 1) + 1), arraySlice(arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), 1, length([2, 1, NULL, 3]) - 1)))] AS t"
},
{
"print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]), 1 < 2)",
"SELECT [if(1 < 2, arraySort([2, 1, NULL, 3]), concat(arraySlice(arraySort([2, 1, NULL, 3]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1))), If(1 < 2, arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), arrayConcat(arraySlice(arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), (length([2, 1, NULL, 3]) - 1) + 1), arraySlice(arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), 1, length([2, 1, NULL, 3]) - 1)))] AS t"
},
{
"print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]))",
"SELECT [if(true, arraySort([2, 1, NULL, 3]), concat(arraySlice(arraySort([2, 1, NULL, 3]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1))), If(true, arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), arrayConcat(arraySlice(arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), (length([2, 1, NULL, 3]) - 1) + 1), arraySlice(arraySort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), 1, length([2, 1, NULL, 3]) - 1)))] AS t"
},
{
"print t = array_sort_desc(dynamic([null, 'd', 'a', 'c', 'c']))",
"SELECT [arrayReverseSort([NULL, 'd', 'a', 'c', 'c'])] AS t"
},
{
"print t = array_sort_desc(dynamic([4, 1, 3, 2]))",
"SELECT [arrayReverseSort([4, 1, 3, 2])] AS t"
},
{
"print t = array_sort_desc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))",
"SELECT [arrayReverseSort(['b', 'a', 'c']), arrayReverseSort((x, y) -> y, ['q', 'p', 'r'], ['b', 'a', 'c'])] AS t"
},
{
"print t = array_sort_desc(dynamic(['2', '1', '3']), dynamic(['clickhouse','hello', 'world']))",
"SELECT [arrayReverseSort(['2', '1', '3']), arrayReverseSort((x, y) -> y, ['clickhouse', 'hello', 'world'], ['2', '1', '3'])] AS t"
},
{
"print t = array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , false)",
"SELECT [if(false, arrayReverseSort(['d', NULL, 'a', 'c', 'c']), concat(arraySlice(arrayReverseSort(['d', NULL, 'a', 'c', 'c']) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1)))] AS t"
},
{
"print t = array_sort_desc( dynamic([null, 'd', null, null, 'a', 'c', 'c', null, null, null]) , false)",
"SELECT [if(false, arrayReverseSort([NULL, 'd', NULL, NULL, 'a', 'c', 'c', NULL, NULL, NULL]), concat(arraySlice(arrayReverseSort([NULL, 'd', NULL, NULL, 'a', 'c', 'c', NULL, NULL, NULL]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1)))] AS t"
},
{
"print t = array_sort_desc( dynamic([null, 'd', null, null, 'a', 'c', 'c', null, null, null]) , true)",
"SELECT [if(true, arrayReverseSort([NULL, 'd', NULL, NULL, 'a', 'c', 'c', NULL, NULL, NULL]), concat(arraySlice(arrayReverseSort([NULL, 'd', NULL, NULL, 'a', 'c', 'c', NULL, NULL, NULL]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1)))] AS t"
},
{
"print t = array_sort_desc( dynamic([null, 'd', null, null, 'a', 'c', 'c', null, null, null]))",
"SELECT [arrayReverseSort([NULL, 'd', NULL, NULL, 'a', 'c', 'c', NULL, NULL, NULL])] AS t"
},
{
"print t = array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , 1 < 2)",
"SELECT [if(1 < 2, arrayReverseSort(['d', NULL, 'a', 'c', 'c']), concat(arraySlice(arrayReverseSort(['d', NULL, 'a', 'c', 'c']) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1)))] AS t"
},
{
"print t = array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2)",
"SELECT [if(1 > 2, arrayReverseSort(['d', NULL, 'a', 'c', 'c']), concat(arraySlice(arrayReverseSort(['d', NULL, 'a', 'c', 'c']) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1)))] AS t"
},
{
"print t = array_sort_desc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]), false)",
"SELECT [if(false, arrayReverseSort([2, 1, NULL, 3]), concat(arraySlice(arrayReverseSort([2, 1, NULL, 3]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1))), If(false, arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), arrayConcat(arraySlice(arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), (length([2, 1, NULL, 3]) - 1) + 1), arraySlice(arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), 1, length([2, 1, NULL, 3]) - 1)))] AS t"
},
{
"print t = array_sort_desc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]), 1 > 2)",
"SELECT [if(1 > 2, arrayReverseSort([2, 1, NULL, 3]), concat(arraySlice(arrayReverseSort([2, 1, NULL, 3]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1))), If(1 > 2, arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), arrayConcat(arraySlice(arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), (length([2, 1, NULL, 3]) - 1) + 1), arraySlice(arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), 1, length([2, 1, NULL, 3]) - 1)))] AS t"
},
{
"print t = array_sort_desc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]), true)",
"SELECT [if(true, arrayReverseSort([2, 1, NULL, 3]), concat(arraySlice(arrayReverseSort([2, 1, NULL, 3]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1))), If(true, arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), arrayConcat(arraySlice(arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), (length([2, 1, NULL, 3]) - 1) + 1), arraySlice(arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), 1, length([2, 1, NULL, 3]) - 1)))] AS t"
},
{
"print t = array_sort_desc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]), 1 < 2)",
"SELECT [if(1 < 2, arrayReverseSort([2, 1, NULL, 3]), concat(arraySlice(arrayReverseSort([2, 1, NULL, 3]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1))), If(1 < 2, arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), arrayConcat(arraySlice(arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), (length([2, 1, NULL, 3]) - 1) + 1), arraySlice(arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), 1, length([2, 1, NULL, 3]) - 1)))] AS t"
},
{
"print t = array_sort_desc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]))",
"SELECT [if(true, arrayReverseSort([2, 1, NULL, 3]), concat(arraySlice(arrayReverseSort([2, 1, NULL, 3]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1))), If(true, arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), arrayConcat(arraySlice(arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), (length([2, 1, NULL, 3]) - 1) + 1), arraySlice(arrayReverseSort((x, y) -> y, [20, 10, 40, 30], [2, 1, NULL, 3]), 1, length([2, 1, NULL, 3]) - 1)))] AS t"
},
{
"print t = array_sort_desc(dynamic(['b', 'a', null]), dynamic(['p', 'q', 'r', 's']), 1 < 2)",
"SELECT [if(1 < 2, arrayReverseSort(['b', 'a', NULL]), concat(arraySlice(arrayReverseSort(['b', 'a', NULL]) AS as1, indexOf(as1, NULL) AS len1), arraySlice(as1, 1, len1 - 1))), [NULL]] 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)]"

View File

@ -9,58 +9,58 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_operator_in_sql, ParserTest,
::testing::ValuesIn(std::initializer_list<ParserTestCase>{
{
"select * from kql(Customers | where FirstName !in ('Peter', 'Latoya'))",
"SELECT *\nFROM\n(\n SELECT *\n FROM Customers\n WHERE FirstName NOT IN ('Peter', 'Latoya')\n)"
"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\n(\n SELECT *\n FROM Customers\n WHERE NOT (FirstName ILIKE '%Pet%')\n)"
"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\n(\n SELECT *\n FROM Customers\n WHERE NOT (FirstName LIKE '%Pet%')\n)"
"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\n(\n SELECT *\n FROM Customers\n WHERE NOT (FirstName ILIKE '%ter')\n)"
"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\n(\n SELECT *\n FROM Customers\n WHERE NOT endsWith(FirstName, 'ter')\n)"
"SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE NOT endsWith(FirstName, 'ter')\n)"
},
{
"select * from kql(Customers | where FirstName != 'Peter');",
"SELECT *\nFROM\n(\n SELECT *\n FROM Customers\n WHERE FirstName != 'Peter'\n)"
"SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE FirstName != 'Peter'\n)"
},
{
"select * from kql(Customers | where FirstName !has 'Peter');",
"SELECT *\nFROM\n(\n SELECT *\n FROM Customers\n WHERE NOT hasTokenCaseInsensitive(FirstName, 'Peter')\n)"
"SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE NOT ifNull(hasTokenCaseInsensitiveOrNull(FirstName, 'Peter'), hasTokenCaseInsensitive(FirstName, 'Peter') AND (positionCaseInsensitive(FirstName, 'Peter') > 0))\n)"
},
{
"select * from kql(Customers | where FirstName !has_cs 'peter');",
"SELECT *\nFROM\n(\n SELECT *\n FROM Customers\n WHERE NOT hasToken(FirstName, 'peter')\n)"
"SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE NOT ifNull(hasTokenOrNull(FirstName, 'peter'), hasToken(FirstName, 'peter') AND (position(FirstName, 'peter') > 0))\n)"
},
{
"select * from kql(Customers | where FirstName !hasprefix 'Peter');",
"SELECT *\nFROM\n(\n SELECT *\n FROM Customers\n WHERE (NOT (FirstName ILIKE 'Peter%')) AND (NOT (FirstName ILIKE '% Peter%'))\n)"
"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\n(\n SELECT *\n FROM Customers\n WHERE (NOT startsWith(FirstName, 'Peter')) AND (NOT (FirstName LIKE '% Peter%'))\n)"
"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\n(\n SELECT *\n FROM Customers\n WHERE (NOT (FirstName ILIKE '%Peter')) AND (NOT (FirstName ILIKE '%Peter %'))\n)"
"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\n(\n SELECT *\n FROM Customers\n WHERE (NOT endsWith(FirstName, 'Peter')) AND (NOT (FirstName LIKE '%Peter %'))\n)"
"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\n(\n SELECT *\n FROM Customers\n WHERE NOT (FirstName ILIKE 'Peter%')\n)"
"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\n(\n SELECT *\n FROM Customers\n WHERE NOT startsWith(FirstName, 'Peter')\n)"
"SELECT *\nFROM view(\n SELECT *\n FROM Customers\n WHERE NOT startsWith(FirstName, 'Peter')\n)"
}
})));

View File

@ -441,19 +441,19 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserTest,
},
{
"Customers | where Education has 'School'",
"SELECT *\nFROM Customers\nWHERE hasTokenCaseInsensitive(Education, 'School')"
"SELECT *\nFROM Customers\nWHERE ifNull(hasTokenCaseInsensitiveOrNull(Education, 'School'), hasTokenCaseInsensitive(Education, 'School') AND (positionCaseInsensitive(Education, 'School') > 0))"
},
{
"Customers | where Education !has 'School'",
"SELECT *\nFROM Customers\nWHERE NOT hasTokenCaseInsensitive(Education, 'School')"
"SELECT *\nFROM Customers\nWHERE NOT ifNull(hasTokenCaseInsensitiveOrNull(Education, 'School'), hasTokenCaseInsensitive(Education, 'School') AND (positionCaseInsensitive(Education, 'School') > 0))"
},
{
"Customers | where Education has_cs 'School'",
"SELECT *\nFROM Customers\nWHERE hasToken(Education, 'School')"
"SELECT *\nFROM Customers\nWHERE ifNull(hasTokenOrNull(Education, 'School'), hasToken(Education, 'School') AND (position(Education, 'School') > 0))"
},
{
"Customers | where Education !has_cs 'School'",
"SELECT *\nFROM Customers\nWHERE NOT hasToken(Education, 'School')"
"SELECT *\nFROM Customers\nWHERE NOT ifNull(hasTokenOrNull(Education, 'School'), hasToken(Education, 'School') AND (position(Education, 'School') > 0))"
},
{
"Customers | where FirstName matches regex 'P.*r'",
@ -473,15 +473,15 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserTest,
},
{
"Customers|where Occupation has_any ('Skilled','abcd')",
"SELECT *\nFROM Customers\nWHERE hasTokenCaseInsensitive(Occupation, 'Skilled') OR hasTokenCaseInsensitive(Occupation, 'abcd')"
"SELECT *\nFROM Customers\nWHERE ifNull(hasTokenCaseInsensitiveOrNull(Occupation, 'Skilled'), hasTokenCaseInsensitive(Occupation, 'Skilled') AND (positionCaseInsensitive(Occupation, 'Skilled') > 0)) OR ifNull(hasTokenCaseInsensitiveOrNull(Occupation, 'abcd'), hasTokenCaseInsensitive(Occupation, 'abcd') AND (positionCaseInsensitive(Occupation, 'abcd') > 0))"
},
{
"Customers|where Occupation has_all ('Skilled','abcd')",
"SELECT *\nFROM Customers\nWHERE hasTokenCaseInsensitive(Occupation, 'Skilled') AND hasTokenCaseInsensitive(Occupation, 'abcd')"
"SELECT *\nFROM Customers\nWHERE ifNull(hasTokenCaseInsensitiveOrNull(Occupation, 'Skilled'), hasTokenCaseInsensitive(Occupation, 'Skilled') AND (positionCaseInsensitive(Occupation, 'Skilled') > 0)) AND ifNull(hasTokenCaseInsensitiveOrNull(Occupation, 'abcd'), hasTokenCaseInsensitive(Occupation, 'abcd') AND (positionCaseInsensitive(Occupation, 'abcd') > 0))"
},
{
"Customers|where Occupation has_all (strcat('Skill','ed'),'Manual')",
"SELECT *\nFROM Customers\nWHERE hasTokenCaseInsensitive(Occupation, concat('Skill', 'ed')) AND hasTokenCaseInsensitive(Occupation, 'Manual')"
"SELECT *\nFROM Customers\nWHERE ifNull(hasTokenCaseInsensitiveOrNull(Occupation, concat('Skill', 'ed')), hasTokenCaseInsensitive(Occupation, 'concat') AND hasTokenCaseInsensitive(Occupation, 'Skill') AND hasTokenCaseInsensitive(Occupation, 'ed') AND (positionCaseInsensitive(Occupation, concat('Skill', 'ed')) > 0)) AND ifNull(hasTokenCaseInsensitiveOrNull(Occupation, 'Manual'), hasTokenCaseInsensitive(Occupation, 'Manual') AND (positionCaseInsensitive(Occupation, 'Manual') > 0))"
},
{
"Customers | where Occupation == strcat('Pro','fessional') | take 1",

View File

@ -29,4 +29,4 @@ Costco Snargaluff 2016-09-10 20000
-- extend #9 --
-- extend #10 --
-- extend #11 --
5 [[2,1],[4,3]]
5 [2,1]

View File

@ -54,5 +54,5 @@ 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] [4,3]
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]));

View File

@ -66,31 +66,31 @@
[[1],[2,3],[4,5]]
[[1,2,3,4],[],[4,5]]
-- array_sort_asc()
[['a','c','c','d',NULL]]
[[1,2,3,4]]
(['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]]
(['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']]
(['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]]
(['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]]
(['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']]
(['Ringo','Paul','John','George'])
(['yellow','green','blue',NULL,NULL])
([NULL,NULL,'yellow','green','blue'])
-- jaccard_index()
0.75
0

View File

@ -73,31 +73,31 @@ 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 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']))[0];
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 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, null, null]) , false);
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 v=array_sort_asc(dynamic([null,"blue","yellow","green",null]), false);
print array_sort_asc(dynamic([null,"blue","yellow","green",null]), false);
print '-- array_sort_desc()';
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']))[0];
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 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, null, null]) , false);
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 v=array_sort_desc(dynamic([null,"blue","yellow","green",null]), false);
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([]));

View File

@ -61,16 +61,16 @@ 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;
--Customers | where Occupation has 'skilled' | order by LastName;
print '';
print '-- Customers | where Occupation !has \'skilled\'';
Customers | where Occupation !has 'skilled' | order by LastName;
--Customers | where Occupation !has 'skilled' | order by LastName;
print '';
print '-- Customers | where Occupation has \'Skilled\'';
Customers | where Occupation has 'Skilled'| order by LastName;
--Customers | where Occupation has 'Skilled'| order by LastName;
print '';
print '-- Customers | where Occupation !has \'Skilled\'';
Customers | where Occupation !has 'Skilled'| order by LastName;
--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;
@ -158,10 +158,10 @@ Customers | where Age in ((Customers|project Age|where Age < 30)) | order by Las
-- 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;
--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');
--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;

View File

@ -24,9 +24,9 @@ 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 * from kql(Customers | where FirstName !has 'Peter');
Select '-- #9 --' ;
select * from kql(Customers | where FirstName !has_cs 'peter');
--select * from kql(Customers | where FirstName !has_cs 'peter');
Select '-- #10 --' ;
select * from kql(Customers | where FirstName !hasprefix 'Peter');
Select '-- #11 --' ;