Fix segmentation fault in KQL parser when the input query exceeds the max_query_size (#59626)

* Fix_kql_issue_found_by_wingfuzz

This commit fix the issues:
 https://github.com/ClickHouse/ClickHouse/issues/59036
 https://github.com/ClickHouse/ClickHouse/issues/59037

both issues are same reason, the input query exceed the max_query_size,
so the condition isEnd() of token is not meet and cause the assertion failure

* fix_kql_issue_found_by_wingfuzz: use isValid instead of TokenType::EndOfStream

* fix_kql_issue_found_by_wingfuzz: make functional test result consist

* fix_kql_issue_found_by_wingfuzz: update test case for makeseries

* fix_kql_issue_found_by_wingfuzz: disable makeseries

* fix_kql_issue_found_by_wingfuzz:
 use isvalid() function to replace isEnd() function of TokenIterator to check the end of stream

* fix_kql_issue_found_by_wingfuzz: add test case for max_query_size

* fix_kql_issue_found_by_wingfuzz: fix AST structure

* fix_kql_issue_found_by_wingfuzz: make sure the max query size test is in the dialect of kusto

* fix_kql_issue_found_by_wingfuzz : restore max query size after test

* fix_kql_issue_found_by_wingfuzz : fix typo

---------

Co-authored-by: János Benjamin Antal <benjamin.antal@clickhouse.com>
This commit is contained in:
Yong Wang 2024-02-26 07:31:47 -05:00 committed by GitHub
parent d98fbcb8d1
commit 795c1a98dc
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
47 changed files with 2623 additions and 51 deletions

View File

@ -103,7 +103,6 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
extern const int QUERY_WAS_CANCELLED;
extern const int INCORRECT_DATA;
extern const int SUPPORT_IS_DISABLED;
}
namespace FailPoints
@ -709,7 +708,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
{
if (settings.dialect == Dialect::kusto && !internal)
{
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Kusto dialect is disabled until these two bugs will be fixed: https://github.com/ClickHouse/ClickHouse/issues/59037 and https://github.com/ClickHouse/ClickHouse/issues/59036");
ParserKQLStatement parser(end, settings.allow_settings_after_format_in_insert);
/// TODO: parser should fail early when max_query_size limit is reached.
ast = parseKQLQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth);
}
else if (settings.dialect == Dialect::prql && !internal)
{

View File

@ -73,7 +73,7 @@ bool IParserKQLFunction::directMapping(
int argument_count = 0;
const auto begin = pos;
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
while (isValidKQLPos(pos) && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
{
if (pos != begin)
out.append(", ");
@ -148,11 +148,11 @@ String IParserKQLFunction::getConvertedArgument(const String & fn_name, IParser:
if (pos->type == TokenType::ClosingRoundBracket || pos->type == TokenType::ClosingSquareBracket)
return {};
if (pos->isEnd() || pos->type == TokenType::PipeMark || pos->type == TokenType::Semicolon)
if (!isValidKQLPos(pos) || pos->type == TokenType::PipeMark || pos->type == TokenType::Semicolon)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Need more argument(s) in function: {}", fn_name);
std::vector<String> tokens;
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
while (isValidKQLPos(pos) && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
{
if (pos->type == TokenType::OpeningRoundBracket)
++round_bracket_count;
@ -191,7 +191,7 @@ String IParserKQLFunction::getConvertedArgument(const String & fn_name, IParser:
{
++pos;
String array_index;
while (!pos->isEnd() && pos->type != TokenType::ClosingSquareBracket)
while (isValidKQLPos(pos) && pos->type != TokenType::ClosingSquareBracket)
{
array_index += getExpression(pos);
++pos;
@ -246,7 +246,7 @@ IParserKQLFunction::getOptionalArgument(const String & function_name, DB::IParse
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)))
while (isValidKQLPos(pos) && (!scopes.empty() || (pos->type != DB::TokenType::Comma && pos->type != DB::TokenType::ClosingRoundBracket)))
{
const auto token_type = pos->type;
if (isOpeningBracket(token_type))
@ -357,7 +357,7 @@ String IParserKQLFunction::getExpression(IParser::Pos & pos)
{
++pos;
String array_index;
while (!pos->isEnd() && pos->type != TokenType::ClosingSquareBracket)
while (isValidKQLPos(pos) && pos->type != TokenType::ClosingSquareBracket)
{
array_index += getExpression(pos);
++pos;

View File

@ -7,6 +7,7 @@
#include <Parsers/Kusto/ParserKQLDateTypeTimespan.h>
#include <Parsers/Kusto/ParserKQLQuery.h>
#include <Parsers/Kusto/ParserKQLStatement.h>
#include <Parsers/Kusto/Utilities.h>
#include <Parsers/ParserSetQuery.h>
#include "Poco/String.h"
#include <format>
@ -51,7 +52,7 @@ bool DatatypeDatetime::convertImpl(String & out, IParser::Pos & pos)
else
{
auto start = pos;
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
while (isValidKQLPos(pos) && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
{
++pos;
if (pos->type == TokenType::ClosingRoundBracket)
@ -77,7 +78,7 @@ bool DatatypeDynamic::convertImpl(String & out, IParser::Pos & pos)
if (pos->type == TokenType::OpeningCurlyBrace)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Property bags are not supported for now in {}", function_name);
while (!pos->isEnd() && pos->type != TokenType::ClosingRoundBracket)
while (isValidKQLPos(pos) && pos->type != TokenType::ClosingRoundBracket)
{
if (const auto token_type = pos->type; token_type == TokenType::BareWord || token_type == TokenType::Number
|| token_type == TokenType::QuotedIdentifier || token_type == TokenType::StringLiteral)
@ -117,7 +118,7 @@ bool DatatypeGuid::convertImpl(String & out, IParser::Pos & pos)
else
{
auto start = pos;
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
while (isValidKQLPos(pos) && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
{
++pos;
if (pos->type == TokenType::ClosingRoundBracket)

View File

@ -14,6 +14,7 @@
#include <Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.h>
#include <Parsers/Kusto/ParserKQLQuery.h>
#include <Parsers/Kusto/ParserKQLStatement.h>
#include <Parsers/Kusto/Utilities.h>
#include <Parsers/ParserSetQuery.h>
#include "Poco/String.h"
namespace DB::ErrorCodes
@ -521,7 +522,7 @@ bool MakeTimeSpan::convertImpl(String & out, IParser::Pos & pos)
String second;
int arg_count = 0;
std::vector<String> args;
while (!pos->isEnd() && pos->type != TokenType::ClosingRoundBracket)
while (isValidKQLPos(pos) && pos->type != TokenType::ClosingRoundBracket)
{
String arg = getConvertedArgument(fn_name, pos);
args.insert(args.begin(), arg);
@ -588,7 +589,7 @@ bool MakeDateTime::convertImpl(String & out, IParser::Pos & pos)
String arguments;
int arg_count = 0;
while (!pos->isEnd() && pos->type != TokenType::ClosingRoundBracket)
while (isValidKQLPos(pos) && pos->type != TokenType::ClosingRoundBracket)
{
String arg = getConvertedArgument(fn_name, pos);
if (pos->type == TokenType::Comma)

View File

@ -3,7 +3,7 @@
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
#include <Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h>
#include <Parsers/Kusto/KustoFunctions/KQLStringFunctions.h>
#include <Parsers/Kusto/Utilities.h>
#include <boost/algorithm/string/classification.hpp>
#include <boost/algorithm/string/split.hpp>
#include <Poco/String.h>
@ -590,7 +590,7 @@ bool StrCatDelim::convertImpl(String & out, IParser::Pos & pos)
int arg_count = 0;
String args;
while (!pos->isEnd() && pos->type != TokenType::Semicolon && pos->type != TokenType::ClosingRoundBracket)
while (isValidKQLPos(pos) && pos->type != TokenType::Semicolon && pos->type != TokenType::ClosingRoundBracket)
{
++pos;
String arg = getConvertedArgument(fn_name, pos);

View File

@ -7,6 +7,7 @@
#include <Parsers/Kusto/ParserKQLOperators.h>
#include <Parsers/Kusto/ParserKQLProject.h>
#include <Parsers/Kusto/ParserKQLQuery.h>
#include <Parsers/Kusto/Utilities.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/ParserTablesInSelectQuery.h>
@ -44,7 +45,7 @@ bool ParserKQLExtend ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
int32_t round_bracket_count = 0;
int32_t square_bracket_count = 0;
while (!npos->isEnd())
while (isValidKQLPos(npos))
{
if (npos->type == TokenType::OpeningRoundBracket)
++round_bracket_count;

View File

@ -7,6 +7,7 @@
#include <Parsers/Kusto/ParserKQLMakeSeries.h>
#include <Parsers/Kusto/ParserKQLOperators.h>
#include <Parsers/Kusto/ParserKQLQuery.h>
#include <Parsers/Kusto/Utilities.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ParserTablesInSelectQuery.h>
@ -49,7 +50,7 @@ bool ParserKQLMVExpand::parseColumnArrayExprs(ColumnArrayExprs & column_array_ex
String to_type;
--expr_end_pos;
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
while (isValidKQLPos(pos) && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
{
if (pos->type == TokenType::OpeningRoundBracket)
++bracket_count;
@ -125,9 +126,9 @@ bool ParserKQLMVExpand::parseColumnArrayExprs(ColumnArrayExprs & column_array_ex
if (String(pos->begin, pos->end) == "limit")
break;
if (!pos->isEnd())
if (isValidKQLPos(pos))
++pos;
if (pos->isEnd() || pos->type == TokenType::PipeMark || pos->type == TokenType::Semicolon)
if (!isValidKQLPos(pos) || pos->type == TokenType::PipeMark || pos->type == TokenType::Semicolon)
{
if (expr_end_pos < expr_begin_pos)
{

View File

@ -6,6 +6,7 @@
#include <Parsers/Kusto/ParserKQLMakeSeries.h>
#include <Parsers/Kusto/ParserKQLOperators.h>
#include <Parsers/Kusto/ParserKQLQuery.h>
#include <Parsers/Kusto/Utilities.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/ParserTablesInSelectQuery.h>
@ -39,7 +40,7 @@ bool ParserKQLMakeSeries ::parseAggregationColumns(AggregationColumns & aggregat
ParserToken close_bracket(TokenType::ClosingRoundBracket);
ParserToken comma(TokenType::Comma);
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
while (isValidKQLPos(pos) && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
{
String alias;
String aggregation_fun;
@ -96,7 +97,7 @@ bool ParserKQLMakeSeries ::parseFromToStepClause(FromToStepClause & from_to_step
auto step_pos = begin;
auto end_pos = begin;
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
while (isValidKQLPos(pos) && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
{
if (String(pos->begin, pos->end) == "from")
from_pos = pos;
@ -175,7 +176,7 @@ bool ParserKQLMakeSeries ::makeSeries(KQLMakeSeries & kql_make_series, ASTPtr &
Tokens tokens(src.c_str(), src.c_str() + src.size());
IParser::Pos pos(tokens, max_depth);
String res;
while (!pos->isEnd())
while (isValidKQLPos(pos))
{
String tmp = String(pos->begin, pos->end);
if (tmp == "parseDateTime64BestEffortOrNull")
@ -201,7 +202,7 @@ bool ParserKQLMakeSeries ::makeSeries(KQLMakeSeries & kql_make_series, ASTPtr &
std::vector<String> group_expression_tokens;
Tokens tokens(group_expression.c_str(), group_expression.c_str() + group_expression.size());
IParser::Pos pos(tokens, max_depth);
while (!pos->isEnd())
while (isValidKQLPos(pos))
{
if (String(pos->begin, pos->end) == "AS")
{

View File

@ -5,6 +5,7 @@
#include <Parsers/Kusto/ParserKQLOperators.h>
#include <Parsers/Kusto/ParserKQLQuery.h>
#include <Parsers/Kusto/ParserKQLStatement.h>
#include <Parsers/Kusto/Utilities.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/formatAST.h>
@ -148,7 +149,7 @@ String KQLOperators::genHasAnyAllOpExpr(std::vector<String> & tokens, IParser::P
String logic_op = (kql_op == "has_all") ? " and " : " or ";
while (!token_pos->isEnd() && token_pos->type != TokenType::PipeMark && token_pos->type != TokenType::Semicolon)
while (isValidKQLPos(token_pos) && token_pos->type != TokenType::PipeMark && token_pos->type != TokenType::Semicolon)
{
auto tmp_arg = IParserKQLFunction::getExpression(token_pos);
if (token_pos->type == TokenType::Comma)
@ -217,7 +218,7 @@ String genInOpExprCis(std::vector<String> & tokens, DB::IParser::Pos & token_pos
--token_pos;
new_expr += ch_op;
while (!token_pos->isEnd() && token_pos->type != DB::TokenType::PipeMark && token_pos->type != DB::TokenType::Semicolon)
while (isValidKQLPos(token_pos) && token_pos->type != DB::TokenType::PipeMark && token_pos->type != DB::TokenType::Semicolon)
{
auto tmp_arg = String(token_pos->begin, token_pos->end);
if (token_pos->type != DB::TokenType::Comma && token_pos->type != DB::TokenType::ClosingRoundBracket
@ -329,7 +330,7 @@ bool KQLOperators::convert(std::vector<String> & tokens, IParser::Pos & pos)
{
auto begin = pos;
if (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
if (isValidKQLPos(pos) && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
{
KQLOperatorValue op_value = KQLOperatorValue::none;
@ -339,14 +340,14 @@ bool KQLOperators::convert(std::vector<String> & tokens, IParser::Pos & pos)
if (token == "!")
{
++pos;
if (pos->isEnd() || pos->type == TokenType::PipeMark || pos->type == TokenType::Semicolon)
if (!isValidKQLPos(pos) || 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 (isValidKQLPos(pos) && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
{
if (String(pos->begin, pos->end) == "regex")
op += " regex";
@ -360,7 +361,7 @@ bool KQLOperators::convert(std::vector<String> & tokens, IParser::Pos & pos)
}
++pos;
if (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
if (isValidKQLPos(pos) && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
{
if (String(pos->begin, pos->end) == "~")
op += "~";

View File

@ -20,6 +20,7 @@
#include <Parsers/Kusto/ParserKQLStatement.h>
#include <Parsers/Kusto/ParserKQLSummarize.h>
#include <Parsers/Kusto/ParserKQLTable.h>
#include <Parsers/Kusto/Utilities.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ParserTablesInSelectQuery.h>
@ -59,33 +60,34 @@ bool ParserKQLBase::setSubQuerySource(ASTPtr & select_query, ASTPtr & source, bo
if (!select_query || !select_query->as<ASTSelectQuery>()->tables()
|| select_query->as<ASTSelectQuery>()->tables()->as<ASTTablesInSelectQuery>()->children.empty())
return false;
table_expr = select_query->as<ASTSelectQuery>()->tables()->as<ASTTablesInSelectQuery>()->children[0];
table_expr = select_query->as<ASTSelectQuery>()->tables()->as<ASTTablesInSelectQuery>()->children.at(0);
table_expr->as<ASTTablesInSelectQueryElement>()->table_expression
= source->as<ASTSelectQuery>()->tables()->children[0]->as<ASTTablesInSelectQueryElement>()->table_expression;
= source->as<ASTSelectQuery>()->tables()->children.at(0)->as<ASTTablesInSelectQueryElement>()->table_expression;
table_expr->children.at(0) = table_expr->as<ASTTablesInSelectQueryElement>()->table_expression;
return true;
}
if (!select_query || select_query->as<ASTTablesInSelectQuery>()->children.empty()
|| !select_query->as<ASTTablesInSelectQuery>()->children[0]->as<ASTTablesInSelectQueryElement>()->table_expression
|| !select_query->as<ASTTablesInSelectQuery>()->children.at(0)->as<ASTTablesInSelectQueryElement>()->table_expression
|| select_query->as<ASTTablesInSelectQuery>()
->children[0]
->children.at(0)
->as<ASTTablesInSelectQueryElement>()
->table_expression->as<ASTTableExpression>()
->subquery->children.empty()
|| select_query->as<ASTTablesInSelectQuery>()
->children[0]
->children.at(0)
->as<ASTTablesInSelectQueryElement>()
->table_expression->as<ASTTableExpression>()
->subquery->children[0]
->subquery->children.at(0)
->as<ASTSelectWithUnionQuery>()
->list_of_selects->children.empty()
|| select_query->as<ASTTablesInSelectQuery>()
->children[0]
->children.at(0)
->as<ASTTablesInSelectQueryElement>()
->table_expression->as<ASTTableExpression>()
->subquery->children[0]
->subquery->children.at(0)
->as<ASTSelectWithUnionQuery>()
->list_of_selects->children[0]
->list_of_selects->children.at(0)
->as<ASTSelectQuery>()
->tables()
->as<ASTTablesInSelectQuery>()
@ -93,28 +95,29 @@ bool ParserKQLBase::setSubQuerySource(ASTPtr & select_query, ASTPtr & source, bo
return false;
table_expr = select_query->as<ASTTablesInSelectQuery>()
->children[0]
->children.at(0)
->as<ASTTablesInSelectQueryElement>()
->table_expression->as<ASTTableExpression>()
->subquery->children[0]
->subquery->children.at(0)
->as<ASTSelectWithUnionQuery>()
->list_of_selects->children[0]
->list_of_selects->children.at(0)
->as<ASTSelectQuery>()
->tables()
->as<ASTTablesInSelectQuery>()
->children[0];
->children.at(0);
if (!src_is_subquery)
{
table_expr->as<ASTTablesInSelectQueryElement>()->table_expression
= source->as<ASTSelectQuery>()->tables()->children[0]->as<ASTTablesInSelectQueryElement>()->table_expression;
= source->as<ASTSelectQuery>()->tables()->children.at(0)->as<ASTTablesInSelectQueryElement>()->table_expression;
}
else
{
table_expr->as<ASTTablesInSelectQueryElement>()->table_expression
= source->children[0]->as<ASTTablesInSelectQueryElement>()->table_expression;
= source->children.at(0)->as<ASTTablesInSelectQueryElement>()->table_expression;
}
table_expr->children.at(0) = table_expr->as<ASTTablesInSelectQueryElement>()->table_expression;
return true;
}
@ -130,7 +133,7 @@ String ParserKQLBase::getExprFromPipe(Pos & pos)
{
BracketCount bracket_count;
auto end = pos;
while (!end->isEnd() && end->type != TokenType::Semicolon)
while (isValidKQLPos(end) && end->type != TokenType::Semicolon)
{
bracket_count.count(end);
if (end->type == TokenType::PipeMark && bracket_count.isZero())
@ -149,7 +152,7 @@ String ParserKQLBase::getExprFromToken(Pos & pos)
comma_pos.push_back(pos);
size_t paren_count = 0;
while (!pos->isEnd() && pos->type != TokenType::Semicolon)
while (isValidKQLPos(pos) && pos->type != TokenType::Semicolon)
{
if (pos->type == TokenType::PipeMark && paren_count == 0)
break;
@ -373,7 +376,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
uint16_t bracket_count = 0;
while (!pos->isEnd() && pos->type != TokenType::Semicolon)
while (isValidKQLPos(pos) && pos->type != TokenType::Semicolon)
{
if (pos->type == TokenType::OpeningRoundBracket)
++bracket_count;
@ -383,6 +386,9 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (pos->type == TokenType::PipeMark && bracket_count == 0)
{
++pos;
if (!isValidKQLPos(pos))
return false;
String kql_operator(pos->begin, pos->end);
auto validate_kql_operator = [&]
@ -390,6 +396,9 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (kql_operator == "order" || kql_operator == "sort")
{
++pos;
if (!isValidKQLPos(pos))
return false;
ParserKeyword s_by("by");
if (s_by.ignore(pos, expected))
{
@ -401,6 +410,9 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
auto op_pos_begin = pos;
++pos;
if (!isValidKQLPos(pos))
return false;
ParserToken s_dash(TokenType::Minus);
if (s_dash.ignore(pos, expected))
{
@ -418,6 +430,9 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!validate_kql_operator())
return false;
++pos;
if (!isValidKQLPos(pos))
return false;
operation_pos.push_back(std::make_pair(kql_operator, pos));
}
else
@ -627,6 +642,7 @@ bool ParserSimpleCHSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
ASTPtr node_table_in_select_query_element = std::make_shared<ASTTablesInSelectQueryElement>();
node_table_in_select_query_element->as<ASTTablesInSelectQueryElement>()->table_expression = node_table_expr;
node_table_in_select_query_element->children.emplace_back(node_table_expr);
ASTPtr res = std::make_shared<ASTTablesInSelectQuery>();
res->children.emplace_back(node_table_in_select_query_element);

View File

@ -4,6 +4,7 @@
#include <Parsers/IParserBase.h>
#include <Parsers/Kusto/ParserKQLQuery.h>
#include <Parsers/Kusto/ParserKQLSort.h>
#include <Parsers/Kusto/Utilities.h>
namespace DB
{
@ -24,7 +25,7 @@ bool ParserKQLSort::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!order_list.parse(pos_backup, order_expression_list, expected))
return false;
while (!new_pos->isEnd() && new_pos->type != TokenType::PipeMark && new_pos->type != TokenType::Semicolon)
while (isValidKQLPos(new_pos) && new_pos->type != TokenType::PipeMark && new_pos->type != TokenType::Semicolon)
{
String tmp(new_pos->begin, new_pos->end);
if (tmp == "desc" || tmp == "asc")

View File

@ -5,6 +5,7 @@
#include <Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h>
#include <Parsers/Kusto/ParserKQLQuery.h>
#include <Parsers/Kusto/ParserKQLStatement.h>
#include <Parsers/Kusto/Utilities.h>
#include <Parsers/ParserSetQuery.h>
namespace DB
@ -79,7 +80,7 @@ bool ParserKQLTableFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
{
++paren_count;
auto pos_start = pos;
while (!pos->isEnd())
while (isValidKQLPos(pos))
{
if (pos->type == TokenType::ClosingRoundBracket)
--paren_count;

View File

@ -15,6 +15,7 @@
#include <Parsers/IParserBase.h>
#include <Parsers/Kusto/ParserKQLQuery.h>
#include <Parsers/Kusto/ParserKQLSummarize.h>
#include <Parsers/Kusto/Utilities.h>
#include <Parsers/ParserSampleRatio.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/ParserSetQuery.h>
@ -120,7 +121,7 @@ bool ParserKQLSummarize::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
if (String(equal_pos->begin, equal_pos->end) != "=")
{
String groupby_fun = String(begin_pos->begin, begin_pos->end);
if (equal_pos->isEnd() || equal_pos->type == TokenType::Comma || equal_pos->type == TokenType::Semicolon
if (!equal_pos.isValid() || equal_pos->type == TokenType::Comma || equal_pos->type == TokenType::Semicolon
|| equal_pos->type == TokenType::PipeMark)
{
expr = groupby_fun;
@ -151,7 +152,7 @@ bool ParserKQLSummarize::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
}
};
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
while (isValidKQLPos(pos) && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
{
if (pos->type == TokenType::OpeningRoundBracket)
++bracket_count;

View File

@ -71,4 +71,12 @@ ASTPtr wrapInSelectWithUnion(const ASTPtr & select_query)
return select_with_union_query;
}
bool isValidKQLPos(IParser::Pos & pos)
{
return (pos.isValid() ||
pos->type == TokenType::ErrorSingleExclamationMark || // allow kql negative operators
pos->type == TokenType::ErrorWrongNumber || // allow kql timespan data type with decimal like 2.6h
std::string_view(pos->begin, pos->end) == "~"); // allow kql Case-Sensitive operators
}
}

View File

@ -10,4 +10,5 @@ String extractTokenWithoutQuotes(IParser::Pos & pos);
void setSelectAll(ASTSelectQuery & select_query);
String wildcardToRegex(const String & wildcard);
ASTPtr wrapInSelectWithUnion(const ASTPtr & select_query);
bool isValidKQLPos(IParser::Pos & pos);
}

View File

@ -0,0 +1,4 @@
-- test create table --
Theodore
Diaz
Theodore Diaz 28

View File

@ -0,0 +1,29 @@
DROP TABLE IF EXISTS Customers;
CREATE TABLE Customers
(
FirstName Nullable(String),
LastName String,
Occupation String,
Education String,
Age Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28),('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38);
Select '-- test create table --' ;
Select * from kql(Customers|project FirstName) limit 1;;
DROP TABLE IF EXISTS kql_table1;
CREATE TABLE kql_table1 ENGINE = Memory AS select *, now() as new_column From kql(Customers | project LastName | filter LastName=='Diaz');
select LastName from kql_table1 limit 1;
DROP TABLE IF EXISTS kql_table2;
CREATE TABLE kql_table2
(
FirstName Nullable(String),
LastName String,
Age Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO kql_table2 select * from kql(Customers|project FirstName,LastName,Age | filter FirstName=='Theodore');
select * from kql_table2 limit 1;
-- select * from kql(Customers | where FirstName !in ("test", "test2"));
DROP TABLE IF EXISTS Customers;
DROP TABLE IF EXISTS kql_table1;
DROP TABLE IF EXISTS kql_table2;

View File

@ -0,0 +1,105 @@
-- bool
true
\N
-- int
123
\N
-- long
123
255
-1
\N
456
-- real
0.01
\N
nan
inf
-inf
-- datetime
2015-12-31 23:59:59.900000000
2015-12-31 00:00:00.000000000
2014-05-25 08:20:03.123456000
2014-11-08 15:55:55.000000000
2014-11-08 15:55:00.000000000
2014-11-08 00:00:00.000000000
\N
2014-05-25 08:20:03.123456000
2014-11-08 15:55:55.123456000
-- time
1216984.12345
45055.123
86400
-86400
6.000000000000001e-9
6e-7
172800
259200
-- guid
\N
-- timespan (time)
172800
1800
10
0.1
0.00001
1e-7
1120343
-- null
1
\N \N \N \N \N
-- decimal
\N
123.345
100000
-- dynamic
\N
1
86400
[1,2,3]
[[1],[2],[3]]
['a','b','c']
-- cast functions
true
1
-- tobool("false")
false
1
-- tobool(1)
true
1
-- tobool(123)
true
1
-- tobool("abc")
\N
\N
-- todouble()
123.4
\N
-- toreal()
123.4
\N
-- toint()
1
\N
-- tostring()
123
1
-- todatetime()
1
\N
-- make_timespan()
01:12:00 01:12:30 1.12:30:55
-- totimespan()
1e-7
60
\N
1120343
-- tolong()
123
\N
-- todecimal()
123.345
\N
\N

View File

@ -0,0 +1,117 @@
set dialect = 'kusto';
print '-- bool'
print bool(true);
print bool(true);
print bool(null);
print '-- int';
print int(123);
print int(null);
print int('4'); -- { clientError BAD_ARGUMENTS }
print '-- long';
print long(123);
print long(0xff);
print long(-1);
print long(null);
print 456;
print '-- real';
print real(0.01);
print real(null);
print real(nan);
print real(+inf);
print real(-inf);
print double('4.2'); -- { clientError BAD_ARGUMENTS }
print '-- datetime';
print datetime(2015-12-31 23:59:59.9);
print datetime(2015-12-31);
print datetime('2014-05-25T08:20:03.123456');
print datetime('2014-11-08 15:55:55');
print datetime('2014-11-08 15:55');
print datetime('2014-11-08');
print datetime(null);
print datetime('2014-05-25T08:20:03.123456Z');
print datetime('2014-11-08 15:55:55.123456Z');
print '-- time';
print time('14.02:03:04.12345');
print time('12:30:55.123');
print time(1d);
print time(-1d);
print time(6nanoseconds);
print time(6tick);
print time(2);
print time(2) + 1d;
print '-- guid'
print guid(74be27de-1e4e-49d9-b579-fe0b331d3642);
print guid(null);
print '-- timespan (time)';
print timespan(2d); -- 2 days
--print timespan(1.5h); -- 1.5 hour
print timespan(30m); -- 30 minutes
print timespan(10s); -- 10 seconds
--print timespan(0.1s); -- 0.1 second
print timespan(100ms); -- 100 millisecond
print timespan(10microsecond); -- 10 microseconds
print timespan(1tick); -- 100 nanoseconds
--print timespan(1.5h) / timespan(30m);
print timespan('12.23:12:23') / timespan(1s);
print '-- null';
print isnull(null);
print bool(null), int(null), long(null), real(null), double(null);
print '-- decimal';
print decimal(null);
print decimal(123.345);
print decimal(1e5);
print '-- dynamic'; -- no support for mixed types and bags for now
print dynamic(null);
print dynamic(1);
print dynamic(timespan(1d));
print dynamic([1,2,3]);
print dynamic([[1], [2], [3]]);
print dynamic(['a', "b", 'c']);
print '-- cast functions'
print '--tobool("true")'; -- == true
print tobool('true'); -- == true
print tobool('true') == toboolean('true'); -- == true
print '-- tobool("false")'; -- == false
print tobool('false'); -- == false
print tobool('false') == toboolean('false'); -- == false
print '-- tobool(1)'; -- == true
print tobool(1); -- == true
print tobool(1) == toboolean(1); -- == true
print '-- tobool(123)'; -- == true
print tobool(123); -- == true
print tobool(123) == toboolean(123); -- == true
print '-- tobool("abc")'; -- == null
print tobool('abc'); -- == null
print tobool('abc') == toboolean('abc'); -- == null
print '-- todouble()';
print todouble('123.4');
print todouble('abc') == null;
print '-- toreal()';
print toreal("123.4");
print toreal('abc') == null;
print '-- toint()';
print toint("123") == int(123);
print toint('abc');
print '-- tostring()';
print tostring(123);
print tostring(null) == '';
print '-- todatetime()';
print todatetime("2015-12-24") == datetime(2015-12-24);
print todatetime('abc') == null;
print '-- make_timespan()';
print v1=make_timespan(1,12), v2=make_timespan(1,12,30), v3=make_timespan(1,12,30,55.123);
print '-- totimespan()';
print totimespan(1tick);
print totimespan('0.00:01:00');
print totimespan('abc');
print totimespan('12.23:12:23') / totimespan(1s);
-- print totimespan(strcat('12.', '23', ':12:', '23')) / timespan(1s); -> 1120343
print '-- tolong()';
print tolong('123');
print tolong('abc');
print '-- todecimal()';
print todecimal(123.345);
print todecimal(null);
print todecimal('abc');
-- print todecimal(4 * 2 + 3); -> 11

View File

@ -0,0 +1,27 @@
-- distinct * --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Peter Nara Skilled Manual Graduate Degree 26
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
\N why Professional Partial College 38
-- distinct one column --
Skilled Manual
Management abcd defg
Professional
-- distinct two column --
Skilled Manual Bachelors
Management abcd defg Bachelors
Skilled Manual Graduate Degree
Professional Graduate Degree
Professional Partial College
-- distinct with where --
Skilled Manual Bachelors
Management abcd defg Bachelors
Skilled Manual Graduate Degree
Professional Graduate Degree
Professional Partial College
-- distinct with where, order --
Skilled Manual Bachelors
Skilled Manual Graduate Degree
Professional Graduate Degree

View File

@ -0,0 +1,28 @@
DROP TABLE IF EXISTS Customers;
CREATE TABLE Customers
(
FirstName Nullable(String),
LastName String,
Occupation String,
Education String,
Age Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28), ('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38);
set dialect = 'kusto';
print '-- distinct * --';
Customers | distinct *;
print '-- distinct one column --';
Customers | distinct Occupation;
print '-- distinct two column --';
Customers | distinct Occupation, Education;
print '-- distinct with where --';
Customers where Age <30 | distinct Occupation, Education;
print '-- distinct with where, order --';
Customers |where Age <30 | order by Age| distinct Occupation, Education;

View File

@ -0,0 +1,32 @@
-- extend #1 --
Aldi Apple 4 2016-09-10 400
Costco Apple 2 2016-09-11 200
-- extend #2 --
Apple 200
Apple 400
-- extend #3 --
Apple cost 480 on average based on 5 samples.
Snargaluff cost 28080 on average based on 5 samples.
-- extend #4 --
1
-- extend #5 --
Aldi Apple 4 2016-09-10 Apple was purchased from Aldi for $4 on 2016-09-10 400
Costco Apple 2 2016-09-11 Apple was purchased from Costco for $2 on 2016-09-11 200
-- extend #6 --
Aldi Apple 2016-09-10 400
Costco Apple 2016-09-11 200
Aldi Apple 2016-09-10 600
Costco Snargaluff 2016-09-12 10000
Aldi Apple 2016-09-12 700
Aldi Snargaluff 2016-09-11 40000
Costco Snargaluff 2016-09-12 10400
Aldi Apple 2016-09-12 500
Aldi Snargaluff 2016-09-11 60000
Costco Snargaluff 2016-09-10 20000
-- extend #7 --
5
-- extend #8 --
-- extend #9 --
-- extend #10 --
-- extend #11 --
5 [2,1]

View File

@ -0,0 +1,58 @@
-- datatable(Supplier:string, Fruit:string, Price: real, Purchase:datetime)
-- [
-- 'Aldi','Apple',4,'2016-09-10',
-- 'Costco','Apple',2,'2016-09-11',
-- 'Aldi','Apple',6,'2016-09-10',
-- 'Costco','Snargaluff',100,'2016-09-12',
-- 'Aldi','Apple',7,'2016-09-12',
-- 'Aldi','Snargaluff',400,'2016-09-11',
-- 'Costco','Snargaluff',104,'2016-09-12',
-- 'Aldi','Apple',5,'2016-09-12',
-- 'Aldi','Snargaluff',600,'2016-09-11',
-- 'Costco','Snargaluff',200,'2016-09-10',
-- ]
DROP TABLE IF EXISTS Ledger;
CREATE TABLE Ledger
(
Supplier Nullable(String),
Fruit String ,
Price Float64,
Purchase Date
) ENGINE = Memory;
INSERT INTO Ledger VALUES ('Aldi','Apple',4,'2016-09-10'), ('Costco','Apple',2,'2016-09-11'), ('Aldi','Apple',6,'2016-09-10'), ('Costco','Snargaluff',100,'2016-09-12'), ('Aldi','Apple',7,'2016-09-12'), ('Aldi','Snargaluff',400,'2016-09-11'),('Costco','Snargaluff',104,'2016-09-12'),('Aldi','Apple',5,'2016-09-12'),('Aldi','Snargaluff',600,'2016-09-11'),('Costco','Snargaluff',200,'2016-09-10');
set dialect = 'kusto';
print '-- extend #1 --';
Ledger | extend PriceInCents = 100 * Price | take 2;
print '-- extend #2 --';
Ledger | extend PriceInCents = 100 * Price | sort by PriceInCents asc | project Fruit, PriceInCents | take 2;
print '-- extend #3 --';
Ledger | extend PriceInCents = 100 * Price | sort by PriceInCents asc | project Fruit, PriceInCents | summarize AveragePrice = avg(PriceInCents), Purchases = count() by Fruit | extend Sentence = strcat(Fruit, ' cost ', tostring(AveragePrice), ' on average based on ', tostring(Purchases), ' samples.') | project Sentence | sort by Sentence asc;
print '-- extend #4 --';
Ledger | extend a = Price | extend b = a | extend c = a, d = b + 500 | extend Pass = bool(b == a and c == a and d == b + 500) | summarize binary_all_and(Pass);
print '-- extend #5 --';
Ledger | take 2 | extend strcat(Fruit, ' was purchased from ', Supplier, ' for $', tostring(Price), ' on ', tostring(Purchase)) | extend PriceInCents = 100 * Price;
print '-- extend #6 --';
Ledger | extend Price = 100 * Price;
print '-- extend #7 --';
print a = 4 | extend a = 5;
print '-- extend #8 --';
-- print x = 5 | extend array_sort_desc(range(0, x), range(1, x + 1))
print '-- extend #9 --';
print x = 19 | extend = 4 + ; -- { clientError SYNTAX_ERROR }
print '-- extend #10 --';
Ledger | extend PriceInCents = * Price | sort by PriceInCents asc | project Fruit, PriceInCents | summarize AveragePrice = avg(PriceInCents), Purchases = count() by Fruit | extend Sentence = strcat(Fruit, ' cost ', tostring(AveragePrice), ' on average based on ', tostring(Purchases), ' samples.') | project Sentence; -- { clientError SYNTAX_ERROR }
print '-- extend #11 --'; -- should ideally return this in the future: 5 [2,1] because of the alias ex
print x = 5 | extend ex = array_sort_desc(dynamic([1, 2]), dynamic([3, 4]));

View File

@ -0,0 +1,7 @@
-- binary functions
4 7
1
1
1
7 3
1

View File

@ -0,0 +1,8 @@
set dialect='kusto';
print ' -- binary functions';
print binary_and(4,7), binary_or(4,7);
print binary_shift_left(1, 1) == binary_shift_left(1, 65);
print binary_shift_right(2, 1) == binary_shift_right(2, 65);
print binary_shift_right(binary_shift_left(1, 65), 65) == 1;
print binary_xor(2, 5), bitset_count_ones(42);
print bitset_count_ones(binary_shift_left(binary_and(4,7), 1));

View File

@ -0,0 +1,76 @@
-- dayofmonth()
31
-- dayofweek()
4.00:00:00
-- dayofyear()
365
-- getmonth()
10
-- getyear()
2015
-- hoursofday()
23
-- startofday()
2017-01-01 00:00:00.000000000
2016-12-31 00:00:00.000000000
2017-01-02 00:00:00.000000000
-- endofday()
2017-01-01 23:59:59.999999000
2016-12-31 23:59:59.999999000
2017-01-02 23:59:59.999999000
-- endofmonth()
2017-01-31 23:59:59.999999000
2016-12-31 23:59:59.999999000
2017-02-28 23:59:59.999999000
2022-09-30 23:59:59.999999000
-- startofweek()
2017-01-01 00:00:00.000000000
2016-12-25 00:00:00.000000000
2017-01-08 00:00:00.000000000
-- endofweek()
2017-01-07 23:59:59.999999000
2016-12-31 23:59:59.999999000
2017-01-14 23:59:59.999999000
-- startofyear()
2017-01-01 00:00:00.000000000
2016-01-01 00:00:00.000000000
2018-01-01 00:00:00.000000000
-- endofyear()
2017-12-31 23:59:59.999999000
2016-12-31 23:59:59.999999000
2018-12-31 23:59:59.999999000
-- unixtime_seconds_todatetime()
2019-01-01 00:00:00.000000000
1970-01-02 00:00:00.000000000
1969-12-31 00:00:00.000000000
-- unixtime_microseconds_todatetime
2019-01-01 00:00:00.000000
-- unixtime_milliseconds_todatetime()
2019-01-01 00:00:00.000
-- unixtime_nanoseconds_todatetime()
2019-01-01 00:00:00.000000000
-- weekofyear()
52
-- monthofyear()
12
-- weekofyear()
52
-- now()
1
-- make_datetime()
1
2017-10-01 12:10:00.0000000
2017-10-01 12:11:00.0000000
-- format_datetime
15-12-14 02:03:04.1234500
17-01-29 [09:00:05] 2017-01-29 [09:00:05] 17-01-29 [09:00:05 AM]
-- format_timespan()
02:03:04.1234500
29.09:00:05:12
-- ago()
-- datetime_diff()
17 2 13 4 29 2 5 10
-- datetime_part()
2017 4 10 44 30 303 01 02 03
-- datetime_add()
2018-01-01 00:00:00.0000000 2017-04-01 00:00:00.0000000 2017-02-01 00:00:00.0000000 2017-01-08 00:00:00.0000000 2017-01-02 00:00:00.0000000 2017-01-01 01:00:00.0000000 2017-01-01 00:01:00.0000000 2017-01-01 00:00:01.0000000

View File

@ -0,0 +1,86 @@
set dialect = 'kusto';
print '-- dayofmonth()';
print dayofmonth(datetime(2015-12-31));
print '-- dayofweek()';
print dayofweek(datetime(2015-12-31));
print '-- dayofyear()';
print dayofyear(datetime(2015-12-31));
print '-- getmonth()';
print getmonth(datetime(2015-10-12));
print '-- getyear()';
print getyear(datetime(2015-10-12));
print '-- hoursofday()';
print hourofday(datetime(2015-12-31 23:59:59.9));
print '-- startofday()';
print startofday(datetime(2017-01-01 10:10:17));
print startofday(datetime(2017-01-01 10:10:17), -1);
print startofday(datetime(2017-01-01 10:10:17), 1);
print '-- endofday()';
print endofday(datetime(2017-01-01 10:10:17));
print endofday(datetime(2017-01-01 10:10:17), -1);
print endofday(datetime(2017-01-01 10:10:17), 1);
print '-- endofmonth()';
print endofmonth(datetime(2017-01-01 10:10:17));
print endofmonth(datetime(2017-01-01 10:10:17), -1);
print endofmonth(datetime(2017-01-01 10:10:17), 1);
print endofmonth(datetime(2022-09-23));
print '-- startofweek()';
print startofweek(datetime(2017-01-01 10:10:17));
print startofweek(datetime(2017-01-01 10:10:17), -1);
print startofweek(datetime(2017-01-01 10:10:17), 1);
print '-- endofweek()';
print endofweek(datetime(2017-01-01 10:10:17));
print endofweek(datetime(2017-01-01 10:10:17), -1);
print endofweek(datetime(2017-01-01 10:10:17), 1);
print '-- startofyear()';
print startofyear(datetime(2017-01-01 10:10:17));
print startofyear(datetime(2017-01-01 10:10:17), -1);
print startofyear(datetime(2017-01-01 10:10:17), 1);
print '-- endofyear()';
print endofyear(datetime(2017-01-01 10:10:17));
print endofyear(datetime(2017-01-01 10:10:17), -1);
print endofyear(datetime(2017-01-01 10:10:17), 1);
print '-- unixtime_seconds_todatetime()';
print unixtime_seconds_todatetime(1546300800);
print unixtime_seconds_todatetime(1d);
print unixtime_seconds_todatetime(-1d);
print '-- unixtime_microseconds_todatetime';
print unixtime_microseconds_todatetime(1546300800000000);
print '-- unixtime_milliseconds_todatetime()';
print unixtime_milliseconds_todatetime(1546300800000);
print '-- unixtime_nanoseconds_todatetime()';
print unixtime_nanoseconds_todatetime(1546300800000000000);
print '-- weekofyear()';
print week_of_year(datetime(2000-01-01));
print '-- monthofyear()';
print monthofyear(datetime(2015-12-31));
print '-- weekofyear()';
print week_of_year(datetime(2000-01-01));
print '-- now()';
print getyear(now(-2d))>1900;
print '-- make_datetime()';
print make_datetime(2017,10,01,12,10) == datetime(2017-10-01 12:10:00);
print year_month_day_hour_minute = make_datetime(2017,10,01,12,10);
print year_month_day_hour_minute_second = make_datetime(2017,10,01,12,11,0.1234567);
print '-- format_datetime';
print format_datetime(datetime(2015-12-14 02:03:04.12345), 'y-M-d h:m:s.fffffff');
print v1=format_datetime(datetime(2017-01-29 09:00:05),'yy-MM-dd [HH:mm:ss]'), v2=format_datetime(datetime(2017-01-29 09:00:05), 'yyyy-M-dd [H:mm:ss]'), v3=format_datetime(datetime(2017-01-29 09:00:05), 'yy-MM-dd [hh:mm:ss tt]');
print '-- format_timespan()';
print format_timespan(time('14.02:03:04.12345'), 'h:m:s.fffffff');
print v1=format_timespan(time('29.09:00:05.12345'), 'dd.hh:mm:ss:FF');
-- print v2=format_timespan(time('29.09:00:05.12345'), 'ddd.h:mm:ss [fffffff]'); == '029.9:00:05 [1234500]'
print '-- ago()';
-- print ago(1d) - now();
print '-- datetime_diff()';
print year = datetime_diff('year',datetime(2017-01-01),datetime(2000-12-31)), quarter = datetime_diff('quarter',datetime(2017-07-01),datetime(2017-03-30)), month = datetime_diff('month',datetime(2017-01-01),datetime(2015-12-30)), week = datetime_diff('week',datetime(2017-10-29 00:00),datetime(2017-09-30 23:59)), day = datetime_diff('day',datetime(2017-10-29 00:00),datetime(2017-09-30 23:59)), hour = datetime_diff('hour',datetime(2017-10-31 01:00),datetime(2017-10-30 23:59)), minute = datetime_diff('minute',datetime(2017-10-30 23:05:01),datetime(2017-10-30 23:00:59)), second = datetime_diff('second',datetime(2017-10-30 23:00:10.100),datetime(2017-10-30 23:00:00.900));
-- millisecond = datetime_diff('millisecond',datetime(2017-10-30 23:00:00.200100),datetime(2017-10-30 23:00:00.100900)),
-- microsecond = datetime_diff('microsecond',datetime(2017-10-30 23:00:00.1009001),datetime(2017-10-30 23:00:00.1008009)),
-- nanosecond = datetime_diff('nanosecond',datetime(2017-10-30 23:00:00.0000000),datetime(2017-10-30 23:00:00.0000007))
print '-- datetime_part()';
print year = datetime_part("year", datetime(2017-10-30 01:02:03.7654321)),quarter = datetime_part("quarter", datetime(2017-10-30 01:02:03.7654321)),month = datetime_part("month", datetime(2017-10-30 01:02:03.7654321)),weekOfYear = datetime_part("week_of_year", datetime(2017-10-30 01:02:03.7654321)),day = datetime_part("day", datetime(2017-10-30 01:02:03.7654321)),dayOfYear = datetime_part("dayOfYear", datetime(2017-10-30 01:02:03.7654321)),hour = datetime_part("hour", datetime(2017-10-30 01:02:03.7654321)),minute = datetime_part("minute", datetime(2017-10-30 01:02:03.7654321)),second = datetime_part("second", datetime(2017-10-30 01:02:03.7654321));
-- millisecond = datetime_part("millisecond", dt),
-- microsecond = datetime_part("microsecond", dt),
-- nanosecond = datetime_part("nanosecond", dt)
print '-- datetime_add()';
print year = datetime_add('year',1,make_datetime(2017,1,1)),quarter = datetime_add('quarter',1,make_datetime(2017,1,1)),month = datetime_add('month',1,make_datetime(2017,1,1)),week = datetime_add('week',1,make_datetime(2017,1,1)),day = datetime_add('day',1,make_datetime(2017,1,1)),hour = datetime_add('hour',1,make_datetime(2017,1,1)),minute = datetime_add('minute',1,make_datetime(2017,1,1)),second = datetime_add('second',1,make_datetime(2017,1,1));

View File

@ -0,0 +1,152 @@
-- constant index value
1 c ['A',NULL,'C']
-- array_length()
1
1
-- array_sum()
1
1
-- array_index_of()
3
1
-- array_iif()
[1,5,3]
[1,5,3]
[1,5,NULL]
[NULL,NULL,NULL]
-- array_concat()
[1,2,3,4,5,6]
-- array_reverse()
[]
[1]
[4,3,2,1]
['example','an','is','this']
-- array_rotate_left()
[]
[]
[]
[3,4,5,1,2]
[1,2,3,4,5]
[3,4,5,1,2]
[4,5,1,2,3]
[1,2,3,4,5]
[4,5,1,2,3]
-- array_rotate_right()
[]
[]
[]
[4,5,1,2,3]
[1,2,3,4,5]
[4,5,1,2,3]
[3,4,5,1,2]
[1,2,3,4,5]
[3,4,5,1,2]
-- array_shift_left()
[]
[]
[]
[3,4,5,NULL,NULL]
[NULL,NULL,1,2,3]
[3,4,5,-1,-1]
['c','','']
-- array_shift_right()
[]
[]
[]
[3,4,5,NULL,NULL]
[NULL,NULL,1,2,3]
[3,4,5,-1,-1]
['c','','']
-- array_slice()
[3,4]
-- array_split()
[[1],[2,3],[4,5]]
[[1,2],[3,4,5]]
[[1],[2,3],[4,5]]
[[1,2,3,4],[],[4,5]]
-- array_sort_asc()
(['a','c','c','d',NULL])
([1,2,3,4])
['a','b','c']
(['p','q','r'],['hello','clickhouse','world'])
([NULL,'a','c','c','d'])
([NULL,'a','c','c','d'])
([NULL,NULL,NULL])
[1,2,3,NULL,NULL]
['a','e','b','c','d']
(['George','John','Paul','Ringo'])
(['blue','green','yellow',NULL,NULL])
([NULL,NULL,'blue','green','yellow'])
-- array_sort_desc()
(['d','c','c','a',NULL])
([4,3,2,1])
['c','b','a']
(['r','q','p'],['world','clickhouse','hello'])
([NULL,'d','c','c','a'])
([NULL,'d','c','c','a'])
([NULL,NULL,NULL])
[3,2,1,NULL,NULL]
['d','c','b','e','a']
(['Ringo','Paul','John','George'])
(['yellow','green','blue',NULL,NULL])
([NULL,NULL,'yellow','green','blue'])
-- jaccard_index()
0.75
0
0
nan
0
0.75
0.25
-- pack_array()
1 2 4 [1,2,4]
['ab','0.0.0.42','4.2']
-- repeat()
[]
[1,1,1]
['asd','asd','asd']
[86400,86400,86400]
[true,true,true]
[NULL]
[NULL]
-- set_difference()
[]
[]
[]
[]
[4,5,6]
[4]
[1,3]
[1,2,3]
['d','s']
['Chewbacca','Han Solo']
-- set_has_element()
0
1
0
1
0
-- set_intersect()
[]
[1,2,3]
[1,2,3]
[]
[5]
[]
['a']
['Darth Vader']
-- set_union()
[]
[1,2,3]
[1,2,3,4,5,6]
[1,2,3,4]
[1,2,3,4,5]
[1,2,3]
['a','d','f','s']
['Chewbacca','Darth Sidious','Darth Vader','Han Solo']
-- zip()
[]
[[1,2],[3,4],[5,6]]
[['Darth','Vader','has a suit'],['Master','Yoda','doesn\'t have a suit']]
[[1,10],[2,20],[3,NULL]]
[[NULL,1],[NULL,2],[NULL,3]]

View File

@ -0,0 +1,161 @@
DROP TABLE IF EXISTS array_test;
CREATE TABLE array_test (floats Array(Float64),
strings Array(String),
nullable_strings Array(Nullable(String))
) ENGINE=Memory;
INSERT INTO array_test VALUES([1.0, 2.5], ['a', 'c'], ['A', NULL, 'C']);
set dialect = 'kusto';
print '-- constant index value';
array_test | project floats[0], strings[1], nullable_strings;
print '-- array_length()';
print array_length(dynamic(['John', 'Denver', 'Bob', 'Marley'])) == 4;
print array_length(dynamic([1, 2, 3])) == 3;
print '-- array_sum()';
print array_sum(dynamic([2, 5, 3])) == 10;
print array_sum(dynamic([2.5, 5.5, 3])) == 11;
print '-- array_index_of()';
print array_index_of(dynamic(['John', 'Denver', 'Bob', 'Marley']), 'Marley');
print array_index_of(dynamic([1, 2, 3]), 2);
print '-- array_iif()';
print array_iif(dynamic([true,false,true]), dynamic([1,2,3]), dynamic([4,5,6]));
print array_iif(dynamic([1,0,1]), dynamic([1,2,3]), dynamic([4,5,6]));
print array_iif(dynamic([true,false,true]), dynamic([1,2]), dynamic([4,5,6]));
print array_iif(dynamic(['a','b','c']), dynamic([1,2,3]), dynamic([4,5,6]));
print '-- array_concat()';
print array_concat(dynamic([1,2,3]),dynamic([4,5,6]));
print '-- array_reverse()';
print array_reverse(dynamic([]));
print array_reverse(dynamic([1]));
print array_reverse(dynamic([1,2,3,4]));
print array_reverse(dynamic(["this", "is", "an", "example"]));
print '-- array_rotate_left()';
print array_rotate_left(dynamic([]), 0);
print array_rotate_left(dynamic([]), 500);
print array_rotate_left(dynamic([]), -500);
print array_rotate_left(dynamic([1,2,3,4,5]), 2);
print array_rotate_left(dynamic([1,2,3,4,5]), 5);
print array_rotate_left(dynamic([1,2,3,4,5]), 7);
print array_rotate_left(dynamic([1,2,3,4,5]), -2);
print array_rotate_left(dynamic([1,2,3,4,5]), -5);
print array_rotate_left(dynamic([1,2,3,4,5]), -7);
print '-- array_rotate_right()';
print array_rotate_right(dynamic([]), 0);
print array_rotate_right(dynamic([]), 500);
print array_rotate_right(dynamic([]), -500);
print array_rotate_right(dynamic([1,2,3,4,5]), 2);
print array_rotate_right(dynamic([1,2,3,4,5]), 5);
print array_rotate_right(dynamic([1,2,3,4,5]), 7);
print array_rotate_right(dynamic([1,2,3,4,5]), -2);
print array_rotate_right(dynamic([1,2,3,4,5]), -5);
print array_rotate_right(dynamic([1,2,3,4,5]), -7);
print '-- array_shift_left()';
print array_shift_left(dynamic([]), 0);
print array_shift_left(dynamic([]), 555);
print array_shift_left(dynamic([]), -555);
print array_shift_left(dynamic([1,2,3,4,5]), 2);
print array_shift_left(dynamic([1,2,3,4,5]), -2);
print array_shift_left(dynamic([1,2,3,4,5]), 2, -1);
print array_shift_left(dynamic(['a', 'b', 'c']), 2);
print '-- array_shift_right()';
print array_shift_left(dynamic([]), 0);
print array_shift_left(dynamic([]), 555);
print array_shift_left(dynamic([]), -555);
print array_shift_right(dynamic([1,2,3,4,5]), -2);
print array_shift_right(dynamic([1,2,3,4,5]), 2);
print array_shift_right(dynamic([1,2,3,4,5]), -2, -1);
print array_shift_right(dynamic(['a', 'b', 'c']), -2);
print '-- array_slice()';
--print array_slice(dynamic([1,2,3]), 1, 2); -- will enable whe analyzer dixed
print array_slice(dynamic([1,2,3,4,5]), -3, -2);
print '-- array_split()';
print array_split(dynamic([1,2,3,4,5]), dynamic([1,-2]));
print array_split(dynamic([1,2,3,4,5]), 2);
print array_split(dynamic([1,2,3,4,5]), dynamic([1,3]));
print array_split(dynamic([1,2,3,4,5]), dynamic([-1,-2]));
print '-- array_sort_asc()';
print array_sort_asc(dynamic([null, 'd', 'a', 'c', 'c']));
print array_sort_asc(dynamic([4, 1, 3, 2]));
print array_sort_asc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))[0];
print array_sort_asc(dynamic(['q', 'p', 'r']), dynamic(['clickhouse','hello', 'world']));
print array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , false);
print array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2);
print array_sort_asc( dynamic([null, null, null]) , false);
print array_sort_asc(dynamic([2, 1, null,3, null]), dynamic([20, 10, 40, 30, 50]), 1 < 2)[0];
print array_sort_asc(dynamic(['1','3','4','5','2']),dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]))[3];
print array_sort_asc(split("John,Paul,George,Ringo", ","));
print array_sort_asc(dynamic([null,"blue","yellow","green",null]));
print array_sort_asc(dynamic([null,"blue","yellow","green",null]), false);
print '-- array_sort_desc()';
print array_sort_desc(dynamic([null, 'd', 'a', 'c', 'c']));
print array_sort_desc(dynamic([4, 1, 3, 2]));
print array_sort_desc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))[0];
print array_sort_desc(dynamic(['q', 'p', 'r']), dynamic(['clickhouse','hello', 'world']));
print array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , false);
print array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2);
print array_sort_desc( dynamic([null, null, null]) , false);
print array_sort_desc(dynamic([2, 1, null,3, null]), dynamic([20, 10, 40, 30, 50]), 1 < 2)[0];
print array_sort_desc(dynamic(['1','3','4','5','2']),dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]))[3];
print array_sort_desc(split("John,Paul,George,Ringo", ","));
print array_sort_desc(dynamic([null,"blue","yellow","green",null]));
print array_sort_desc(dynamic([null,"blue","yellow","green",null]), false);
print '-- jaccard_index()';
print jaccard_index(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3, 4, 4, 4]));
print jaccard_index(dynamic([1, 2, 3]), dynamic([]));
print jaccard_index(dynamic([]), dynamic([1, 2, 3, 4]));
print jaccard_index(dynamic([]), dynamic([]));
print jaccard_index(dynamic([1, 2, 3]), dynamic([4, 5, 6, 7]));
print jaccard_index(dynamic(['a', 's', 'd']), dynamic(['f', 'd', 's', 'a']));
print jaccard_index(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader']));
print '-- pack_array()';
print pack_array(); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
print x = 1 | extend y = x * 2 | extend z = y * 2 | extend pack_array(x,y,z);
print pack_array(strcat('a', 'b'), format_ipv4(42), tostring(4.2));
print '-- repeat()';
print repeat(1, 0);
print repeat(1, 3);
print repeat("asd", 3);
print repeat(timespan(1d), 3);
print repeat(true, 3);
print repeat(1, -3);
print repeat(6.7,-4);
print '-- set_difference()';
print set_difference(dynamic([]), dynamic([]));
print set_difference(dynamic([]), dynamic([9]));
print set_difference(dynamic([]), dynamic(["asd"]));
print set_difference(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3]));
print array_sort_asc(set_difference(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[0];
print set_difference(dynamic([4]), dynamic([1, 2, 3]));
print array_sort_asc(set_difference(dynamic([1, 2, 3, 4, 5]), dynamic([5]), dynamic([2, 4])))[0];
print array_sort_asc(set_difference(dynamic([1, 2, 3]), dynamic([])))[0];
print array_sort_asc(set_difference(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])))[0];
print array_sort_asc(set_difference(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])))[0];
print '-- set_has_element()';
print set_has_element(dynamic([]), 9);
print set_has_element(dynamic(["this", "is", "an", "example"]), "example");
print set_has_element(dynamic(["this", "is", "an", "example"]), "examplee");
print set_has_element(dynamic([1, 2, 3]), 2);
print set_has_element(dynamic([1, 2, 3, 4.2]), 4);
print '-- set_intersect()';
print set_intersect(dynamic([]), dynamic([]));
print array_sort_asc(set_intersect(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])))[0];
print array_sort_asc(set_intersect(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[0];
print set_intersect(dynamic([4]), dynamic([1, 2, 3]));
print set_intersect(dynamic([1, 2, 3, 4, 5]), dynamic([1, 3, 5]), dynamic([2, 5]));
print set_intersect(dynamic([1, 2, 3]), dynamic([]));
print set_intersect(dynamic(['a', 's', 'd']), dynamic(['a', 'f']));
print set_intersect(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader']));
print '-- set_union()';
print set_union(dynamic([]), dynamic([]));
print array_sort_asc(set_union(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])))[0];
print array_sort_asc(set_union(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[0];
print array_sort_asc(set_union(dynamic([4]), dynamic([1, 2, 3])))[0];
print array_sort_asc(set_union(dynamic([1, 3, 4]), dynamic([5]), dynamic([2, 4])))[0];
print array_sort_asc(set_union(dynamic([1, 2, 3]), dynamic([])))[0];
print array_sort_asc(set_union(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])))[0];
print array_sort_asc(set_union(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])))[0];
print '-- zip()';
print zip(dynamic([]), dynamic([]));
print zip(dynamic([1,3,5]), dynamic([2,4,6]));
print zip(dynamic(['Darth','Master']), dynamic(['Vader','Yoda']), dynamic(['has a suit','doesn\'t have a suit']));
print zip(dynamic([1,2,3]), dynamic([10,20]));
print zip(dynamic([]), dynamic([1,2,3]));

View File

@ -0,0 +1,123 @@
-- ipv4_is_private(\'127.0.0.1\')
0
-- ipv4_is_private(\'10.1.2.3\')
1
-- ipv4_is_private(\'192.168.1.1/24\')
1
ipv4_is_private(strcat(\'192.\',\'168.\',\'1.\',\'1\',\'/24\'))
1
-- ipv4_is_private(\'abc\')
\N
-- ipv4_netmask_suffix(\'192.168.1.1/24\')
24
-- ipv4_netmask_suffix(\'192.168.1.1\')
32
-- ipv4_netmask_suffix(\'127.0.0.1/16\')
16
-- ipv4_netmask_suffix(\'abc\')
\N
ipv4_netmask_suffix(strcat(\'127.\', \'0.\', \'0.1/16\'))
16
-- ipv4_is_in_range(\'127.0.0.1\', \'127.0.0.1\')
1
-- ipv4_is_in_range(\'192.168.1.6\', \'192.168.1.1/24\')
1
-- ipv4_is_in_range(\'192.168.1.1\', \'192.168.2.1/24\')
0
-- ipv4_is_in_range(strcat(\'192.\',\'168.\', \'1.1\'), \'192.168.2.1/24\')
0
-- ipv4_is_in_range(\'abc\', \'127.0.0.1\')
\N
-- parse_ipv6(127.0.0.1)
0000:0000:0000:0000:0000:ffff:7f00:0001
-- parse_ipv6(fe80::85d:e82c:9446:7994)
fe80:0000:0000:0000:085d:e82c:9446:7994
-- parse_ipv4(\'127.0.0.1\')
2130706433
-- parse_ipv4(\'192.1.168.1\') < parse_ipv4(\'192.1.168.2\')
1
-- parse_ipv4(arrayStringConcat([\'127\', \'0\', \'0\', \'1\'], \'.\'))
-- parse_ipv4_mask(\'127.0.0.1\', 24) == 2130706432
2130706432
-- parse_ipv4_mask(\'abc\', 31)
\N
\N
-- parse_ipv4_mask(\'192.1.168.2\', 31) == parse_ipv4_mask(\'192.1.168.3\', 31)
3221334018
3221334018
-- ipv4_is_match(\'127.0.0.1\', \'127.0.0.1\')
1
-- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\')
0
-- ipv4_is_match(\'192.168.1.1/24\', \'192.168.1.255/24\')
1
-- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\', 24)
1
-- ipv4_is_match(\'abc\', \'def\', 24)
\N
-- ipv4_compare()
0
-1
1
0
0
0
0
0
0
0
0
-- format_ipv4()
192.168.1.0
192.168.1.1
192.168.1.0
192.168.1.0
1
1
127.0.0.0
-- format_ipv4_mask()
192.168.1.0/24
192.168.1.0/24
192.168.1.0/24
192.168.1.1/32
192.168.1.0/24
1
1
127.0.0.0/24
-- parse_ipv6_mask()
0000:0000:0000:0000:0000:0000:0000:0000
fe80:0000:0000:0000:085d:e82c:9446:7900
0000:0000:0000:0000:0000:ffff:c0a8:ff00
0000:0000:0000:0000:0000:ffff:c0a8:ff00
0000:0000:0000:0000:0000:ffff:ffff:ffff
fe80:0000:0000:0000:085d:e82c:9446:7994
fe80:0000:0000:0000:085d:e82c:9446:7900
0000:0000:0000:0000:0000:ffff:c0a8:ffff
0000:0000:0000:0000:0000:ffff:c0a8:ff00
-- ipv6_is_match()
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1

View File

@ -0,0 +1,131 @@
set dialect='kusto';
print '-- ipv4_is_private(\'127.0.0.1\')';
print ipv4_is_private('127.0.0.1');
print '-- ipv4_is_private(\'10.1.2.3\')';
print ipv4_is_private('10.1.2.3');
print '-- ipv4_is_private(\'192.168.1.1/24\')';
print ipv4_is_private('192.168.1.1/24');
print 'ipv4_is_private(strcat(\'192.\',\'168.\',\'1.\',\'1\',\'/24\'))';
print ipv4_is_private(strcat('192.','168.','1.','1','/24'));
print '-- ipv4_is_private(\'abc\')';
print ipv4_is_private('abc'); -- == null
print '-- ipv4_netmask_suffix(\'192.168.1.1/24\')';
print ipv4_netmask_suffix('192.168.1.1/24'); -- == 24
print '-- ipv4_netmask_suffix(\'192.168.1.1\')';
print ipv4_netmask_suffix('192.168.1.1'); -- == 32
print '-- ipv4_netmask_suffix(\'127.0.0.1/16\')';
print ipv4_netmask_suffix('127.0.0.1/16'); -- == 16
print '-- ipv4_netmask_suffix(\'abc\')';
print ipv4_netmask_suffix('abc'); -- == null
print 'ipv4_netmask_suffix(strcat(\'127.\', \'0.\', \'0.1/16\'))';
print ipv4_netmask_suffix(strcat('127.', '0.', '0.1/16')); -- == 16
print '-- ipv4_is_in_range(\'127.0.0.1\', \'127.0.0.1\')';
print ipv4_is_in_range('127.0.0.1', '127.0.0.1'); -- == true
print '-- ipv4_is_in_range(\'192.168.1.6\', \'192.168.1.1/24\')';
print ipv4_is_in_range('192.168.1.6', '192.168.1.1/24'); -- == true
print '-- ipv4_is_in_range(\'192.168.1.1\', \'192.168.2.1/24\')';
print ipv4_is_in_range('192.168.1.1', '192.168.2.1/24'); -- == false
print '-- ipv4_is_in_range(strcat(\'192.\',\'168.\', \'1.1\'), \'192.168.2.1/24\')';
print ipv4_is_in_range(strcat('192.','168.', '1.1'), '192.168.2.1/24'); -- == false
print '-- ipv4_is_in_range(\'abc\', \'127.0.0.1\')'; -- == null
print ipv4_is_in_range('abc', '127.0.0.1');
print '-- parse_ipv6(127.0.0.1)';
print parse_ipv6('127.0.0.1');
print '-- parse_ipv6(fe80::85d:e82c:9446:7994)';
print parse_ipv6('fe80::85d:e82c:9446:7994');
print '-- parse_ipv4(\'127.0.0.1\')';
print parse_ipv4('127.0.0.1');
print '-- parse_ipv4(\'192.1.168.1\') < parse_ipv4(\'192.1.168.2\')';
print parse_ipv4('192.1.168.1') < parse_ipv4('192.1.168.2');
print '-- parse_ipv4(arrayStringConcat([\'127\', \'0\', \'0\', \'1\'], \'.\'))';
print parse_ipv4(arrayStringConcat(['127', '0', '0', '1'], '.')); -- { clientError UNKNOWN_FUNCTION }
print '-- parse_ipv4_mask(\'127.0.0.1\', 24) == 2130706432';
print parse_ipv4_mask('127.0.0.1', 24);
print '-- parse_ipv4_mask(\'abc\', 31)';
print parse_ipv4_mask('abc', 31)
print '-- parse_ipv4_mask(\'192.1.168.2\', 1000)';
print parse_ipv4_mask('192.1.168.2', 1000);
print '-- parse_ipv4_mask(\'192.1.168.2\', 31) == parse_ipv4_mask(\'192.1.168.3\', 31)';
--print parse_ipv4_mask('192.1.168.2', 31) == parse_ipv4_mask('192.1.168.3', 31); // this qual failed in analyzer 3221334018
print parse_ipv4_mask('192.1.168.2', 31);
print parse_ipv4_mask('192.1.168.3', 31);
print '-- ipv4_is_match(\'127.0.0.1\', \'127.0.0.1\')';
print ipv4_is_match('127.0.0.1', '127.0.0.1');
print '-- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\')';
print ipv4_is_match('192.168.1.1', '192.168.1.255');
print '-- ipv4_is_match(\'192.168.1.1/24\', \'192.168.1.255/24\')';
print ipv4_is_match('192.168.1.1/24', '192.168.1.255/24');
print '-- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\', 24)';
print ipv4_is_match('192.168.1.1', '192.168.1.255', 24);
print '-- ipv4_is_match(\'abc\', \'def\', 24)';
print ipv4_is_match('abc', 'dev', 24);
print '-- ipv4_compare()';
print ipv4_compare('127.0.0.1', '127.0.0.1');
print ipv4_compare('192.168.1.1', '192.168.1.255');
print ipv4_compare('192.168.1.255', '192.168.1.1');
print ipv4_compare('192.168.1.1/24', '192.168.1.255/24');
print ipv4_compare('192.168.1.1', '192.168.1.255', 24);
print ipv4_compare('192.168.1.1/24', '192.168.1.255');
print ipv4_compare('192.168.1.1', '192.168.1.255/24');
print ipv4_compare('192.168.1.1/30', '192.168.1.255/24');
print ipv4_compare('192.168.1.1', '192.168.1.0', 31);
print ipv4_compare('192.168.1.1/24', '192.168.1.255', 31);
print ipv4_compare('192.168.1.1', '192.168.1.255', 24);
print '-- format_ipv4()';
print format_ipv4('192.168.1.255', 24);
print format_ipv4('192.168.1.1', 32);
print format_ipv4('192.168.1.1/24', 32);
print format_ipv4(3232236031, 24);
print format_ipv4('192.168.1.1/24', -1) == '';
print format_ipv4('abc', 24) == '';
print format_ipv4(strcat('127.0', '.0.', '1', '/32'), 12 + 12);
print '-- format_ipv4_mask()';
print format_ipv4_mask('192.168.1.255', 24);
print format_ipv4_mask(3232236031, 24);
print format_ipv4_mask('192.168.1.1', 24);
print format_ipv4_mask('192.168.1.1', 32);
print format_ipv4_mask('192.168.1.1/24', 32);
print format_ipv4_mask('192.168.1.1/24', -1) == '';
print format_ipv4_mask('abc', 24) == '';
print format_ipv4_mask(strcat('127.0', '.0.', '1', '/32'), 12 + 12);
print '-- parse_ipv6_mask()';
print parse_ipv6_mask("127.0.0.1", 24);
print parse_ipv6_mask("fe80::85d:e82c:9446:7994", 120);
print parse_ipv6_mask("192.168.255.255", 120);
print parse_ipv6_mask("192.168.255.255/24", 124);
print parse_ipv6_mask("255.255.255.255", 128);
print parse_ipv6_mask("fe80::85d:e82c:9446:7994", 128);
print parse_ipv6_mask("fe80::85d:e82c:9446:7994/120", 124);
print parse_ipv6_mask("::192.168.255.255", 128);
print parse_ipv6_mask("::192.168.255.255/24", 128);
print '-- ipv6_is_match()';
print ipv6_is_match('::ffff:7f00:1', '127.0.0.1') == true;
print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995') == false;
print ipv6_is_match('192.168.1.1/24', '192.168.1.255/24') == true;
print ipv6_is_match('fe80::85d:e82c:9446:7994/127', 'fe80::85d:e82c:9446:7995/127') == true;
print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995', 127) == true;
print ipv6_is_match('192.168.1.1', '192.168.1.1'); -- // Equal IPs
print ipv6_is_match('192.168.1.1/24', '192.168.1.255'); -- // 24 bit IP4-prefix is used for comparison
print ipv6_is_match('192.168.1.1', '192.168.1.255/24'); -- // 24 bit IP4-prefix is used for comparison
print ipv6_is_match('192.168.1.1/30', '192.168.1.255/24'); -- // 24 bit IP4-prefix is used for comparison
print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7994'); -- // Equal IPs
print ipv6_is_match('fe80::85d:e82c:9446:7994/120', 'fe80::85d:e82c:9446:7998'); -- // 120 bit IP6-prefix is used for comparison
print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7998/120'); -- // 120 bit IP6-prefix is used for comparison
print ipv6_is_match('fe80::85d:e82c:9446:7994/120', 'fe80::85d:e82c:9446:7998/120'); -- // 120 bit IP6-prefix is used for comparison
print ipv6_is_match('192.168.1.1', '::ffff:c0a8:0101'); -- // Equal IPs
print ipv6_is_match('192.168.1.1/24', '::ffff:c0a8:01ff'); -- // 24 bit IP-prefix is used for comparison
print ipv6_is_match('::ffff:c0a8:0101', '192.168.1.255/24'); -- // 24 bit IP-prefix is used for comparison
print ipv6_is_match('::192.168.1.1/30', '192.168.1.255/24'); -- // 24 bit IP-prefix is used for comparison
print ipv6_is_match('192.168.1.1', '192.168.1.0', 31); -- // 31 bit IP4-prefix is used for comparison
print ipv6_is_match('192.168.1.1/24', '192.168.1.255', 31); -- // 24 bit IP4-prefix is used for comparison
print ipv6_is_match('192.168.1.1', '192.168.1.255', 24); -- // 24 bit IP4-prefix is used for comparison
print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995', 127); -- // 127 bit IP6-prefix is used for comparison
print ipv6_is_match('fe80::85d:e82c:9446:7994/127', 'fe80::85d:e82c:9446:7998', 120); -- // 120 bit IP6-prefix is used for comparison
print ipv6_is_match('fe80::85d:e82c:9446:7994/120', 'fe80::85d:e82c:9446:7998', 127); -- // 120 bit IP6-prefix is used for comparison
print ipv6_is_match('192.168.1.1/24', '::ffff:c0a8:01ff', 127); -- // 127 bit IP6-prefix is used for comparison
print ipv6_is_match('::ffff:c0a8:0101', '192.168.1.255', 120); -- // 120 bit IP6-prefix is used for comparison
print ipv6_is_match('::192.168.1.1/30', '192.168.1.255/24', 127); -- // 120 bit IP6-prefix is used for comparison

View File

@ -0,0 +1,4 @@
-- isnan --
1
0
0

View File

@ -0,0 +1,7 @@
set dialect = 'kusto';
print '-- isnan --';
print isnan(double(nan));
print isnan(4.2);
print isnan(4); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }
print isnan(real(+inf));
print isnan(dynamic(null)); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }

View File

@ -0,0 +1,16 @@
-- bin_at()
4.5
-12:0:0
2017-05-14 12:00:00.000000000
2017-05-14 00:00:00.000000000
2018-02-25 15:14:00.000000000 5
2018-02-24 15:14:00.000000000 3
2018-02-23 15:14:00.000000000 4
-- bin()
4
1970-05-11 00:00:00.000000000
336:0:0
1970-05-11 13:45:07.345000000
1970-05-11 13:45:07.345623000
2022-09-26 10:13:23.987232000
1970-05-11 13:45:07.456336000

View File

@ -0,0 +1,26 @@
DROP TABLE IF EXISTS Bin_at_test;
CREATE TABLE Bin_at_test
(
`Date` DateTime('UTC'),
Num Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO Bin_at_test VALUES ('2018-02-24T15:14:01',3), ('2018-02-23T16:14:01',4), ('2018-02-26T15:14:01',5);
set dialect = 'kusto';
print '-- bin_at()';
print bin_at(6.5, 2.5, 7);
print bin_at(1h, 1d, 12h);
print bin_at(datetime(2017-05-15 10:20:00.0), 1d, datetime(1970-01-01 12:00:00.0));
print bin_at(datetime(2017-05-17 10:20:00.0), 7d, datetime(2017-06-04 00:00:00.0));
Bin_at_test | summarize sum(Num) by d = todatetime(bin_at(Date, 1d, datetime('2018-02-24 15:14:00'))) | order by d;
print '-- bin()';
print bin(4.5, 1);
print bin(datetime(1970-05-11 13:45:07), 1d);
print bin(16d, 7d);
print bin(datetime(1970-05-11 13:45:07.345623), 1ms);
-- print bin(datetime(2022-09-26 10:13:23.987234), 6ms); -> 2022-09-26 10:13:23.982000000
print bin(datetime(1970-05-11 13:45:07.345623), 1microsecond);
print bin(datetime(2022-09-26 10:13:23.987234), 6microseconds);
print bin(datetime(1970-05-11 13:45:07.456345672), 16microseconds);
-- print bin(datetime(2022-09-26 10:13:23.987234128), 1tick); -> 2022-09-26 10:13:23.987234100
-- print bin(datetime(2022-09-26 10:13:23.987234128), 99nanosecond); -> null

View File

@ -0,0 +1,360 @@
-- test String Functions --
-- Customers |where Education contains \'degree\'
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
-- Customers |where Education !contains \'degree\'
\N why Professional Partial College 38
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers |where Education contains \'Degree\'
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
-- Customers |where Education !contains \'Degree\'
\N why Professional Partial College 38
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers | where FirstName endswith \'RE\'
Theodore Diaz Skilled Manual Bachelors 28
-- Customers | where ! FirstName endswith \'RE\'
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
--Customers | where FirstName endswith_cs \'re\'
Theodore Diaz Skilled Manual Bachelors 28
-- Customers | where FirstName !endswith_cs \'re\'
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers | where Occupation == \'Skilled Manual\'
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Apple Skilled Manual Bachelors 28
-- Customers | where Occupation != \'Skilled Manual\'
\N why Professional Partial College 38
Latoya Shen Professional Graduate Degree 25
Stephanie Cox Management abcd defg Bachelors 33
-- Customers | where Occupation has \'skilled\'
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Apple Skilled Manual Bachelors 28
-- Customers | where Occupation !has \'skilled\'
\N why Professional Partial College 38
Latoya Shen Professional Graduate Degree 25
Stephanie Cox Management abcd defg Bachelors 33
-- Customers | where Occupation has \'Skilled\'
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Apple Skilled Manual Bachelors 28
-- Customers | where Occupation !has \'Skilled\'
\N why Professional Partial College 38
Latoya Shen Professional Graduate Degree 25
Stephanie Cox Management abcd defg Bachelors 33
-- Customers | where Occupation hasprefix_cs \'Ab\'
-- Customers | where Occupation !hasprefix_cs \'Ab\'
\N why Professional Partial College 38
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers | where Occupation hasprefix_cs \'ab\'
Stephanie Cox Management abcd defg Bachelors 33
-- Customers | where Occupation !hasprefix_cs \'ab\'
\N why Professional Partial College 38
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Apple Skilled Manual Bachelors 28
-- Customers | where Occupation hassuffix \'Ent\'
Stephanie Cox Management abcd defg Bachelors 33
-- Customers | where Occupation !hassuffix \'Ent\'
\N why Professional Partial College 38
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Apple Skilled Manual Bachelors 28
-- Customers | where Occupation hassuffix \'ent\'
Stephanie Cox Management abcd defg Bachelors 33
-- Customers | where Occupation hassuffix \'ent\'
Stephanie Cox Management abcd defg Bachelors 33
-- Customers |where Education in (\'Bachelors\',\'High School\')
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers | where Education !in (\'Bachelors\',\'High School\')
\N why Professional Partial College 38
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
-- Customers | where FirstName matches regex \'P.*r\'
Peter Nara Skilled Manual Graduate Degree 26
-- Customers | where FirstName startswith \'pet\'
Peter Nara Skilled Manual Graduate Degree 26
-- Customers | where FirstName !startswith \'pet\'
Latoya Shen Professional Graduate Degree 25
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers | where FirstName startswith_cs \'pet\'
-- Customers | where FirstName !startswith_cs \'pet\'
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers | where isempty(LastName)
Apple Skilled Manual Bachelors 28
-- Customers | where isnotempty(LastName)
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Peter Nara Skilled Manual Graduate Degree 26
Latoya Shen Professional Graduate Degree 25
\N why Professional Partial College 38
-- Customers | where isnotnull(FirstName)
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- Customers | where isnull(FirstName)
\N why Professional Partial College 38
-- Customers | project url_decode(\'https%3A%2F%2Fwww.test.com%2Fhello%20word\') | take 1
https://www.test.com/hello word
-- Customers | project url_encode(\'https://www.test.com/hello word\') | take 1
https%3A%2F%2Fwww.test.com%2Fhello%20word
-- Customers | project name_abbr = strcat(substring(FirstName,0,3), \' \', substring(LastName,2))
\N
Lat en
Pet ra
The az
Ste x
App
-- Customers | project name = strcat(FirstName, \' \', LastName)
\N
Latoya Shen
Peter Nara
Theodore Diaz
Stephanie Cox
Apple
-- Customers | project FirstName, strlen(FirstName)
\N \N
Latoya 6
Peter 5
Theodore 8
Stephanie 9
Apple 5
-- Customers | project strrep(FirstName,2,\'_\')
\N
Latoya_Latoya
Peter_Peter
Theodore_Theodore
Stephanie_Stephanie
Apple_Apple
-- Customers | project toupper(FirstName)
\N
LATOYA
PETER
THEODORE
STEPHANIE
APPLE
-- Customers | project tolower(FirstName)
\N
latoya
peter
theodore
stephanie
apple
-- support subquery for in orerator (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/in-cs-operator) (subquery need to be wraped with bracket inside bracket); TODO: case-insensitive not supported yet
Latoya Shen Professional Graduate Degree 25
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Apple Skilled Manual Bachelors 28
-- has_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-all-operator); TODO: subquery not supported yet
Peter Nara Skilled Manual Graduate Degree 26
Theodore Diaz Skilled Manual Bachelors 28
Apple Skilled Manual Bachelors 28
-- has_any (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-anyoperator); TODO: subquery not supported yet
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Peter Nara Skilled Manual Graduate Degree 26
Apple Skilled Manual Bachelors 28
-- countof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/countoffunction)
3
3
1
-- extract ( https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractfunction)
PINEAPPLE ice cream is 20
PINEAPPLE
20
20
\N
\N
\N
\N
\N
45.6
45.6
-- extract_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractallfunction); TODO: captureGroups not supported yet
[['T','h','e'],['p','ric','e'],['P','INEAPPL','E'],['i','c','e'],['c','rea','m']]
-- extract_json (https://learn.microsoft.com/en-us/azure/data-explorer/kusto/query/extractjsonfunction)
John
iPhone
\N
26
26
26
26
\N
-- split (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/splitfunction)
['aa','bb']
['bbb']
['']
['a','','b']
['aa','cc']
['aabbcc']
['aaa','bbb','ccc']
[NULL]
-- strcat_delim (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/strcat-delimfunction); TODO: only support string now.
1-2-Ab
-- indexof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/indexoffunction); TODO: length and occurrence not supported yet
2
2
-1
-- base64_encode_fromguid()
8jMxriJurkmwahbmqbIS6w==
-- base64_decode_toarray()
[]
[75,117,115,116,111]
-- base64_decode_toguid()
10e99626-bc2b-4c75-bb3e-fe606de25700
1
-- base64_encode_tostring
S3VzdG8x
-- base64_decode_tostring
Kusto1
-- parse_url()
{"Scheme":"scheme","Host":"","Port":"0","Path":"/this/is/a/path","Username":"username","Password":"password","Query Parameters":{"k1":"v1","k2":"v2"},"Fragment":"fragment"}
-- parse_urlquery()
{"Query Parameters":{"k1":"v1","k2":"v2","k3":"v3"}}
-- strcmp()
0 1 -1 1
-- substring()
CD
-- translate()
kusto xxx
-- trim()
https://www.ibm.com
Te st1
asd
asd
sd
-- trim_start()
www.ibm.com
Te st1// $
asdw
asd
-- trim_end()
https
- Te st1
wasd
asd
-- trim, trim_start, trim_end all at once
--https://bing.com-- -- https://bing.com-- --https://bing.com https://bing.com
-- replace_regex
Number was: 1
-- has_any_index()
0 1 -1 -1
-- parse_version()
1000000020000000300000040
1000000020000000000000000
1000000020000000000000000
\N
\N
\N
\N
1000000020000000300000004
1000000020000000000000000
1000000020000000300000000
1000000000000000000000000
-- parse_json()
[1,2,3]
[{"a":123.5,"b":"{\\"c\\":456}"}]
-- parse_command_line()
[NULL]
[NULL]
-- reverse()
321
43.321
dsa
][
]3,2,1[
]\'redaV\',\'htraD\'[
000000000.00:00:21 51-01-7102
Peter Nara Skilled Manual Graduate Degree 26
Latoya Shen Professional Graduate Degree 25
-- parse_csv()
['']
['aaa']
['aa','b','cc']
['record1','a','b','c']

View File

@ -0,0 +1,313 @@
-- Tags: no-fasttest
DROP TABLE IF EXISTS Customers;
CREATE TABLE Customers
(
FirstName Nullable(String),
LastName String,
Occupation String,
Education String,
Age Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28), ('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38);
-- datatable (Version:string) [
-- '1.2.3.4',
-- '1.2',
-- '1.2.3',
-- '1'
-- ]
DROP TABLE IF EXISTS Versions;
CREATE TABLE Versions
(
Version String
) ENGINE = Memory;
INSERT INTO Versions VALUES ('1.2.3.4'),('1.2'),('1.2.3'),('1');
set dialect='kusto';
print '-- test String Functions --';
print '-- Customers |where Education contains \'degree\'';
Customers |where Education contains 'degree' | order by LastName;
print '';
print '-- Customers |where Education !contains \'degree\'';
Customers |where Education !contains 'degree' | order by LastName;
print '';
print '-- Customers |where Education contains \'Degree\'';
Customers |where Education contains 'Degree' | order by LastName;
print '';
print '-- Customers |where Education !contains \'Degree\'';
Customers |where Education !contains 'Degree' | order by LastName;
print '';
print '-- Customers | where FirstName endswith \'RE\'';
Customers | where FirstName endswith 'RE' | order by LastName;
print '';
print '-- Customers | where ! FirstName endswith \'RE\'';
Customers | where FirstName ! endswith 'RE' | order by LastName;
print '';
print '--Customers | where FirstName endswith_cs \'re\'';
Customers | where FirstName endswith_cs 're' | order by LastName;
print '';
print '-- Customers | where FirstName !endswith_cs \'re\'';
Customers | where FirstName !endswith_cs 're' | order by LastName;
print '';
print '-- Customers | where Occupation == \'Skilled Manual\'';
Customers | where Occupation == 'Skilled Manual' | order by LastName;
print '';
print '-- Customers | where Occupation != \'Skilled Manual\'';
Customers | where Occupation != 'Skilled Manual' | order by LastName;
print '';
print '-- Customers | where Occupation has \'skilled\'';
Customers | where Occupation has 'skilled' | order by LastName;
print '';
print '-- Customers | where Occupation !has \'skilled\'';
Customers | where Occupation !has 'skilled' | order by LastName;
print '';
print '-- Customers | where Occupation has \'Skilled\'';
Customers | where Occupation has 'Skilled'| order by LastName;
print '';
print '-- Customers | where Occupation !has \'Skilled\'';
Customers | where Occupation !has 'Skilled'| order by LastName;
print '';
print '-- Customers | where Occupation hasprefix_cs \'Ab\'';
Customers | where Occupation hasprefix_cs 'Ab'| order by LastName;
print '';
print '-- Customers | where Occupation !hasprefix_cs \'Ab\'';
Customers | where Occupation !hasprefix_cs 'Ab'| order by LastName;
print '';
print '-- Customers | where Occupation hasprefix_cs \'ab\'';
Customers | where Occupation hasprefix_cs 'ab'| order by LastName;
print '';
print '-- Customers | where Occupation !hasprefix_cs \'ab\'';
Customers | where Occupation !hasprefix_cs 'ab'| order by LastName;
print '';
print '-- Customers | where Occupation hassuffix \'Ent\'';
Customers | where Occupation hassuffix 'Ent'| order by LastName;
print '';
print '-- Customers | where Occupation !hassuffix \'Ent\'';
Customers | where Occupation !hassuffix 'Ent'| order by LastName;
print '';
print '-- Customers | where Occupation hassuffix \'ent\'';
Customers | where Occupation hassuffix 'ent'| order by LastName;
print '';
print '-- Customers | where Occupation hassuffix \'ent\'';
Customers | where Occupation hassuffix 'ent'| order by LastName;
print '';
print '-- Customers |where Education in (\'Bachelors\',\'High School\')';
Customers |where Education in ('Bachelors','High School')| order by LastName;
print '';
print '-- Customers | where Education !in (\'Bachelors\',\'High School\')';
Customers | where Education !in ('Bachelors','High School')| order by LastName;
print '';
print '-- Customers | where FirstName matches regex \'P.*r\'';
Customers | where FirstName matches regex 'P.*r'| order by LastName;
print '';
print '-- Customers | where FirstName startswith \'pet\'';
Customers | where FirstName startswith 'pet'| order by LastName;
print '';
print '-- Customers | where FirstName !startswith \'pet\'';
Customers | where FirstName !startswith 'pet'| order by LastName;
print '';
print '-- Customers | where FirstName startswith_cs \'pet\'';
Customers | where FirstName startswith_cs 'pet'| order by LastName;
print '';
print '-- Customers | where FirstName !startswith_cs \'pet\'';
Customers | where FirstName !startswith_cs 'pet'| order by LastName;
print '';
print '-- Customers | where isempty(LastName)';
Customers | where isempty(LastName);
print '';
print '-- Customers | where isnotempty(LastName)';
Customers | where isnotempty(LastName);
print '';
print '-- Customers | where isnotnull(FirstName)';
Customers | where isnotnull(FirstName)| order by LastName;
print '';
print '-- Customers | where isnull(FirstName)';
Customers | where isnull(FirstName)| order by LastName;
print '';
print '-- Customers | project url_decode(\'https%3A%2F%2Fwww.test.com%2Fhello%20word\') | take 1';
Customers | project url_decode('https%3A%2F%2Fwww.test.com%2Fhello%20word') | take 1;
print '';
print '-- Customers | project url_encode(\'https://www.test.com/hello word\') | take 1';
Customers | project url_encode('https://www.test.com/hello word') | take 1;
print '';
print '-- Customers | project name_abbr = strcat(substring(FirstName,0,3), \' \', substring(LastName,2))';
Customers | project name_abbr = strcat(substring(FirstName,0,3), ' ', substring(LastName,2))| order by LastName;
print '';
print '-- Customers | project name = strcat(FirstName, \' \', LastName)';
Customers | project name = strcat(FirstName, ' ', LastName)| order by LastName;
print '';
print '-- Customers | project FirstName, strlen(FirstName)';
Customers | project FirstName, strlen(FirstName)| order by LastName;
print '';
print '-- Customers | project strrep(FirstName,2,\'_\')';
Customers | project strrep(FirstName,2,'_')| order by LastName;
print '';
print '-- Customers | project toupper(FirstName)';
Customers | project toupper(FirstName)| order by LastName;
print '';
print '-- Customers | project tolower(FirstName)';
Customers | project tolower(FirstName)| order by LastName;
print '';
print '-- support subquery for in orerator (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/in-cs-operator) (subquery need to be wraped with bracket inside bracket); TODO: case-insensitive not supported yet';
Customers | where Age in ((Customers|project Age|where Age < 30)) | order by LastName;
-- Customer | where LastName in~ ("diaz", "cox")
print '';
print '-- has_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-all-operator); TODO: subquery not supported yet';
Customers | where Occupation has_all ('manual', 'skilled') | order by LastName;
print '';
print '-- has_any (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-anyoperator); TODO: subquery not supported yet';
Customers|where Occupation has_any ('Skilled','abcd');
print '';
print '-- countof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/countoffunction)';
Customers | project countof('The cat sat on the mat', 'at') | take 1;
Customers | project countof('The cat sat on the mat', 'at', 'normal') | take 1;
Customers | project countof('The cat sat on the mat', '\\s.he', 'regex') | take 1;
print '';
print '-- extract ( https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractfunction)';
print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 0, 'The price of PINEAPPLE ice cream is 20');
print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 1, 'The price of PINEAPPLE ice cream is 20');
print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 20');
print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 3, 'The price of PINEAPPLE ice cream is 20');
print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 20', typeof(real));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(bool));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(date));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(guid));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(int));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(long));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(real));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(decimal));
print '';
print '-- extract_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractallfunction); TODO: captureGroups not supported yet';
Customers | project extract_all('(\\w)(\\w+)(\\w)','The price of PINEAPPLE ice cream is 20') | take 1;
print '';
print '-- extract_json (https://learn.microsoft.com/en-us/azure/data-explorer/kusto/query/extractjsonfunction)';
print extract_json('', ''); -- { serverError BAD_ARGUMENTS }
print extract_json('a', ''); -- { serverError BAD_ARGUMENTS }
print extract_json('$.firstName', '');
print extract_json('$.phoneNumbers[0].type', '');
print extractjson('$.firstName', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}');
print extract_json('$.phoneNumbers[0].type', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(string));
print extract_json('$.phoneNumbers[0].type', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(int));
print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}');
print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(int));
print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(long));
-- print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(bool)); -> true
print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(double));
print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(guid));
-- print extract_json('$.phoneNumbers', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(dynamic)); we won't be able to handle this particular case for a while, because it should return a dictionary
print '';
print '-- split (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/splitfunction)';
Customers | project split('aa_bb', '_') | take 1;
Customers | project split('aaa_bbb_ccc', '_', 1) | take 1;
Customers | project split('', '_') | take 1;
Customers | project split('a__b', '_') | take 1;
Customers | project split('aabbcc', 'bb') | take 1;
Customers | project split('aabbcc', '') | take 1;
Customers | project split('aaa_bbb_ccc', '_', -1) | take 1;
Customers | project split('aaa_bbb_ccc', '_', 10) | take 1;
print '';
print '-- strcat_delim (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/strcat-delimfunction); TODO: only support string now.';
Customers | project strcat_delim('-', '1', '2', strcat('A','b')) | take 1;
-- Customers | project strcat_delim('-', '1', '2', 'A' , 1s);
print '';
print '-- indexof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/indexoffunction); TODO: length and occurrence not supported yet';
Customers | project indexof('abcdefg','cde') | take 1;
Customers | project indexof('abcdefg','cde',2) | take 1;
Customers | project indexof('abcdefg','cde',6) | take 1;
print '-- base64_encode_fromguid()';
-- print base64_encode_fromguid(guid(null));
print base64_encode_fromguid(guid('ae3133f2-6e22-49ae-b06a-16e6a9b212eb'));
print base64_encode_fromguid(dynamic(null)); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }
print base64_encode_fromguid("abcd1231"); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }
print '-- base64_decode_toarray()';
print base64_decode_toarray('');
print base64_decode_toarray('S3VzdG8=');
print '-- base64_decode_toguid()';
print base64_decode_toguid("JpbpECu8dUy7Pv5gbeJXAA==");
print base64_decode_toguid(base64_encode_fromguid(guid('ae3133f2-6e22-49ae-b06a-16e6a9b212eb'))) == guid('ae3133f2-6e22-49ae-b06a-16e6a9b212eb');
print '-- base64_encode_tostring';
print base64_encode_tostring('');
print base64_encode_tostring('Kusto1');
print '-- base64_decode_tostring';
print base64_decode_tostring('');
print base64_decode_tostring('S3VzdG8x');
print '-- parse_url()';
print parse_url('scheme://username:password@host:1234/this/is/a/path?k1=v1&k2=v2#fragment');
print '-- parse_urlquery()';
print parse_urlquery('k1=v1&k2=v2&k3=v3');
print '-- strcmp()';
print strcmp('ABC','ABC'), strcmp('abc','ABC'), strcmp('ABC','abc'), strcmp('abcde','abc');
print '-- substring()';
print substring("ABCD", -2, 2);
print '-- translate()';
print translate('krasp', 'otsku', 'spark'), translate('abc', '', 'ab'), translate('abc', 'x', 'abc');
print '-- trim()';
print trim("--", "--https://www.ibm.com--");
print trim("[^\w]+", strcat("- ","Te st", "1", "// $"));
print trim("", " asd ");
print trim("a$", "asd");
print trim("^a", "asd");
print '-- trim_start()';
print trim_start("https://", "https://www.ibm.com");
print trim_start("[^\w]+", strcat("- ","Te st", "1", "// $"));
print trim_start("asd$", "asdw");
print trim_start("asd$", "asd");
print trim_start("d$", "asd");
print '-- trim_end()';
print trim_end("://www.ibm.com", "https://www.ibm.com");
print trim_end("[^\w]+", strcat("- ","Te st", "1", "// $"));
print trim_end("^asd", "wasd");
print trim_end("^asd", "asd");
print trim_end("^a", "asd");
print '-- trim, trim_start, trim_end all at once';
print str = "--https://bing.com--", pattern = '--' | extend start = trim_start(pattern, str), end = trim_end(pattern, str), both = trim(pattern, str);
print '-- replace_regex';
print replace_regex(strcat('Number is ', '1'), 'is (\d+)', 'was: \1');
print '-- has_any_index()';
print has_any_index('this is an example', dynamic(['this', 'example'])), has_any_index("this is an example", dynamic(['not', 'example'])), has_any_index("this is an example", dynamic(['not', 'found'])), has_any_index("this is an example", dynamic([]));
print '-- parse_version()';
print parse_version(42); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- print parse_version(''); -> NULL
print parse_version('1.2.3.40');
print parse_version('1.2');
print parse_version(strcat('1.', '2'));
print parse_version('1.2.4.5.6');
print parse_version('moo');
print parse_version('moo.boo.foo');
print parse_version(strcat_delim('.', 'moo', 'boo', 'foo'));
Versions | project parse_version(Version);
print '-- parse_json()';
print parse_json(dynamic([1, 2, 3]));
print parse_json('{"a":123.5, "b":"{\\"c\\":456}"}');
print '-- parse_command_line()';
print parse_command_line(55, 'windows'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- print parse_command_line((52 + 3) * 4 % 2, 'windows'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
print parse_command_line('', 'windows');
print parse_command_line(strrep(' ', 6), 'windows');
-- print parse_command_line('echo \"hello world!\" print$?', 'windows'); -> ["echo","hello world!","print$?"]
-- print parse_command_line("yolo swag 'asd bcd' \"moo moo \"", 'windows'); -> ["yolo","swag","'asd","bcd'","moo moo "]
-- print parse_command_line(strcat_delim(' ', "yolo", "swag", "\'asd bcd\'", "\"moo moo \""), 'windows'); -> ["yolo","swag","'asd","bcd'","moo moo "]
print '-- reverse()';
print reverse(123);
print reverse(123.34);
print reverse('');
print reverse("asd");
print reverse(dynamic([]));
print reverse(dynamic([1, 2, 3]));
print reverse(dynamic(['Darth', "Vader"]));
print reverse(datetime(2017-10-15 12:00));
-- print reverse(timespan(3h)); -> 00:00:30
Customers | where Education contains 'degree' | order by reverse(FirstName);
print '-- parse_csv()';
print parse_csv('');
print parse_csv(65); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
print parse_csv('aaa');
print result=parse_csv('aa,b,cc');
print result_multi_record=parse_csv('record1,a,b,c\nrecord2,x,y,z');
-- print result=parse_csv('aa,"b,b,b",cc,"Escaping quotes: ""Title""","line1\nline2"'); -> ["aa","b,b,b","cc","Escaping quotes: \"Title\"","line1\nline2"]
-- print parse_csv(strcat(strcat_delim(',', 'aa', '"b,b,b"', 'cc', '"Escaping quotes: ""Title"""', '"line1\nline2"'), '\r\n', strcat_delim(',', 'asd', 'qcf'))); -> ["aa","b,b,b","cc","Escaping quotes: \"Title\"","line1\nline2"]

View File

@ -0,0 +1,65 @@
-- mv-expand --
-- mv_expand_test_table | mv-expand c --
1 ['Salmon','Steak','Chicken'] 1 [5,6,7,8]
1 ['Salmon','Steak','Chicken'] 2 [5,6,7,8]
1 ['Salmon','Steak','Chicken'] 3 [5,6,7,8]
1 ['Salmon','Steak','Chicken'] 4 [5,6,7,8]
-- mv_expand_test_table | mv-expand c, d --
1 ['Salmon','Steak','Chicken'] 1 5
1 ['Salmon','Steak','Chicken'] 2 6
1 ['Salmon','Steak','Chicken'] 3 7
1 ['Salmon','Steak','Chicken'] 4 8
-- mv_expand_test_table | mv-expand b | mv-expand c --
1 Salmon 1 [5,6,7,8]
1 Salmon 2 [5,6,7,8]
1 Salmon 3 [5,6,7,8]
1 Salmon 4 [5,6,7,8]
1 Steak 1 [5,6,7,8]
1 Steak 2 [5,6,7,8]
1 Steak 3 [5,6,7,8]
1 Steak 4 [5,6,7,8]
1 Chicken 1 [5,6,7,8]
1 Chicken 2 [5,6,7,8]
1 Chicken 3 [5,6,7,8]
1 Chicken 4 [5,6,7,8]
-- mv_expand_test_table | mv-expand with_itemindex=index b, c, d --
0 1 Salmon 1 5
1 1 Steak 2 6
2 1 Chicken 3 7
3 1 4 8
-- mv_expand_test_table | mv-expand array_concat(c,d) --
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 1
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 2
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 3
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 4
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 5
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 6
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 7
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 8
-- mv_expand_test_table | mv-expand x = c, y = d --
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 1 5
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 2 6
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 3 7
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 4 8
-- mv_expand_test_table | mv-expand xy = array_concat(c, d) --
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 1
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 2
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 3
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 4
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 5
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 6
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 7
1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 8
-- mv_expand_test_table | mv-expand xy = array_concat(c, d) limit 2| summarize count() by xy --
1 1
2 1
-- mv_expand_test_table | mv-expand with_itemindex=index c,d to typeof(bool) --
0 1 ['Salmon','Steak','Chicken'] 1 true
1 1 ['Salmon','Steak','Chicken'] 2 true
2 1 ['Salmon','Steak','Chicken'] 3 true
3 1 ['Salmon','Steak','Chicken'] 4 true
-- mv_expand_test_table | mv-expand c to typeof(bool) --
1 ['Salmon','Steak','Chicken'] [5,6,7,8] true
1 ['Salmon','Steak','Chicken'] [5,6,7,8] true
1 ['Salmon','Steak','Chicken'] [5,6,7,8] true
1 ['Salmon','Steak','Chicken'] [5,6,7,8] true

View File

@ -0,0 +1,35 @@
-- datatable(a: int, b: dynamic, c: dynamic, d: dynamic) [
-- 1, dynamic(['Salmon', 'Steak', 'Chicken']), dynamic([1, 2, 3, 4]), dynamic([5, 6, 7, 8])
-- ]
DROP TABLE IF EXISTS mv_expand_test_table;
CREATE TABLE mv_expand_test_table
(
a UInt8,
b Array(String),
c Array(Int8),
d Array(Int8)
) ENGINE = Memory;
INSERT INTO mv_expand_test_table VALUES (1, ['Salmon', 'Steak','Chicken'],[1,2,3,4],[5,6,7,8]);
set dialect='kusto';
print '-- mv-expand --';
print '-- mv_expand_test_table | mv-expand c --';
mv_expand_test_table | mv-expand c;
print '-- mv_expand_test_table | mv-expand c, d --';
mv_expand_test_table | mv-expand c, d;
print '-- mv_expand_test_table | mv-expand b | mv-expand c --';
mv_expand_test_table | mv-expand b | mv-expand c;
print '-- mv_expand_test_table | mv-expand with_itemindex=index b, c, d --';
mv_expand_test_table | mv-expand with_itemindex=index b, c, d;
print '-- mv_expand_test_table | mv-expand array_concat(c,d) --';
mv_expand_test_table | mv-expand array_concat(c,d);
print '-- mv_expand_test_table | mv-expand x = c, y = d --';
mv_expand_test_table | mv-expand x = c, y = d;
print '-- mv_expand_test_table | mv-expand xy = array_concat(c, d) --';
mv_expand_test_table | mv-expand xy = array_concat(c, d);
print '-- mv_expand_test_table | mv-expand xy = array_concat(c, d) limit 2| summarize count() by xy --';
mv_expand_test_table | mv-expand xy = array_concat(c, d) limit 2| summarize count() by xy;
print '-- mv_expand_test_table | mv-expand with_itemindex=index c,d to typeof(bool) --';
mv_expand_test_table | mv-expand with_itemindex=index c,d to typeof(bool);
print '-- mv_expand_test_table | mv-expand c to typeof(bool) --';
mv_expand_test_table | mv-expand c to typeof(bool);

View File

@ -0,0 +1,23 @@
numeric
kusto
00:00:00
00:00:00.0000001
00:00:00.0010000
00:00:42
01:06:00
2.18:00:00
5.00:00:00
7.00:00:00
14.00:00:00
('00:01:12','21.00:00:00','00:00:00.0000002')
numeric
99
100
1
42
66
66
5
1
2
(72,3,200)

View File

@ -0,0 +1,16 @@
select value from system.settings where name = 'interval_output_format';
{% for format in ['kusto', 'numeric'] -%}
select '{{ format }}';
set interval_output_format = '{{ format }}';
select toIntervalNanosecond(99);
select toIntervalNanosecond(100);
select toIntervalMillisecond(1);
select toIntervalSecond(42);
select toIntervalMinute(66);
select toIntervalHour(66);
select toIntervalDay(5);
select toIntervalWeek(1);
select toIntervalWeek(2);
select toIntervalSecond(72) + toIntervalWeek(3) + toIntervalNanosecond(200);
{% endfor -%}

View File

@ -0,0 +1,60 @@
-- #1 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Apple Skilled Manual Bachelors 28
-- #2 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Peter Nara Skilled Manual Graduate Degree 26
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #3 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #4 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #5 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #6 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #7 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #8 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #9 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Peter Nara Skilled Manual Graduate Degree 26
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #10 --
-- #11 --
-- #12 --
-- #13 --
-- #14 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28
-- #15 --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management abcd defg Bachelors 33
Latoya Shen Professional Graduate Degree 25
Apple Skilled Manual Bachelors 28

View File

@ -0,0 +1,42 @@
DROP TABLE IF EXISTS Customers;
CREATE TABLE Customers
(
FirstName Nullable(String),
LastName String,
Occupation String,
Education String,
Age Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28),('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38);
Select '-- #1 --' ;
select * from kql($$Customers | where FirstName !in ('Peter', 'Latoya')$$);
Select '-- #2 --' ;
select * from kql($$Customers | where FirstName !in ("test", "test2")$$);
Select '-- #3 --' ;
select * from kql($$Customers | where FirstName !contains 'Pet'$$);
Select '-- #4 --' ;
select * from kql($$Customers | where FirstName !contains_cs 'Pet'$$);
Select '-- #5 --' ;
select * from kql($$Customers | where FirstName !endswith 'ter'$$);
Select '-- #6 --' ;
select * from kql($$Customers | where FirstName !endswith_cs 'ter'$$);
Select '-- #7 --' ;
select * from kql($$Customers | where FirstName != 'Peter'$$);
Select '-- #8 --' ;
select * from kql($$Customers | where FirstName !has 'Peter'$$);
Select '-- #9 --' ;
select * from kql($$Customers | where FirstName !has_cs 'peter'$$);
Select '-- #10 --' ;
-- select * from kql($$Customers | where FirstName !hasprefix 'Peter'$$); -- will enable when analyzer fixed `and` issue
Select '-- #11 --' ;
--select * from kql($$Customers | where FirstName !hasprefix_cs 'Peter'$$);
Select '-- #12 --' ;
--select * from kql($$Customers | where FirstName !hassuffix 'Peter'$$);
Select '-- #13 --' ;
--select * from kql($$Customers | where FirstName !hassuffix_cs 'Peter'$$);
Select '-- #14 --' ;
select * from kql($$Customers | where FirstName !startswith 'Peter'$$);
Select '-- #15 --' ;
select * from kql($$Customers | where FirstName !startswith_cs 'Peter'$$);
DROP TABLE IF EXISTS Customers;

View File

@ -0,0 +1,92 @@
-- test summarize --
12 25 46 32.416666666666664 389
Skilled Manual 5 26 36 30.2 151
Professional 6 25 46 34.166666666666664 205
Management abcd defg 1 33 33 33 33
Skilled Manual 0
Professional 2
Management abcd defg 0
Skilled Manual 36
Professional 38
Management abcd defg 33
Skilled Manual 26
Professional 25
Management abcd defg 33
Skilled Manual 30.2
Professional 29.25
Management abcd defg 33
Skilled Manual 151
Professional 117
Management abcd defg 33
4
2
40 2
30 4
20 6
Skilled Manual 5
Professional 6
Management abcd defg 1
-- make_list() --
Skilled Manual ['Bachelors','Graduate Degree','High School','Partial College','Bachelors']
Professional ['Graduate Degree','Partial College','Partial College','Partial College','Partial College','Partial College']
Management abcd defg ['Bachelors']
Skilled Manual ['Bachelors','Graduate Degree']
Professional ['Graduate Degree','Partial College']
Management abcd defg ['Bachelors']
-- make_list_if() --
Skilled Manual ['Edward','Christine']
Professional ['Dalton','Angel']
Management abcd defg ['Stephanie']
Skilled Manual ['Edward']
Professional ['Dalton']
Management abcd defg ['Stephanie']
-- make_set() --
Skilled Manual ['Graduate Degree','High School','Partial College','Bachelors']
Professional ['Graduate Degree','Partial College']
Management abcd defg ['Bachelors']
Skilled Manual ['Graduate Degree','Bachelors']
Professional ['Graduate Degree','Partial College']
Management abcd defg ['Bachelors']
-- make_set_if() --
Skilled Manual ['Partial College','High School']
Professional ['Partial College']
Management abcd defg ['Bachelors']
Skilled Manual ['High School']
Professional ['Partial College']
Management abcd defg ['Bachelors']
-- stdev() --
6.855102059227432
-- stdevif() --
7.557189365836421
-- binary_all_and --
42
-- binary_all_or --
46
-- binary_all_xor --
4
43.8
25.55 30.5 43.8
30.5
35
[25,35,45]
-- Summarize following sort --
Skilled Manual 5
Professional 6
Management abcd defg 1
-- summarize with bin --
0 1
245000 2
0 1
245 2
0 1
245 2
2015-10-12 00:00:00.000000000
2016-10-12 00:00:00.000000000
-- make_list_with_nulls --
['Theodore','Stephanie','Peter','Latoya','Joshua','Edward','Dalton','Christine','Cameron','Angel','Apple',NULL]
Skilled Manual ['Theodore','Peter','Edward','Christine','Apple']
Professional ['Latoya','Joshua','Dalton','Cameron','Angel',NULL]
Management abcd defg ['Stephanie']
Skilled Manual ['Theodore','Peter','Edward','Christine','Apple'] [28,26,36,33,28]
Professional ['Latoya','Joshua','Dalton','Cameron','Angel',NULL] [25,26,42,28,46,38]
Management abcd defg ['Stephanie'] [33]

View File

@ -0,0 +1,102 @@
-- datatable(FirstName:string, LastName:string, Occupation:string, Education:string, Age:int) [
-- 'Theodore', 'Diaz', 'Skilled Manual', 'Bachelors', 28,
-- 'Stephanie', 'Cox', 'Management abcd defg', 'Bachelors', 33,
-- 'Peter', 'Nara', 'Skilled Manual', 'Graduate Degree', 26,
-- 'Latoya', 'Shen', 'Professional', 'Graduate Degree', 25,
-- 'Joshua', 'Lee', 'Professional', 'Partial College', 26,
-- 'Edward', 'Hernandez', 'Skilled Manual', 'High School', 36,
-- 'Dalton', 'Wood', 'Professional', 'Partial College', 42,
-- 'Christine', 'Nara', 'Skilled Manual', 'Partial College', 33,
-- 'Cameron', 'Rodriguez', 'Professional', 'Partial College', 28,
-- 'Angel', 'Stewart', 'Professional', 'Partial College', 46,
-- 'Apple', '', 'Skilled Manual', 'Bachelors', 28,
-- dynamic(null), 'why', 'Professional', 'Partial College', 38
-- ]
DROP TABLE IF EXISTS Customers;
CREATE TABLE Customers
(
FirstName Nullable(String),
LastName String,
Occupation String,
Education String,
Age Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28),('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Joshua','Lee','Professional','Partial College',26),('Edward','Hernandez','Skilled Manual','High School',36),('Dalton','Wood','Professional','Partial College',42),('Christine','Nara','Skilled Manual','Partial College',33),('Cameron','Rodriguez','Professional','Partial College',28),('Angel','Stewart','Professional','Partial College',46),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38);
drop table if exists EventLog;
create table EventLog
(
LogEntry String,
Created Int64
) ENGINE = Memory;
insert into EventLog values ('Darth Vader has entered the room.', 546), ('Rambo is suspciously looking at Darth Vader.', 245234), ('Darth Sidious electrocutes both using Force Lightning.', 245554);
drop table if exists Dates;
create table Dates
(
EventTime DateTime,
) ENGINE = Memory;
Insert into Dates VALUES ('2015-10-12') , ('2016-10-12')
Select '-- test summarize --' ;
set dialect='kusto';
Customers | summarize count(), min(Age), max(Age), avg(Age), sum(Age);
Customers | summarize count(), min(Age), max(Age), avg(Age), sum(Age) by Occupation | order by Occupation;
Customers | summarize countif(Age>40) by Occupation | order by Occupation;
Customers | summarize MyMax = maxif(Age, Age<40) by Occupation | order by Occupation;
Customers | summarize MyMin = minif(Age, Age<40) by Occupation | order by Occupation;
Customers | summarize MyAvg = avgif(Age, Age<40) by Occupation | order by Occupation;
Customers | summarize MySum = sumif(Age, Age<40) by Occupation | order by Occupation;
Customers | summarize dcount(Education);
Customers | summarize dcountif(Education, Occupation=='Professional');
Customers | summarize count_ = count() by bin(Age, 10) | order by count_ asc;
Customers | summarize job_count = count() by Occupation | where job_count > 0 | order by Occupation;
Customers | summarize 'Edu Count'=count() by Education | sort by 'Edu Count' desc; -- { clientError 62 }
print '-- make_list() --';
Customers | summarize f_list = make_list(Education) by Occupation | sort by Occupation;
Customers | summarize f_list = make_list(Education, 2) by Occupation | sort by Occupation;
print '-- make_list_if() --';
Customers | summarize f_list = make_list_if(FirstName, Age>30) by Occupation | sort by Occupation;
Customers | summarize f_list = make_list_if(FirstName, Age>30, 1) by Occupation | sort by Occupation;
print '-- make_set() --';
Customers | summarize f_list = make_set(Education) by Occupation | sort by Occupation;
Customers | summarize f_list = make_set(Education, 2) by Occupation | sort by Occupation;
print '-- make_set_if() --';
Customers | summarize f_list = make_set_if(Education, Age>30) by Occupation | sort by Occupation;
Customers | summarize f_list = make_set_if(Education, Age>30, 1) by Occupation | sort by Occupation;
print '-- stdev() --';
Customers | project Age | summarize stdev(Age);
print '-- stdevif() --';
Customers | project Age | summarize stdevif(Age, Age%2==0);
print '-- binary_all_and --';
Customers | project Age | where Age > 40 | summarize binary_all_and(Age);
print '-- binary_all_or --';
Customers | project Age | where Age > 40 | summarize binary_all_or(Age);
print '-- binary_all_xor --';
Customers | project Age | where Age > 40 | summarize binary_all_xor(Age);
Customers | project Age | summarize percentile(Age, 95);
Customers | project Age | summarize percentiles(Age, 5, 50, 95)|project round(percentiles_Age[0],2),round(percentiles_Age[1],2),round(percentiles_Age[2],2);
Customers | project Age | summarize percentiles(Age, 5, 50, 95)[1];
Customers | summarize w=count() by AgeBucket=bin(Age, 5) | summarize percentilew(AgeBucket, w, 75);
Customers | summarize w=count() by AgeBucket=bin(Age, 5) | summarize percentilesw(AgeBucket, w, 50, 75, 99.9);
print '-- Summarize following sort --';
Customers | sort by FirstName | summarize count() by Occupation | sort by Occupation;
print '-- summarize with bin --';
EventLog | summarize count=count() by bin(Created, 1000);
EventLog | summarize count=count() by bin(unixtime_seconds_todatetime(Created/1000), 1s);
EventLog | summarize count=count() by time_label=bin(Created/1000, 1s);
Dates | project bin(datetime(EventTime), 1m);
print '-- make_list_with_nulls --';
Customers | summarize t = make_list_with_nulls(FirstName);
Customers | summarize f_list = make_list_with_nulls(FirstName) by Occupation | sort by Occupation;
Customers | summarize f_list = make_list_with_nulls(FirstName), a_list = make_list_with_nulls(Age) by Occupation | sort by Occupation;
-- TODO:
-- arg_max()
-- arg_min()

View File

@ -0,0 +1,139 @@
-- test Query only has table name: --
Theodore Diaz Skilled Manual Bachelors 28
Stephanie Cox Management Bachelors 33
Peter Nara Skilled Manual Graduate Degree 26
Latoya Shen Professional Graduate Degree 25
Joshua Lee Professional Partial College 26
Edward Hernandez Skilled Manual High School 36
Dalton Wood Professional Partial College 42
Christine Nara Skilled Manual Partial College 33
Cameron Rodriguez Professional Partial College 28
Angel Stewart Professional Partial College 46
-- Query has Column Selection --
Theodore Diaz Skilled Manual
Stephanie Cox Management
Peter Nara Skilled Manual
Latoya Shen Professional
Joshua Lee Professional
Edward Hernandez Skilled Manual
Dalton Wood Professional
Christine Nara Skilled Manual
Cameron Rodriguez Professional
Angel Stewart Professional
-- Query has limit --
Theodore Diaz Skilled Manual
Stephanie Cox Management
Peter Nara Skilled Manual
Latoya Shen Professional
Joshua Lee Professional
Theodore Diaz Skilled Manual
Stephanie Cox Management
Peter Nara Skilled Manual
Latoya Shen Professional
Joshua Lee Professional
-- Query has second limit with bigger value --
Theodore Diaz Skilled Manual
Stephanie Cox Management
Peter Nara Skilled Manual
Latoya Shen Professional
Joshua Lee Professional
-- Query has second limit with smaller value --
Theodore Diaz Skilled Manual
Stephanie Cox Management
Peter Nara Skilled Manual
-- Query has second Column selection --
Theodore Diaz
Stephanie Cox
Peter Nara
-- Query has second Column selection with extra column --
-- Query with desc sort --
Theodore
Stephanie
Peter
Latoya
Joshua
Skilled Manual
Skilled Manual
Professional
Professional
Management
-- Query with asc sort --
Management
Professional
Professional
Skilled Manual
Skilled Manual
-- Query with sort (without keyword asc desc) --
Theodore
Stephanie
Peter
Latoya
Joshua
Skilled Manual
Skilled Manual
Professional
Professional
Management
-- Query with sort 2 Columns with different direction --
Stephanie Cox Management
Latoya Shen Professional
Joshua Lee Professional
Peter Nara Skilled Manual
Theodore Diaz Skilled Manual
-- Query with second sort --
Stephanie Cox Management
Latoya Shen Professional
Joshua Lee Professional
Peter Nara Skilled Manual
Theodore Diaz Skilled Manual
-- Test String Equals (==) --
Theodore Diaz Skilled Manual
Peter Nara Skilled Manual
Edward Hernandez Skilled Manual
Christine Nara Skilled Manual
-- Test String Not equals (!=) --
Stephanie Cox Management
Latoya Shen Professional
Joshua Lee Professional
Dalton Wood Professional
Cameron Rodriguez Professional
Angel Stewart Professional
-- Test Filter using a list (in) --
Theodore Diaz Skilled Manual Bachelors
Stephanie Cox Management Bachelors
Edward Hernandez Skilled Manual High School
-- Test Filter using a list (!in) --
Peter Nara Skilled Manual Graduate Degree
Latoya Shen Professional Graduate Degree
Joshua Lee Professional Partial College
Dalton Wood Professional Partial College
Christine Nara Skilled Manual Partial College
Cameron Rodriguez Professional Partial College
Angel Stewart Professional Partial College
-- Test Filter using common string operations (contains_cs) --
Joshua Lee Professional Partial College
Dalton Wood Professional Partial College
Christine Nara Skilled Manual Partial College
Cameron Rodriguez Professional Partial College
Angel Stewart Professional Partial College
-- Test Filter using common string operations (startswith_cs) --
Latoya Shen Professional Graduate Degree
Joshua Lee Professional Partial College
Dalton Wood Professional Partial College
Cameron Rodriguez Professional Partial College
Angel Stewart Professional Partial College
-- Test Filter using common string operations (endswith_cs) --
Latoya Shen Professional Graduate Degree
Joshua Lee Professional Partial College
-- Test Filter using numerical equal (==) --
Peter Nara Skilled Manual Graduate Degree 26
Joshua Lee Professional Partial College 26
-- Test Filter using numerical great and less (> , <) --
Stephanie Cox Management Bachelors 33
Edward Hernandez Skilled Manual High School 36
Christine Nara Skilled Manual Partial College 33
-- Test Filter using multi where --
Dalton Wood Professional Partial College 42
Angel Stewart Professional Partial College 46
-- Complex query with unknown function --
-- Missing column in front of startsWith --

View File

@ -0,0 +1,93 @@
DROP TABLE IF EXISTS Customers;
CREATE TABLE Customers
(
FirstName Nullable(String),
LastName String,
Occupation String,
Education String,
Age Nullable(UInt8)
) ENGINE = Memory;
INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28), ('Stephanie','Cox','Management','Bachelors',33), ('Peter','Nara','Skilled Manual','Graduate Degree',26), ('Latoya','Shen','Professional','Graduate Degree',25), ('Joshua','Lee','Professional','Partial College',26), ('Edward','Hernandez','Skilled Manual','High School',36), ('Dalton','Wood','Professional','Partial College',42), ('Christine','Nara','Skilled Manual','Partial College',33), ('Cameron','Rodriguez','Professional','Partial College',28), ('Angel','Stewart','Professional','Partial College',46);
set dialect='kusto';
print '-- test Query only has table name: --';
Customers;
print '-- Query has Column Selection --';
Customers | project FirstName,LastName,Occupation;
print '-- Query has limit --';
Customers | project FirstName,LastName,Occupation | take 5;
Customers | project FirstName,LastName,Occupation | limit 5;
print '-- Query has second limit with bigger value --';
Customers | project FirstName,LastName,Occupation | take 5 | take 7;
print '-- Query has second limit with smaller value --';
Customers | project FirstName,LastName,Occupation | take 5 | take 3;
print '-- Query has second Column selection --';
Customers | project FirstName,LastName,Occupation | take 3 | project FirstName,LastName;
print '-- Query has second Column selection with extra column --';
Customers| project FirstName,LastName,Occupation | take 3 | project FirstName,LastName,Education;-- { serverError 47 }
print '-- Query with desc sort --';
Customers | project FirstName | take 5 | sort by FirstName desc;
Customers | project Occupation | take 5 | order by Occupation desc;
print '-- Query with asc sort --';
Customers | project Occupation | take 5 | sort by Occupation asc;
print '-- Query with sort (without keyword asc desc) --';
Customers | project FirstName | take 5 | sort by FirstName;
Customers | project Occupation | take 5 | order by Occupation;
print '-- Query with sort 2 Columns with different direction --';
Customers | project FirstName,LastName,Occupation | take 5 | sort by Occupation asc, LastName desc;
print '-- Query with second sort --';
Customers | project FirstName,LastName,Occupation | take 5 | sort by Occupation desc |sort by Occupation asc, LastName desc;
print '-- Test String Equals (==) --';
Customers | project FirstName,LastName,Occupation | where Occupation == 'Skilled Manual';
print '-- Test String Not equals (!=) --';
Customers | project FirstName,LastName,Occupation | where Occupation != 'Skilled Manual';
print '-- Test Filter using a list (in) --';
Customers | project FirstName,LastName,Occupation,Education | where Education in ('Bachelors','High School');
print '-- Test Filter using a list (!in) --';
set dialect='kusto';
Customers | project FirstName,LastName,Occupation,Education | where Education !in ('Bachelors','High School');
print '-- Test Filter using common string operations (contains_cs) --';
Customers | project FirstName,LastName,Occupation,Education | where Education contains_cs 'Coll';
print '-- Test Filter using common string operations (startswith_cs) --';
Customers | project FirstName,LastName,Occupation,Education | where Occupation startswith_cs 'Prof';
print '-- Test Filter using common string operations (endswith_cs) --';
Customers | project FirstName,LastName,Occupation,Education | where FirstName endswith_cs 'a';
print '-- Test Filter using numerical equal (==) --';
Customers | project FirstName,LastName,Occupation,Education,Age | where Age == 26;
print '-- Test Filter using numerical great and less (> , <) --';
Customers | project FirstName,LastName,Occupation,Education,Age | where Age > 30 and Age < 40;
print '-- Test Filter using multi where --';
Customers | project FirstName,LastName,Occupation,Education,Age | where Age > 30 | where Occupation == 'Professional';
print '-- Complex query with unknown function --';
hits | where CounterID == 62 and EventDate >= '2013-07-14' and EventDate <= '2013-07-15' and IsRefresh == 0 and DontCountHits == 0 | summarize count() by d=bin(poopoo(EventTime), 1m) | order by d | limit 10; -- { clientError UNKNOWN_FUNCTION }
print '-- Missing column in front of startsWith --';
StormEvents | where startswith "W" | summarize Count=count() by State; -- { clientError SYNTAX_ERROR }
SET max_query_size = 55;
SET dialect='kusto';
Customers | where Education contains 'degree' | order by LastName; -- { serverError 62 }
SET max_query_size=262144;