mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Merge remote-tracking branch 'rschu1ze/master' into better-use-mysql-types-in-show-columns
This commit is contained in:
commit
9d04d3c3ad
2
contrib/curl
vendored
2
contrib/curl
vendored
@ -1 +1 @@
|
||||
Subproject commit eb3b049df526bf125eda23218e680ce7fa9ec46c
|
||||
Subproject commit d755a5f7c009dd63a61b2c745180d8ba937cbfeb
|
@ -64,6 +64,7 @@ set (SRCS
|
||||
"${LIBRARY_DIR}/lib/hostsyn.c"
|
||||
"${LIBRARY_DIR}/lib/hsts.c"
|
||||
"${LIBRARY_DIR}/lib/http.c"
|
||||
"${LIBRARY_DIR}/lib/http1.c"
|
||||
"${LIBRARY_DIR}/lib/http2.c"
|
||||
"${LIBRARY_DIR}/lib/http_aws_sigv4.c"
|
||||
"${LIBRARY_DIR}/lib/http_chunks.c"
|
||||
|
@ -14,12 +14,12 @@ add_library(_gtest_all INTERFACE)
|
||||
target_link_libraries(_gtest_all INTERFACE _gtest _gtest_main)
|
||||
add_library(ch_contrib::gtest_all ALIAS _gtest_all)
|
||||
|
||||
|
||||
add_library(_gmock "${SRC_DIR}/googlemock/src/gmock-all.cc")
|
||||
set_target_properties(_gmock PROPERTIES VERSION "1.0.0")
|
||||
target_compile_definitions (_gmock PUBLIC GTEST_HAS_POSIX_RE=0)
|
||||
target_include_directories(_gmock SYSTEM PUBLIC "${SRC_DIR}/googlemock/include" "${SRC_DIR}/googletest/include")
|
||||
target_include_directories(_gmock PRIVATE "${SRC_DIR}/googlemock")
|
||||
target_link_libraries(_gmock PUBLIC _gtest)
|
||||
|
||||
add_library(_gmock_main "${SRC_DIR}/googlemock/src/gmock_main.cc")
|
||||
set_target_properties(_gmock_main PROPERTIES VERSION "1.0.0")
|
||||
|
@ -10,6 +10,5 @@ RUN curl -L -o /mysql-connector-j-${ver}.jar https://repo1.maven.org/maven2/com/
|
||||
ENV CLASSPATH=$CLASSPATH:/mysql-connector-j-${ver}.jar
|
||||
|
||||
WORKDIR /jdbc
|
||||
COPY Test.java Test.java
|
||||
COPY PreparedStatementsTest.java PreparedStatementsTest.java
|
||||
RUN javac Test.java PreparedStatementsTest.java
|
||||
COPY MySQLJavaClientTest.java MySQLJavaClientTest.java
|
||||
RUN javac MySQLJavaClientTest.java
|
||||
|
@ -2,7 +2,7 @@ import com.mysql.cj.MysqlType;
|
||||
|
||||
import java.sql.*;
|
||||
|
||||
public class PreparedStatementsTest {
|
||||
public class MySQLJavaClientTest {
|
||||
public static void main(String[] args) {
|
||||
int i = 0;
|
||||
String host = "127.0.0.1";
|
||||
@ -10,6 +10,7 @@ public class PreparedStatementsTest {
|
||||
String user = "default";
|
||||
String password = "";
|
||||
String database = "default";
|
||||
String binary = "false";
|
||||
while (i < args.length) {
|
||||
switch (args[i]) {
|
||||
case "--host":
|
||||
@ -27,16 +28,19 @@ public class PreparedStatementsTest {
|
||||
case "--database":
|
||||
database = args[++i];
|
||||
break;
|
||||
case "--binary":
|
||||
binary = args[++i];
|
||||
break;
|
||||
default:
|
||||
i++;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// useServerPrepStmts uses COM_STMT_PREPARE and COM_STMT_EXECUTE
|
||||
// instead of COM_QUERY which allows us to test the binary protocol
|
||||
String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false&useServerPrepStmts=true",
|
||||
host, port, database);
|
||||
// useServerPrepStmts=true -> COM_STMT_PREPARE + COM_STMT_EXECUTE -> binary
|
||||
// useServerPrepStmts=false -> COM_QUERY -> text
|
||||
String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false&useServerPrepStmts=%s",
|
||||
host, port, database, binary);
|
||||
|
||||
try {
|
||||
Class.forName("com.mysql.cj.jdbc.Driver");
|
||||
@ -49,6 +53,7 @@ public class PreparedStatementsTest {
|
||||
testDateTypes(conn);
|
||||
testUnusualDateTime64Scales(conn);
|
||||
testDateTimeTimezones(conn);
|
||||
testSuspiciousNullableLowCardinalityTypes(conn);
|
||||
conn.close();
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
@ -58,7 +63,7 @@ public class PreparedStatementsTest {
|
||||
|
||||
private static void testSimpleDataTypes(Connection conn) throws SQLException {
|
||||
System.out.println("### testSimpleDataTypes");
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_simple_data_types").executeQuery();
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM simple_data_types").executeQuery();
|
||||
int rowNum = 1;
|
||||
while (rs.next()) {
|
||||
System.out.printf("Row #%d\n", rowNum++);
|
||||
@ -83,7 +88,7 @@ public class PreparedStatementsTest {
|
||||
|
||||
private static void testStringTypes(Connection conn) throws SQLException {
|
||||
System.out.println("### testStringTypes");
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_string_types").executeQuery();
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM string_types").executeQuery();
|
||||
int rowNum = 1;
|
||||
while (rs.next()) {
|
||||
System.out.printf("Row #%d\n", rowNum++);
|
||||
@ -97,7 +102,7 @@ public class PreparedStatementsTest {
|
||||
|
||||
private static void testLowCardinalityAndNullableTypes(Connection conn) throws SQLException {
|
||||
System.out.println("### testLowCardinalityAndNullableTypes");
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_low_cardinality_and_nullable_types").executeQuery();
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM low_cardinality_and_nullable_types").executeQuery();
|
||||
int rowNum = 1;
|
||||
while (rs.next()) {
|
||||
System.out.printf("Row #%d\n", rowNum++);
|
||||
@ -111,7 +116,7 @@ public class PreparedStatementsTest {
|
||||
|
||||
private static void testDecimalTypes(Connection conn) throws SQLException {
|
||||
System.out.println("### testDecimalTypes");
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_decimal_types").executeQuery();
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM decimal_types").executeQuery();
|
||||
int rowNum = 1;
|
||||
while (rs.next()) {
|
||||
System.out.printf("Row #%d\n", rowNum++);
|
||||
@ -127,7 +132,7 @@ public class PreparedStatementsTest {
|
||||
|
||||
private static void testDateTypes(Connection conn) throws SQLException {
|
||||
System.out.println("### testDateTypes");
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_date_types").executeQuery();
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM date_types").executeQuery();
|
||||
int rowNum = 1;
|
||||
while (rs.next()) {
|
||||
System.out.printf("Row #%d\n", rowNum++);
|
||||
@ -143,7 +148,7 @@ public class PreparedStatementsTest {
|
||||
|
||||
private static void testUnusualDateTime64Scales(Connection conn) throws SQLException {
|
||||
System.out.println("### testUnusualDateTime64Scales");
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_unusual_datetime64_scales").executeQuery();
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM unusual_datetime64_scales").executeQuery();
|
||||
int rowNum = 1;
|
||||
while (rs.next()) {
|
||||
System.out.printf("Row #%d\n", rowNum++);
|
||||
@ -160,7 +165,7 @@ public class PreparedStatementsTest {
|
||||
|
||||
private static void testDateTimeTimezones(Connection conn) throws SQLException {
|
||||
System.out.println("### testDateTimeTimezones");
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_datetime_timezones").executeQuery();
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM datetime_timezones").executeQuery();
|
||||
int rowNum = 1;
|
||||
while (rs.next()) {
|
||||
System.out.printf("Row #%d\n", rowNum++);
|
||||
@ -172,7 +177,7 @@ public class PreparedStatementsTest {
|
||||
|
||||
private static void testMiscTypes(Connection conn) throws SQLException {
|
||||
System.out.println("### testMiscTypes");
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_misc_types").executeQuery();
|
||||
ResultSet rs = conn.prepareStatement("SELECT * FROM misc_types").executeQuery();
|
||||
int rowNum = 1;
|
||||
while (rs.next()) {
|
||||
System.out.printf("Row #%d\n", rowNum++);
|
||||
@ -184,6 +189,20 @@ public class PreparedStatementsTest {
|
||||
System.out.println();
|
||||
}
|
||||
|
||||
private static void testSuspiciousNullableLowCardinalityTypes(Connection conn) throws SQLException {
|
||||
System.out.println("### testSuspiciousNullableLowCardinalityTypes");
|
||||
String query = "SELECT * FROM suspicious_nullable_low_cardinality_types";
|
||||
ResultSet rs = conn.prepareStatement(query).executeQuery();
|
||||
int rowNum = 1;
|
||||
while (rs.next()) {
|
||||
System.out.printf("Row #%d\n", rowNum++);
|
||||
System.out.printf("%s, value: %s\n", getMysqlType(rs, "f"), rs.getFloat("f"));
|
||||
System.out.printf("%s, value: %s\n", getMysqlType(rs, "d"), rs.getDate("d"));
|
||||
System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt"), rs.getTimestamp("dt"));
|
||||
}
|
||||
System.out.println();
|
||||
}
|
||||
|
||||
private static String getMysqlType(ResultSet rs, String columnLabel) throws SQLException {
|
||||
ResultSetMetaData meta = rs.getMetaData();
|
||||
return String.format("%s type is %s", columnLabel,
|
@ -1,78 +0,0 @@
|
||||
import java.sql.Connection;
|
||||
import java.sql.DriverManager;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
|
||||
class JavaConnectorTest {
|
||||
private static final String CREATE_TABLE_SQL = "CREATE TABLE IF NOT EXISTS default.test1 (`age` Int32, `name` String, `int_nullable` Nullable(Int32)) Engine = Memory";
|
||||
private static final String INSERT_SQL = "INSERT INTO default.test1(`age`, `name`) VALUES(33, 'jdbc'),(44, 'ck')";
|
||||
private static final String SELECT_SQL = "SELECT * FROM default.test1";
|
||||
private static final String SELECT_NUMBER_SQL = "SELECT * FROM system.numbers LIMIT 13";
|
||||
private static final String DROP_TABLE_SQL = "DROP TABLE default.test1";
|
||||
|
||||
public static void main(String[] args) {
|
||||
int i = 0;
|
||||
String host = "127.0.0.1";
|
||||
String port = "9004";
|
||||
String user = "default";
|
||||
String password = "";
|
||||
String database = "default";
|
||||
while (i < args.length) {
|
||||
switch (args[i]) {
|
||||
case "--host":
|
||||
host = args[++i];
|
||||
break;
|
||||
case "--port":
|
||||
port = args[++i];
|
||||
break;
|
||||
case "--user":
|
||||
user = args[++i];
|
||||
break;
|
||||
case "--password":
|
||||
password = args[++i];
|
||||
break;
|
||||
case "--database":
|
||||
database = args[++i];
|
||||
break;
|
||||
default:
|
||||
i++;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false", host, port, database);
|
||||
|
||||
Connection conn = null;
|
||||
Statement stmt = null;
|
||||
try {
|
||||
Class.forName("com.mysql.cj.jdbc.Driver");
|
||||
conn = DriverManager.getConnection(jdbcUrl, user, password);
|
||||
stmt = conn.createStatement();
|
||||
stmt.executeUpdate(CREATE_TABLE_SQL);
|
||||
stmt.executeUpdate(INSERT_SQL);
|
||||
|
||||
ResultSet rs = stmt.executeQuery(SELECT_SQL);
|
||||
while (rs.next()) {
|
||||
System.out.print(rs.getString("age"));
|
||||
System.out.print(rs.getString("name"));
|
||||
System.out.print(rs.getString("int_nullable"));
|
||||
System.out.println();
|
||||
}
|
||||
|
||||
stmt.executeUpdate(DROP_TABLE_SQL);
|
||||
|
||||
rs = stmt.executeQuery(SELECT_NUMBER_SQL);
|
||||
while (rs.next()) {
|
||||
System.out.print(rs.getString(1));
|
||||
System.out.println();
|
||||
}
|
||||
|
||||
stmt.close();
|
||||
conn.close();
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
System.exit(1);
|
||||
}
|
||||
}
|
||||
}
|
@ -1924,6 +1924,14 @@ SELECT * FROM test.hits format Protobuf SETTINGS format_protobuf_use_autogenerat
|
||||
|
||||
In this case autogenerated Protobuf schema will be saved in file `path/to/schema/schema.capnp`.
|
||||
|
||||
### Drop Protobuf cache
|
||||
|
||||
To reload Protobuf schema loaded from [format_schema_path](../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-format_schema_path) use [SYSTEM DROP ... FORMAT CACHE](../sql-reference/statements/system.md/#system-drop-schema-format) statement.
|
||||
|
||||
```sql
|
||||
SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf
|
||||
```
|
||||
|
||||
## ProtobufSingle {#protobufsingle}
|
||||
|
||||
Same as [Protobuf](#protobuf) but for storing/parsing single Protobuf message without length delimiters.
|
||||
|
@ -119,6 +119,18 @@ The compiled expression cache is enabled/disabled with the query/user/profile-le
|
||||
|
||||
Clears the [query cache](../../operations/query-cache.md).
|
||||
|
||||
## DROP FORMAT SCHEMA CACHE {#system-drop-schema-format}
|
||||
|
||||
Clears cache for schemas loaded from [format_schema_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path).
|
||||
|
||||
Supported formats:
|
||||
|
||||
- Protobuf
|
||||
|
||||
```sql
|
||||
SYSTEM DROP FORMAT SCHEMA CACHE [FOR Protobuf]
|
||||
```
|
||||
|
||||
## FLUSH LOGS
|
||||
|
||||
Flushes buffered log messages to system tables, e.g. system.query_log. Mainly useful for debugging since most system tables have a default flush interval of 7.5 seconds.
|
||||
|
@ -706,6 +706,17 @@ bool Client::processWithFuzzing(const String & full_query)
|
||||
return true;
|
||||
}
|
||||
|
||||
// Kusto is not a subject for fuzzing (yet)
|
||||
if (global_context->getSettingsRef().dialect == DB::Dialect::kusto)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
if (auto *q = orig_ast->as<ASTSetQuery>())
|
||||
{
|
||||
if (auto *setDialect = q->changes.tryGet("dialect"); setDialect && setDialect->safeGet<String>() == "kusto")
|
||||
return true;
|
||||
}
|
||||
|
||||
// Don't repeat:
|
||||
// - INSERT -- Because the tables may grow too big.
|
||||
// - CREATE -- Because first we run the unmodified query, it will succeed,
|
||||
|
@ -155,6 +155,7 @@ enum class AccessType
|
||||
M(SYSTEM_DROP_FILESYSTEM_CACHE, "SYSTEM DROP FILESYSTEM CACHE, DROP FILESYSTEM CACHE", GLOBAL, SYSTEM_DROP_CACHE) \
|
||||
M(SYSTEM_SYNC_FILESYSTEM_CACHE, "SYSTEM REPAIR FILESYSTEM CACHE, REPAIR FILESYSTEM CACHE, SYNC FILESYSTEM CACHE", GLOBAL, SYSTEM) \
|
||||
M(SYSTEM_DROP_SCHEMA_CACHE, "SYSTEM DROP SCHEMA CACHE, DROP SCHEMA CACHE", GLOBAL, SYSTEM_DROP_CACHE) \
|
||||
M(SYSTEM_DROP_FORMAT_SCHEMA_CACHE, "SYSTEM DROP FORMAT SCHEMA CACHE, DROP FORMAT SCHEMA CACHE", GLOBAL, SYSTEM_DROP_CACHE) \
|
||||
M(SYSTEM_DROP_S3_CLIENT_CACHE, "SYSTEM DROP S3 CLIENT, DROP S3 CLIENT CACHE", GLOBAL, SYSTEM_DROP_CACHE) \
|
||||
M(SYSTEM_DROP_CACHE, "DROP CACHE", GROUP, SYSTEM) \
|
||||
M(SYSTEM_RELOAD_CONFIG, "RELOAD CONFIG", GLOBAL, SYSTEM_RELOAD) \
|
||||
|
@ -591,8 +591,8 @@ if (ENABLE_TESTS)
|
||||
)
|
||||
|
||||
target_link_libraries(unit_tests_dbms PRIVATE
|
||||
ch_contrib::gtest_all
|
||||
ch_contrib::gmock_all
|
||||
ch_contrib::gtest_all
|
||||
clickhouse_functions
|
||||
clickhouse_aggregate_functions
|
||||
clickhouse_parsers
|
||||
|
@ -47,6 +47,7 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/PRQL/ParserPRQLQuery.h>
|
||||
#include <Parsers/Kusto/parseKQLQuery.h>
|
||||
|
||||
#include <Processors/Formats/Impl/NullFormat.h>
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
@ -349,7 +350,10 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu
|
||||
if (is_interactive || ignore_error)
|
||||
{
|
||||
String message;
|
||||
res = tryParseQuery(*parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth);
|
||||
if (dialect == Dialect::kusto)
|
||||
res = tryParseKQLQuery(*parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth);
|
||||
else
|
||||
res = tryParseQuery(*parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth);
|
||||
|
||||
if (!res)
|
||||
{
|
||||
@ -359,7 +363,10 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu
|
||||
}
|
||||
else
|
||||
{
|
||||
res = parseQueryAndMovePosition(*parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth);
|
||||
if (dialect == Dialect::kusto)
|
||||
res = parseKQLQueryAndMovePosition(*parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth);
|
||||
else
|
||||
res = parseQueryAndMovePosition(*parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth);
|
||||
}
|
||||
|
||||
if (is_interactive)
|
||||
|
65
src/Core/MySQL/MySQLUtils.cpp
Normal file
65
src/Core/MySQL/MySQLUtils.cpp
Normal file
@ -0,0 +1,65 @@
|
||||
#include "Common/assert_cast.h"
|
||||
#include "Columns/ColumnNullable.h"
|
||||
#include "Columns/ColumnsDateTime.h"
|
||||
#include "Core/DecimalFunctions.h"
|
||||
#include "DataTypes/IDataType.h"
|
||||
#include "base/types.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace MySQLProtocol
|
||||
{
|
||||
namespace MySQLUtils
|
||||
{
|
||||
|
||||
DecimalUtils::DecimalComponents<DateTime64>
|
||||
getNormalizedDateTime64Components(const DataTypePtr data_type, const ColumnPtr col, const int row_num)
|
||||
{
|
||||
const auto * date_time_type = typeid_cast<const DataTypeDateTime64 *>(data_type.get());
|
||||
|
||||
static constexpr UInt32 MaxScale = DecimalUtils::max_precision<DateTime64>;
|
||||
UInt32 scale = std::min(MaxScale, date_time_type->getScale());
|
||||
|
||||
const auto value = assert_cast<const ColumnDateTime64 &>(*col).getData()[row_num];
|
||||
auto components = DecimalUtils::split(value, scale);
|
||||
|
||||
using T = typename DateTime64::NativeType;
|
||||
if (value.value < 0 && components.fractional)
|
||||
{
|
||||
components.fractional = DecimalUtils::scaleMultiplier<T>(scale) + (components.whole ? T(-1) : T(1)) * components.fractional;
|
||||
--components.whole;
|
||||
}
|
||||
|
||||
if (components.fractional != 0)
|
||||
{
|
||||
if (scale > 6)
|
||||
{
|
||||
// MySQL Timestamp has max scale of 6
|
||||
components.fractional /= static_cast<int>(pow(10, scale - 6));
|
||||
}
|
||||
else
|
||||
{
|
||||
// fractional == 1 is a different microsecond value depending on the scale
|
||||
// Scale 1 = 100000
|
||||
// Scale 2 = 010000
|
||||
// Scale 3 = 001000
|
||||
// Scale 4 = 000100
|
||||
// Scale 5 = 000010
|
||||
// Scale 6 = 000001
|
||||
components.fractional *= static_cast<int>(pow(10, 6 - scale));
|
||||
}
|
||||
}
|
||||
|
||||
return components;
|
||||
};
|
||||
|
||||
ColumnPtr getBaseColumn(const DB::Columns & columns, size_t i)
|
||||
{
|
||||
ColumnPtr col = columns[i]->convertToFullIfNeeded();
|
||||
if (col->isNullable())
|
||||
return assert_cast<const ColumnNullable &>(*col).getNestedColumnPtr();
|
||||
return col;
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
22
src/Core/MySQL/MySQLUtils.h
Normal file
22
src/Core/MySQL/MySQLUtils.h
Normal file
@ -0,0 +1,22 @@
|
||||
#pragma once
|
||||
|
||||
#include "Core/DecimalFunctions.h"
|
||||
#include "DataTypes/IDataType.h"
|
||||
#include "base/types.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace MySQLProtocol
|
||||
{
|
||||
namespace MySQLUtils
|
||||
{
|
||||
/// Splits DateTime64 column data at a certain row number into whole and fractional part
|
||||
/// Additionally, normalizes the fractional part as if it was scale 6 for MySQL compatibility purposes
|
||||
DecimalUtils::DecimalComponents<DateTime64> getNormalizedDateTime64Components(DataTypePtr data_type, ColumnPtr col, int row_num);
|
||||
|
||||
/// If a column is ColumnSparse/ColumnLowCardinality/ColumnNullable, it is unwrapped in a correct order;
|
||||
/// otherwise, the original column is returned
|
||||
ColumnPtr getBaseColumn(const DB::Columns & columns, size_t i);
|
||||
}
|
||||
}
|
||||
}
|
@ -15,6 +15,7 @@
|
||||
#include "DataTypes/DataTypesNumber.h"
|
||||
#include "Formats/FormatSettings.h"
|
||||
#include "IO/WriteBufferFromString.h"
|
||||
#include "MySQLUtils.h"
|
||||
#include "base/DayNum.h"
|
||||
#include "base/Decimal.h"
|
||||
#include "base/types.h"
|
||||
@ -32,7 +33,7 @@ ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTyp
|
||||
FormatSettings format_settings;
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
ColumnPtr col = getColumn(i);
|
||||
ColumnPtr col = MySQLUtils::getBaseColumn(columns, i);
|
||||
if (col->isNullAt(row_num))
|
||||
{
|
||||
// See https://dev.mysql.com/doc/dev/mysql-server/8.1.0/page_protocol_binary_resultset.html#sect_protocol_binary_resultset_row
|
||||
@ -42,7 +43,7 @@ ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTyp
|
||||
continue; // NULLs are stored in the null bitmap only
|
||||
}
|
||||
|
||||
DataTypePtr data_type = removeLowCardinality(removeNullable((data_types[i])));
|
||||
DataTypePtr data_type = removeLowCardinalityAndNullable(data_types[i]);
|
||||
TypeIndex type_index = data_type->getTypeId();
|
||||
switch (type_index)
|
||||
{
|
||||
@ -50,10 +51,7 @@ ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTyp
|
||||
payload_size += 1;
|
||||
break;
|
||||
case TypeIndex::UInt8:
|
||||
if (data_type->getName() == "Bool")
|
||||
payload_size += 2; // BIT MySQL type is string<lenenc> in binary
|
||||
else
|
||||
payload_size += 1;
|
||||
payload_size += 1;
|
||||
break;
|
||||
case TypeIndex::Int16:
|
||||
case TypeIndex::UInt16:
|
||||
@ -98,13 +96,7 @@ ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTyp
|
||||
break;
|
||||
}
|
||||
case TypeIndex::DateTime64: {
|
||||
auto [components, scale] = getDateTime64ComponentsWithScale(data_type, col);
|
||||
if (scale > 6)
|
||||
{
|
||||
// MySQL Timestamp has max scale of 6
|
||||
components.fractional /= static_cast<int>(pow(10, scale - 6));
|
||||
}
|
||||
|
||||
auto components = MySQLUtils::getNormalizedDateTime64Components(data_type, col, row_num);
|
||||
LocalDateTime ldt = LocalDateTime(components.whole, DateLUT::instance(getDateTimeTimezone(*data_type)));
|
||||
|
||||
bool has_microseconds = components.fractional != 0;
|
||||
@ -153,11 +145,11 @@ void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const
|
||||
buffer.write(null_bitmap.data(), null_bitmap_size);
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
ColumnPtr col = getColumn(i);
|
||||
ColumnPtr col = MySQLUtils::getBaseColumn(columns, i);
|
||||
if (col->isNullAt(row_num))
|
||||
continue;
|
||||
|
||||
DataTypePtr data_type = removeLowCardinality(removeNullable((data_types[i])));
|
||||
DataTypePtr data_type = removeLowCardinalityAndNullable(data_types[i]);
|
||||
TypeIndex type_index = data_type->getTypeId();
|
||||
switch (type_index)
|
||||
{
|
||||
@ -168,8 +160,6 @@ void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const
|
||||
}
|
||||
case TypeIndex::UInt8: {
|
||||
UInt8 value = assert_cast<const ColumnVector<UInt8> &>(*col).getData()[row_num];
|
||||
if (data_type->getName() == "Bool")
|
||||
buffer.write(static_cast<char>(1));
|
||||
buffer.write(reinterpret_cast<char *>(&value), 1);
|
||||
break;
|
||||
}
|
||||
@ -262,27 +252,7 @@ void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const
|
||||
break;
|
||||
}
|
||||
case TypeIndex::DateTime64: {
|
||||
auto [components, scale] = getDateTime64ComponentsWithScale(data_type, col);
|
||||
if (components.fractional != 0)
|
||||
{
|
||||
if (scale > 6)
|
||||
{
|
||||
// MySQL Timestamp has max scale of 6
|
||||
components.fractional /= static_cast<int>(pow(10, scale - 6));
|
||||
}
|
||||
else
|
||||
{
|
||||
// fractional == 1 is a different microsecond value depending on the scale
|
||||
// Scale 1 = 100000
|
||||
// Scale 2 = 010000
|
||||
// Scale 3 = 001000
|
||||
// Scale 4 = 000100
|
||||
// Scale 5 = 000010
|
||||
// Scale 6 = 000001
|
||||
components.fractional *= static_cast<int>(pow(10, 6 - scale));
|
||||
}
|
||||
}
|
||||
|
||||
auto components = MySQLUtils::getNormalizedDateTime64Components(data_type, col, row_num);
|
||||
String timezone = getDateTimeTimezone(*data_type);
|
||||
LocalDateTime ldt = LocalDateTime(components.whole, DateLUT::instance(timezone));
|
||||
UInt16 year = ldt.year();
|
||||
@ -332,34 +302,6 @@ void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ResultSetRow::DateTime64ComponentsWithScale ResultSetRow::getDateTime64ComponentsWithScale(DataTypePtr data_type, ColumnPtr col) const
|
||||
{
|
||||
const auto * date_time_type = typeid_cast<const DataTypeDateTime64 *>(data_type.get());
|
||||
|
||||
static constexpr UInt32 MaxScale = DecimalUtils::max_precision<DateTime64>;
|
||||
UInt32 scale = std::min(MaxScale, date_time_type->getScale());
|
||||
|
||||
const auto value = assert_cast<const ColumnDateTime64 &>(*col).getData()[row_num];
|
||||
auto components = DecimalUtils::split(value, scale);
|
||||
|
||||
using T = typename DateTime64::NativeType;
|
||||
if (value.value < 0 && components.fractional)
|
||||
{
|
||||
components.fractional = DecimalUtils::scaleMultiplier<T>(scale) + (components.whole ? T(-1) : T(1)) * components.fractional;
|
||||
--components.whole;
|
||||
}
|
||||
|
||||
return {components, scale};
|
||||
}
|
||||
|
||||
ColumnPtr ResultSetRow::getColumn(size_t i) const
|
||||
{
|
||||
ColumnPtr col = columns[i]->convertToFullIfNeeded();
|
||||
if (col->isNullable())
|
||||
return assert_cast<const ColumnNullable &>(*col).getNestedColumnPtr();
|
||||
return col;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -16,12 +16,6 @@ namespace ProtocolBinary
|
||||
{
|
||||
class ResultSetRow : public IMySQLWritePacket
|
||||
{
|
||||
using DateTime64ComponentsWithScale = std::pair<DecimalUtils::DecimalComponents<DateTime64>, UInt32>;
|
||||
|
||||
private:
|
||||
DateTime64ComponentsWithScale getDateTime64ComponentsWithScale(DataTypePtr data_type, ColumnPtr col) const;
|
||||
ColumnPtr getColumn(size_t i) const;
|
||||
|
||||
protected:
|
||||
int row_num;
|
||||
const Columns & columns;
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include "DataTypes/DataTypeLowCardinality.h"
|
||||
#include "DataTypes/DataTypeNullable.h"
|
||||
#include "DataTypes/DataTypesDecimal.h"
|
||||
#include "MySQLUtils.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -17,20 +18,36 @@ namespace MySQLProtocol
|
||||
namespace ProtocolText
|
||||
{
|
||||
|
||||
ResultSetRow::ResultSetRow(const Serializations & serializations, const Columns & columns_, int row_num_)
|
||||
ResultSetRow::ResultSetRow(const Serializations & serializations, const DataTypes & data_types, const Columns & columns_, int row_num_)
|
||||
: columns(columns_), row_num(row_num_)
|
||||
{
|
||||
static FormatSettings format_settings = {.bool_true_representation = "1", .bool_false_representation = "0"};
|
||||
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
DataTypePtr data_type = removeLowCardinalityAndNullable(data_types[i]);
|
||||
TypeIndex type_index = data_type->getTypeId();
|
||||
if (columns[i]->isNullAt(row_num))
|
||||
{
|
||||
payload_size += 1;
|
||||
serialized.emplace_back("\xfb");
|
||||
}
|
||||
// Arbitrary precision DateTime64 needs to be forced into precision 6, as it is the maximum that MySQL supports
|
||||
else if (type_index == TypeIndex::DateTime64)
|
||||
{
|
||||
WriteBufferFromOwnString ostr;
|
||||
ColumnPtr col = MySQLUtils::getBaseColumn(columns, i);
|
||||
auto components = MySQLUtils::getNormalizedDateTime64Components(data_type, col, row_num);
|
||||
writeDateTimeText<'-', ':', ' '>(LocalDateTime(components.whole, DateLUT::instance(getDateTimeTimezone(*data_type))), ostr);
|
||||
ostr.write('.');
|
||||
writeDateTime64FractionalText<DateTime64>(components.fractional, 6, ostr);
|
||||
payload_size += getLengthEncodedStringSize(ostr.str());
|
||||
serialized.push_back(std::move(ostr.str()));
|
||||
}
|
||||
else
|
||||
{
|
||||
WriteBufferFromOwnString ostr;
|
||||
serializations[i]->serializeText(*columns[i], row_num, ostr, FormatSettings());
|
||||
serializations[i]->serializeText(*columns[i], row_num, ostr, format_settings);
|
||||
payload_size += getLengthEncodedStringSize(ostr.str());
|
||||
serialized.push_back(std::move(ostr.str()));
|
||||
}
|
||||
@ -45,12 +62,10 @@ size_t ResultSetRow::getPayloadSize() const
|
||||
void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const
|
||||
{
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
if (columns[i]->isNullAt(row_num))
|
||||
buffer.write(serialized[i].data(), 1);
|
||||
else
|
||||
writeLengthEncodedString(serialized[i], buffer);
|
||||
}
|
||||
}
|
||||
|
||||
void ComFieldList::readPayloadImpl(ReadBuffer & payload)
|
||||
@ -142,19 +157,13 @@ ColumnDefinition getColumnDefinition(const String & column_name, const DataTypeP
|
||||
CharacterSet charset = CharacterSet::binary;
|
||||
int flags = 0;
|
||||
uint8_t decimals = 0;
|
||||
TypeIndex type_index = removeLowCardinality(removeNullable(data_type))->getTypeId();
|
||||
DataTypePtr normalized_data_type = removeLowCardinalityAndNullable(data_type);
|
||||
TypeIndex type_index = normalized_data_type->getTypeId();
|
||||
switch (type_index)
|
||||
{
|
||||
case TypeIndex::UInt8:
|
||||
if (data_type->getName() == "Bool")
|
||||
{
|
||||
column_type = ColumnType::MYSQL_TYPE_BIT;
|
||||
}
|
||||
else
|
||||
{
|
||||
column_type = ColumnType::MYSQL_TYPE_TINY;
|
||||
flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG;
|
||||
}
|
||||
column_type = ColumnType::MYSQL_TYPE_TINY;
|
||||
flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG;
|
||||
break;
|
||||
case TypeIndex::UInt16:
|
||||
column_type = ColumnType::MYSQL_TYPE_SHORT;
|
||||
@ -213,7 +222,7 @@ ColumnDefinition getColumnDefinition(const String & column_name, const DataTypeP
|
||||
// MySQL Decimal has max 65 precision and 30 scale
|
||||
// Decimal256 (min scale is 39) is higher than the MySQL supported range and handled in the default case
|
||||
// See https://dev.mysql.com/doc/refman/8.0/en/precision-math-decimal-characteristics.html
|
||||
const auto & type = assert_cast<const DataTypeDecimal128 &>(*data_type);
|
||||
const auto & type = assert_cast<const DataTypeDecimal128 &>(*normalized_data_type);
|
||||
if (type.getPrecision() > 65 || type.getScale() > 30)
|
||||
{
|
||||
column_type = ColumnType::MYSQL_TYPE_STRING;
|
||||
|
@ -76,7 +76,7 @@ protected:
|
||||
void writePayloadImpl(WriteBuffer & buffer) const override;
|
||||
|
||||
public:
|
||||
ResultSetRow(const Serializations & serializations, const Columns & columns_, int row_num_);
|
||||
ResultSetRow(const Serializations & serializations, const DataTypes & data_types, const Columns & columns_, int row_num_);
|
||||
};
|
||||
|
||||
class ComFieldList : public LimitedReadPacket
|
||||
|
@ -680,7 +680,6 @@ class IColumn;
|
||||
M(Bool, query_plan_aggregation_in_order, true, "Use query plan for aggregation-in-order optimisation", 0) \
|
||||
M(Bool, query_plan_remove_redundant_sorting, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries", 0) \
|
||||
M(Bool, query_plan_remove_redundant_distinct, true, "Remove redundant Distinct step in query plan", 0) \
|
||||
M(Bool, query_plan_optimize_projection, true, "Use query plan for aggregation-in-order optimisation", 0) \
|
||||
M(UInt64, regexp_max_matches_per_row, 1000, "Max matches of any single regexp per row, used to safeguard 'extractAllGroupsHorizontal' against consuming too much memory with greedy RE.", 0) \
|
||||
\
|
||||
M(UInt64, limit, 0, "Limit on read rows from the most 'end' result for select query, default 0 means no limit length", 0) \
|
||||
@ -866,6 +865,7 @@ class IColumn;
|
||||
MAKE_OBSOLETE(M, UInt64, backup_threads, 16) \
|
||||
MAKE_OBSOLETE(M, UInt64, restore_threads, 16) \
|
||||
MAKE_OBSOLETE(M, Bool, optimize_duplicate_order_by_and_distinct, false) \
|
||||
MAKE_OBSOLETE(M, Bool, query_plan_optimize_projection, true) \
|
||||
|
||||
/** The section above is for obsolete settings. Do not add anything there. */
|
||||
|
||||
|
@ -140,7 +140,7 @@ IMPLEMENT_SETTING_ENUM(Dialect, ErrorCodes::BAD_ARGUMENTS,
|
||||
{{"clickhouse", Dialect::clickhouse},
|
||||
{"kusto", Dialect::kusto},
|
||||
{"prql", Dialect::prql}})
|
||||
// FIXME: do not add 'kusto_auto' to the list. Maybe remove it from code completely?
|
||||
|
||||
|
||||
IMPLEMENT_SETTING_ENUM(ParallelReplicasCustomKeyFilterType, ErrorCodes::BAD_ARGUMENTS,
|
||||
{{"default", ParallelReplicasCustomKeyFilterType::DEFAULT},
|
||||
|
@ -207,7 +207,6 @@ enum class Dialect
|
||||
{
|
||||
clickhouse,
|
||||
kusto,
|
||||
kusto_auto,
|
||||
prql,
|
||||
};
|
||||
|
||||
|
@ -176,4 +176,8 @@ DataTypePtr removeLowCardinality(const DataTypePtr & type)
|
||||
return type;
|
||||
}
|
||||
|
||||
DataTypePtr removeLowCardinalityAndNullable(const DataTypePtr & type)
|
||||
{
|
||||
return removeNullable(removeLowCardinality(type));
|
||||
};
|
||||
}
|
||||
|
@ -91,4 +91,7 @@ ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column);
|
||||
/// Convert column of type from_type to type to_type by converting nested LowCardinality columns.
|
||||
ColumnPtr recursiveLowCardinalityTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type);
|
||||
|
||||
/// Removes LowCardinality and Nullable in a correct order and returns T
|
||||
/// if the type is LowCardinality(T) or LowCardinality(Nullable(T)); type otherwise
|
||||
DataTypePtr removeLowCardinalityAndNullable(const DataTypePtr & type);
|
||||
}
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <atomic>
|
||||
#include <IO/ReadBufferFromFileBase.h>
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
@ -33,7 +34,7 @@ public:
|
||||
|
||||
void setReadUntilPosition(size_t position) override;
|
||||
|
||||
size_t getFileOffsetOfBufferEnd() const override { return offset; }
|
||||
size_t getFileOffsetOfBufferEnd() const override { return offset.load(std::memory_order_relaxed); }
|
||||
|
||||
bool supportsRightBoundedReads() const override { return true; }
|
||||
|
||||
@ -54,7 +55,10 @@ private:
|
||||
|
||||
bool use_external_buffer;
|
||||
|
||||
off_t offset = 0;
|
||||
/// atomic is required for CachedOnDiskReadBufferFromFile, which can access
|
||||
/// to this variable via getFileOffsetOfBufferEnd()/seek() from multiple
|
||||
/// threads.
|
||||
std::atomic<off_t> offset = 0;
|
||||
off_t read_until_position = 0;
|
||||
};
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <ranges>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Disks/WriteMode.h>
|
||||
#include <base/defines.h>
|
||||
|
||||
#include <Disks/ObjectStorages/MetadataStorageFromDisk.h>
|
||||
@ -682,10 +683,17 @@ std::unique_ptr<WriteBufferFromFileBase> DiskObjectStorageTransaction::writeFile
|
||||
|
||||
if (autocommit)
|
||||
{
|
||||
create_metadata_callback = [tx = shared_from_this(), mode, path, blob_name] (size_t count)
|
||||
create_metadata_callback = [tx = shared_from_this(), mode, path, blob_name](size_t count)
|
||||
{
|
||||
if (mode == WriteMode::Rewrite)
|
||||
{
|
||||
// Otherwise we will produce lost blobs which nobody points to
|
||||
/// WriteOnce storages are not affected by the issue
|
||||
if (!tx->object_storage.isWriteOnce() && tx->metadata_storage.exists(path))
|
||||
tx->object_storage.removeObjectsIfExist(tx->metadata_storage.getStorageObjects(path));
|
||||
|
||||
tx->metadata_transaction->createMetadataFile(path, blob_name, count);
|
||||
}
|
||||
else
|
||||
tx->metadata_transaction->addBlobToMetadata(path, blob_name, count);
|
||||
|
||||
@ -694,7 +702,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskObjectStorageTransaction::writeFile
|
||||
}
|
||||
else
|
||||
{
|
||||
create_metadata_callback = [write_op = write_operation.get(), mode, path, blob_name] (size_t count)
|
||||
create_metadata_callback = [object_storage_tx = shared_from_this(), write_op = write_operation.get(), mode, path, blob_name](size_t count)
|
||||
{
|
||||
/// This callback called in WriteBuffer finalize method -- only there we actually know
|
||||
/// how many bytes were written. We don't control when this finalize method will be called
|
||||
@ -706,15 +714,24 @@ std::unique_ptr<WriteBufferFromFileBase> DiskObjectStorageTransaction::writeFile
|
||||
/// ...
|
||||
/// buf1->finalize() // shouldn't do anything with metadata operations, just memoize what to do
|
||||
/// tx->commit()
|
||||
write_op->setOnExecute([mode, path, blob_name, count](MetadataTransactionPtr tx)
|
||||
write_op->setOnExecute([object_storage_tx, mode, path, blob_name, count](MetadataTransactionPtr tx)
|
||||
{
|
||||
if (mode == WriteMode::Rewrite)
|
||||
{
|
||||
/// Otherwise we will produce lost blobs which nobody points to
|
||||
/// WriteOnce storages are not affected by the issue
|
||||
if (!object_storage_tx->object_storage.isWriteOnce() && object_storage_tx->metadata_storage.exists(path))
|
||||
{
|
||||
object_storage_tx->object_storage.removeObjectsIfExist(
|
||||
object_storage_tx->metadata_storage.getStorageObjects(path));
|
||||
}
|
||||
|
||||
tx->createMetadataFile(path, blob_name, count);
|
||||
}
|
||||
else
|
||||
tx->addBlobToMetadata(path, blob_name, count);
|
||||
});
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
operations_to_execute.emplace_back(std::move(write_operation));
|
||||
@ -767,7 +784,12 @@ void DiskObjectStorageTransaction::writeFileUsingBlobWritingFunction(
|
||||
|
||||
/// Create metadata (see create_metadata_callback in DiskObjectStorageTransaction::writeFile()).
|
||||
if (mode == WriteMode::Rewrite)
|
||||
{
|
||||
if (!object_storage.isWriteOnce() && metadata_storage.exists(path))
|
||||
object_storage.removeObjectsIfExist(metadata_storage.getStorageObjects(path));
|
||||
|
||||
metadata_transaction->createMetadataFile(path, blob_name, object_size);
|
||||
}
|
||||
else
|
||||
metadata_transaction->addBlobToMetadata(path, blob_name, object_size);
|
||||
}
|
||||
|
@ -162,7 +162,22 @@ bool WebObjectStorage::exists(const std::string & path) const
|
||||
|
||||
if (startsWith(it->first, path)
|
||||
|| (it != files.begin() && startsWith(std::prev(it)->first, path)))
|
||||
{
|
||||
shared_lock.unlock();
|
||||
std::unique_lock unique_lock(metadata_mutex);
|
||||
|
||||
/// The code relies on invariant that if this function returned true
|
||||
/// the file exists in files.
|
||||
/// In this case we have a directory which doesn't explicitly exists (like store/xxx/yyy)
|
||||
/// ^^^^^
|
||||
/// Adding it to the files
|
||||
files.emplace(std::make_pair(path, FileData({.type = FileType::Directory})));
|
||||
|
||||
unique_lock.unlock();
|
||||
shared_lock.lock();
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
@ -21,6 +21,12 @@ ProtobufSchemas & ProtobufSchemas::instance()
|
||||
return instance;
|
||||
}
|
||||
|
||||
void ProtobufSchemas::clear()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
importers.clear();
|
||||
}
|
||||
|
||||
class ProtobufSchemas::ImporterWithSourceTree : public google::protobuf::compiler::MultiFileErrorCollector
|
||||
{
|
||||
public:
|
||||
|
@ -54,6 +54,8 @@ public:
|
||||
};
|
||||
|
||||
static ProtobufSchemas & instance();
|
||||
// Clear cached protobuf schemas
|
||||
void clear();
|
||||
|
||||
/// Parses the format schema, then parses the corresponding proto file, and returns the descriptor of the message type.
|
||||
/// The function never returns nullptr, it throws an exception if it cannot load or parse the file.
|
||||
|
@ -106,6 +106,9 @@ list (APPEND OBJECT_LIBS $<TARGET_OBJECTS:clickhouse_functions_url>)
|
||||
add_subdirectory(array)
|
||||
list (APPEND OBJECT_LIBS $<TARGET_OBJECTS:clickhouse_functions_array>)
|
||||
|
||||
add_subdirectory(Kusto)
|
||||
list (APPEND OBJECT_LIBS $<TARGET_OBJECTS:clickhouse_functions_kusto>)
|
||||
|
||||
if (TARGET ch_contrib::datasketches)
|
||||
add_subdirectory(UniqTheta)
|
||||
list (APPEND OBJECT_LIBS $<TARGET_OBJECTS:clickhouse_functions_uniqtheta>)
|
||||
|
8
src/Functions/Kusto/CMakeLists.txt
Normal file
8
src/Functions/Kusto/CMakeLists.txt
Normal file
@ -0,0 +1,8 @@
|
||||
include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake")
|
||||
add_headers_and_sources(clickhouse_functions_kusto .)
|
||||
add_library(clickhouse_functions_kusto OBJECT ${clickhouse_functions_kusto_sources} ${clickhouse_functions_kusto_headers})
|
||||
target_link_libraries(clickhouse_functions_kusto PRIVATE dbms clickhouse_functions_gatherutils)
|
||||
|
||||
if (OMIT_HEAVY_DEBUG_SYMBOLS)
|
||||
target_compile_options(clickhouse_functions_kusto PRIVATE "-g0")
|
||||
endif()
|
264
src/Functions/Kusto/KqlArraySort.cpp
Normal file
264
src/Functions/Kusto/KqlArraySort.cpp
Normal file
@ -0,0 +1,264 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/Kusto/KqlFunctionBase.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
template <typename Name, bool is_desc>
|
||||
class FunctionKqlArraySort : public KqlFunctionBase
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
explicit FunctionKqlArraySort(ContextPtr context_) : context(context_) { }
|
||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionKqlArraySort>(context); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (arguments.empty())
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Function {} needs at least one argument; passed {}.",
|
||||
getName(),
|
||||
arguments.size());
|
||||
|
||||
auto array_count = arguments.size();
|
||||
|
||||
if (!isArray(arguments.at(array_count - 1).type))
|
||||
--array_count;
|
||||
|
||||
DataTypes nested_types;
|
||||
for (size_t index = 0; index < array_count; ++index)
|
||||
{
|
||||
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[index].type.get());
|
||||
if (!array_type)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Argument {} of function {} must be array. Found {} instead.",
|
||||
index + 1,
|
||||
getName(),
|
||||
arguments[0].type->getName());
|
||||
|
||||
nested_types.emplace_back(array_type->getNestedType());
|
||||
}
|
||||
|
||||
DataTypes data_types(array_count);
|
||||
|
||||
for (size_t i = 0; i < array_count; ++i)
|
||||
data_types[i] = std::make_shared<DataTypeArray>(makeNullable(nested_types[i]));
|
||||
|
||||
return std::make_shared<DataTypeTuple>(data_types);
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
size_t array_count = arguments.size();
|
||||
const auto & last_arg = arguments[array_count - 1];
|
||||
|
||||
size_t input_rows_count_local = input_rows_count;
|
||||
|
||||
bool null_last = true;
|
||||
if (!isArray(last_arg.type))
|
||||
{
|
||||
--array_count;
|
||||
null_last = check_condition(last_arg, context, input_rows_count_local);
|
||||
}
|
||||
|
||||
ColumnsWithTypeAndName new_args;
|
||||
ColumnPtr first_array_column;
|
||||
std::unordered_set<size_t> null_indices;
|
||||
DataTypes nested_types;
|
||||
|
||||
String sort_function = is_desc ? "arrayReverseSort" : "arraySort";
|
||||
|
||||
for (size_t i = 0; i < array_count; ++i)
|
||||
{
|
||||
ColumnPtr holder = arguments[i].column->convertToFullColumnIfConst();
|
||||
|
||||
const ColumnArray * column_array = checkAndGetColumn<ColumnArray>(holder.get());
|
||||
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[i].type.get());
|
||||
|
||||
if (!column_array)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Argument {} of function {} must be array. Found column {} instead.",
|
||||
i + 1,
|
||||
getName(),
|
||||
holder->getName());
|
||||
|
||||
nested_types.emplace_back(makeNullable(array_type->getNestedType()));
|
||||
if (i == 0)
|
||||
{
|
||||
first_array_column = holder;
|
||||
new_args.push_back(arguments[i]);
|
||||
}
|
||||
else if (!column_array->hasEqualOffsets(static_cast<const ColumnArray &>(*first_array_column)))
|
||||
{
|
||||
null_indices.insert(i);
|
||||
}
|
||||
else
|
||||
new_args.push_back(arguments[i]);
|
||||
}
|
||||
|
||||
auto zipped
|
||||
= FunctionFactory::instance().get("arrayZip", context)->build(new_args)->execute(new_args, result_type, input_rows_count_local);
|
||||
|
||||
ColumnsWithTypeAndName sort_arg({{zipped, std::make_shared<DataTypeArray>(result_type), "zipped"}});
|
||||
auto sorted_tuple
|
||||
= FunctionFactory::instance().get(sort_function, context)->build(sort_arg)->execute(sort_arg, result_type, input_rows_count_local);
|
||||
|
||||
auto null_type = std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt8>());
|
||||
|
||||
Columns tuple_columns(array_count);
|
||||
size_t sorted_index = 0;
|
||||
for (size_t i = 0; i < array_count; ++i)
|
||||
{
|
||||
if (null_indices.contains(i))
|
||||
{
|
||||
auto fun_array = FunctionFactory::instance().get("array", context);
|
||||
|
||||
DataTypePtr arg_type
|
||||
= std::make_shared<DataTypeArray>(makeNullable(nested_types[i]));
|
||||
|
||||
ColumnsWithTypeAndName null_array_arg({
|
||||
{null_type->createColumnConstWithDefaultValue(input_rows_count_local), null_type, "NULL"},
|
||||
});
|
||||
|
||||
tuple_columns[i] = fun_array->build(null_array_arg)->execute(null_array_arg, arg_type, input_rows_count_local);
|
||||
tuple_columns[i] = tuple_columns[i]->convertToFullColumnIfConst();
|
||||
}
|
||||
else
|
||||
{
|
||||
ColumnsWithTypeAndName untuple_args(
|
||||
{{ColumnWithTypeAndName(sorted_tuple, std::make_shared<DataTypeArray>(result_type), "sorted")},
|
||||
{DataTypeUInt8().createColumnConst(1, toField(UInt8(sorted_index + 1))), std::make_shared<DataTypeUInt8>(), ""}});
|
||||
auto tuple_coulmn = FunctionFactory::instance()
|
||||
.get("tupleElement", context)
|
||||
->build(untuple_args)
|
||||
->execute(untuple_args, result_type, input_rows_count_local);
|
||||
|
||||
auto out_tmp = ColumnArray::create(nested_types[i]->createColumn());
|
||||
|
||||
size_t array_size = tuple_coulmn->size();
|
||||
const auto * arr = checkAndGetColumn<ColumnArray>(tuple_coulmn.get());
|
||||
|
||||
for (size_t j = 0; j < array_size; ++j)
|
||||
{
|
||||
Field arr_field;
|
||||
arr->get(j, arr_field);
|
||||
out_tmp->insert(arr_field);
|
||||
}
|
||||
|
||||
tuple_columns[i] = std::move(out_tmp);
|
||||
|
||||
++sorted_index;
|
||||
}
|
||||
}
|
||||
|
||||
if (!null_last)
|
||||
{
|
||||
Columns adjusted_columns(array_count);
|
||||
|
||||
ColumnWithTypeAndName arg_of_index{nullptr, std::make_shared<DataTypeArray>(nested_types[0]), "array"};
|
||||
arg_of_index.column = tuple_columns[0];
|
||||
|
||||
auto inside_null_type = nested_types[0];
|
||||
ColumnsWithTypeAndName indexof_args({
|
||||
arg_of_index,
|
||||
{inside_null_type->createColumnConstWithDefaultValue(input_rows_count_local), inside_null_type, "NULL"},
|
||||
});
|
||||
|
||||
auto null_index_datetype = std::make_shared<DataTypeUInt64>();
|
||||
|
||||
ColumnWithTypeAndName slice_index{nullptr, null_index_datetype, ""};
|
||||
slice_index.column = FunctionFactory::instance()
|
||||
.get("indexOf", context)
|
||||
->build(indexof_args)
|
||||
->execute(indexof_args, result_type, input_rows_count_local);
|
||||
|
||||
auto null_index_in_array = slice_index.column->get64(0);
|
||||
if (null_index_in_array > 0)
|
||||
{
|
||||
ColumnWithTypeAndName slice_index_len{nullptr, null_index_datetype, ""};
|
||||
slice_index_len.column = DataTypeUInt64().createColumnConst(1, toField(UInt64(null_index_in_array - 1)));
|
||||
|
||||
auto fun_slice = FunctionFactory::instance().get("arraySlice", context);
|
||||
|
||||
for (size_t i = 0; i < array_count; ++i)
|
||||
{
|
||||
if (null_indices.contains(i))
|
||||
{
|
||||
adjusted_columns[i] = std::move(tuple_columns[i]);
|
||||
}
|
||||
else
|
||||
{
|
||||
DataTypePtr arg_type = std::make_shared<DataTypeArray>(nested_types[i]);
|
||||
|
||||
ColumnsWithTypeAndName slice_args_left(
|
||||
{{ColumnWithTypeAndName(tuple_columns[i], arg_type, "array")},
|
||||
{DataTypeUInt8().createColumnConst(1, toField(UInt8(1))), std::make_shared<DataTypeUInt8>(), ""},
|
||||
slice_index_len});
|
||||
|
||||
ColumnsWithTypeAndName slice_args_right(
|
||||
{{ColumnWithTypeAndName(tuple_columns[i], arg_type, "array")}, slice_index});
|
||||
ColumnWithTypeAndName arr_left{
|
||||
fun_slice->build(slice_args_left)->execute(slice_args_left, arg_type, input_rows_count_local), arg_type, ""};
|
||||
ColumnWithTypeAndName arr_right{
|
||||
fun_slice->build(slice_args_right)->execute(slice_args_right, arg_type, input_rows_count_local), arg_type, ""};
|
||||
|
||||
ColumnsWithTypeAndName arr_cancat({arr_right, arr_left});
|
||||
auto out_tmp = FunctionFactory::instance()
|
||||
.get("arrayConcat", context)
|
||||
->build(arr_cancat)
|
||||
->execute(arr_cancat, arg_type, input_rows_count_local);
|
||||
adjusted_columns[i] = std::move(out_tmp);
|
||||
}
|
||||
}
|
||||
return ColumnTuple::create(adjusted_columns);
|
||||
}
|
||||
}
|
||||
return ColumnTuple::create(tuple_columns);
|
||||
}
|
||||
|
||||
private:
|
||||
ContextPtr context;
|
||||
};
|
||||
|
||||
struct NameKqlArraySortAsc
|
||||
{
|
||||
static constexpr auto name = "kql_array_sort_asc";
|
||||
};
|
||||
|
||||
struct NameKqlArraySortDesc
|
||||
{
|
||||
static constexpr auto name = "kql_array_sort_desc";
|
||||
};
|
||||
|
||||
using FunctionKqlArraySortAsc = FunctionKqlArraySort<NameKqlArraySortAsc, false>;
|
||||
using FunctionKqlArraySortDesc = FunctionKqlArraySort<NameKqlArraySortDesc, true>;
|
||||
|
||||
REGISTER_FUNCTION(KqlArraySort)
|
||||
{
|
||||
factory.registerFunction<FunctionKqlArraySortAsc>();
|
||||
factory.registerFunction<FunctionKqlArraySortDesc>();
|
||||
}
|
||||
|
||||
}
|
32
src/Functions/Kusto/KqlFunctionBase.h
Normal file
32
src/Functions/Kusto/KqlFunctionBase.h
Normal file
@ -0,0 +1,32 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include "Functions/array/FunctionArrayMapped.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class KqlFunctionBase : public IFunction
|
||||
{
|
||||
public:
|
||||
static bool check_condition (const ColumnWithTypeAndName & condition, ContextPtr context, size_t input_rows_count)
|
||||
{
|
||||
ColumnsWithTypeAndName if_columns(
|
||||
{
|
||||
condition,
|
||||
{DataTypeUInt8().createColumnConst(1, toField(UInt8(1))), std::make_shared<DataTypeUInt8>(), ""},
|
||||
{DataTypeUInt8().createColumnConst(1, toField(UInt8(2))), std::make_shared<DataTypeUInt8>(), ""}
|
||||
});
|
||||
auto if_res = FunctionFactory::instance().get("if", context)->build(if_columns)->execute(if_columns, std::make_shared<DataTypeUInt8>(), input_rows_count);
|
||||
auto result = if_res->getUInt(0);
|
||||
return (result == 1);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -44,6 +44,10 @@ public:
|
||||
|
||||
virtual String getInfoForLog() { return ""; }
|
||||
|
||||
/// NOTE: This method should be thread-safe against seek(), since it can be
|
||||
/// used in CachedOnDiskReadBufferFromFile from multiple threads (because
|
||||
/// it first releases the buffer, and then do logging, and so other thread
|
||||
/// can already call seek() which will lead to data-race).
|
||||
virtual size_t getFileOffsetOfBufferEnd() const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getFileOffsetOfBufferEnd() not implemented"); }
|
||||
|
||||
/// If true, setReadUntilPosition() guarantees that eof will be reported at the given position.
|
||||
|
@ -863,20 +863,6 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map<const
|
||||
return dag;
|
||||
}
|
||||
|
||||
void ActionsDAG::reorderAggregationKeysForProjection(const std::unordered_map<std::string_view, size_t> & key_names_pos_map)
|
||||
{
|
||||
::sort(outputs.begin(), outputs.end(), [&key_names_pos_map](const Node * lhs, const Node * rhs)
|
||||
{
|
||||
return key_names_pos_map.find(lhs->result_name)->second < key_names_pos_map.find(rhs->result_name)->second;
|
||||
});
|
||||
}
|
||||
|
||||
void ActionsDAG::addAggregatesViaProjection(const Block & aggregates)
|
||||
{
|
||||
for (const auto & aggregate : aggregates)
|
||||
outputs.push_back(&addInput(aggregate));
|
||||
}
|
||||
|
||||
void ActionsDAG::addAliases(const NamesWithAliases & aliases)
|
||||
{
|
||||
std::unordered_map<std::string_view, size_t> names_map;
|
||||
|
@ -245,12 +245,6 @@ public:
|
||||
const std::unordered_map<const Node *, const Node *> & new_inputs,
|
||||
const NodeRawConstPtrs & required_outputs);
|
||||
|
||||
/// Reorder the output nodes using given position mapping.
|
||||
void reorderAggregationKeysForProjection(const std::unordered_map<std::string_view, size_t> & key_names_pos_map);
|
||||
|
||||
/// Add aggregate columns to output nodes from projection
|
||||
void addAggregatesViaProjection(const Block & aggregates);
|
||||
|
||||
bool hasArrayJoin() const;
|
||||
bool hasStatefulFunctions() const;
|
||||
bool trivial() const; /// If actions has no functions or array join.
|
||||
|
@ -96,6 +96,7 @@ namespace ErrorCodes
|
||||
extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY;
|
||||
extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_INDEX;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int UNKNOWN_DATABASE;
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
@ -698,6 +699,8 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
|
||||
for (const auto & index : create.columns_list->indices->children)
|
||||
{
|
||||
IndexDescription index_desc = IndexDescription::getIndexFromAST(index->clone(), properties.columns, getContext());
|
||||
if (properties.indices.has(index_desc.name))
|
||||
throw Exception(ErrorCodes::ILLEGAL_INDEX, "Duplicated index name {} is not allowed. Please use different index names.", backQuoteIfNeed(index_desc.name));
|
||||
const auto & settings = getContext()->getSettingsRef();
|
||||
if (index_desc.type == INVERTED_INDEX_NAME && !settings.allow_experimental_inverted_index)
|
||||
{
|
||||
@ -712,6 +715,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
|
||||
|
||||
properties.indices.push_back(index_desc);
|
||||
}
|
||||
|
||||
if (create.columns_list->projections)
|
||||
for (const auto & projection_ast : create.columns_list->projections->children)
|
||||
{
|
||||
@ -880,7 +884,7 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat
|
||||
{
|
||||
for (const auto & [name, type] : properties.columns.getAllPhysical())
|
||||
{
|
||||
auto basic_type = removeLowCardinality(removeNullable(type));
|
||||
auto basic_type = removeLowCardinalityAndNullable(type);
|
||||
if (const auto * fixed_string = typeid_cast<const DataTypeFixedString *>(basic_type.get()))
|
||||
{
|
||||
if (fixed_string->getN() > MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS)
|
||||
|
@ -123,8 +123,6 @@ public:
|
||||
|
||||
bool supportsTransactions() const override { return true; }
|
||||
|
||||
FilterDAGInfoPtr getAdditionalQueryInfo() const { return additional_filter_info; }
|
||||
|
||||
RowPolicyFilterPtr getRowPolicyFilter() const;
|
||||
|
||||
void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override;
|
||||
|
@ -64,6 +64,10 @@
|
||||
#include <algorithm>
|
||||
#include <unistd.h>
|
||||
|
||||
#if USE_PROTOBUF
|
||||
#include <Formats/ProtobufSchemas.h>
|
||||
#endif
|
||||
|
||||
#if USE_AWS_S3
|
||||
#include <IO/S3/Client.h>
|
||||
#endif
|
||||
@ -462,6 +466,20 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
if (caches_to_drop.contains("AZURE"))
|
||||
StorageAzureBlob::getSchemaCache(getContext()).clear();
|
||||
#endif
|
||||
break;
|
||||
}
|
||||
case Type::DROP_FORMAT_SCHEMA_CACHE:
|
||||
{
|
||||
getContext()->checkAccess(AccessType::SYSTEM_DROP_FORMAT_SCHEMA_CACHE);
|
||||
std::unordered_set<String> caches_to_drop;
|
||||
if (query.schema_cache_format.empty())
|
||||
caches_to_drop = {"Protobuf"};
|
||||
else
|
||||
caches_to_drop = {query.schema_cache_format};
|
||||
#if USE_PROTOBUF
|
||||
if (caches_to_drop.contains("Protobuf"))
|
||||
ProtobufSchemas::instance().clear();
|
||||
#endif
|
||||
break;
|
||||
}
|
||||
@ -1082,6 +1100,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster()
|
||||
case Type::DROP_FILESYSTEM_CACHE:
|
||||
case Type::SYNC_FILESYSTEM_CACHE:
|
||||
case Type::DROP_SCHEMA_CACHE:
|
||||
case Type::DROP_FORMAT_SCHEMA_CACHE:
|
||||
#if USE_AWS_S3
|
||||
case Type::DROP_S3_CLIENT_CACHE:
|
||||
#endif
|
||||
|
@ -77,6 +77,7 @@
|
||||
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/PRQL/ParserPRQLQuery.h>
|
||||
#include <Parsers/Kusto/parseKQLQuery.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
@ -708,7 +709,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
ParserKQLStatement parser(end, settings.allow_settings_after_format_in_insert);
|
||||
|
||||
/// TODO: parser should fail early when max_query_size limit is reached.
|
||||
ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth);
|
||||
ast = parseKQLQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth);
|
||||
}
|
||||
else if (settings.dialect == Dialect::prql && !internal)
|
||||
{
|
||||
|
@ -48,7 +48,7 @@ void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings
|
||||
|
||||
if (!settings.allow_suspicious_fixed_string_types)
|
||||
{
|
||||
auto basic_type = removeLowCardinality(removeNullable(type));
|
||||
auto basic_type = removeLowCardinalityAndNullable(type);
|
||||
if (const auto * fixed_string = typeid_cast<const DataTypeFixedString *>(basic_type.get()))
|
||||
{
|
||||
if (fixed_string->getN() > MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS)
|
||||
|
@ -20,6 +20,8 @@
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Core/QualifiedTableName.h>
|
||||
|
||||
#include <boost/algorithm/string.hpp>
|
||||
|
||||
|
||||
using namespace std::literals;
|
||||
|
||||
@ -80,6 +82,8 @@ namespace
|
||||
|
||||
void markSecretArgument(size_t index, bool argument_is_named = false)
|
||||
{
|
||||
if (index >= arguments->size())
|
||||
return;
|
||||
if (!result.count)
|
||||
{
|
||||
result.start = index;
|
||||
@ -100,7 +104,8 @@ namespace
|
||||
/// mongodb('host:port', 'database', 'collection', 'user', 'password', ...)
|
||||
findMySQLFunctionSecretArguments();
|
||||
}
|
||||
else if ((function.name == "s3") || (function.name == "cosn") || (function.name == "oss"))
|
||||
else if ((function.name == "s3") || (function.name == "cosn") || (function.name == "oss") ||
|
||||
(function.name == "deltaLake") || (function.name == "hudi") || (function.name == "iceberg"))
|
||||
{
|
||||
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...)
|
||||
findS3FunctionSecretArguments(/* is_cluster_function= */ false);
|
||||
@ -150,41 +155,26 @@ namespace
|
||||
return;
|
||||
}
|
||||
|
||||
/// We're going to replace 'aws_secret_access_key' with '[HIDDEN'] for the following signatures:
|
||||
/// We should check other arguments first because we don't need to do any replacement in case of
|
||||
/// s3('url', NOSIGN, 'format' [, 'compression'])
|
||||
/// s3('url', 'format', 'structure' [, 'compression'])
|
||||
if ((url_arg_idx + 3 <= arguments->size()) && (arguments->size() <= url_arg_idx + 4))
|
||||
{
|
||||
String second_arg;
|
||||
if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg))
|
||||
{
|
||||
if (boost::iequals(second_arg, "NOSIGN"))
|
||||
return; /// The argument after 'url' is "NOSIGN".
|
||||
|
||||
if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg))
|
||||
return; /// The argument after 'url' is a format: s3('url', 'format', ...)
|
||||
}
|
||||
}
|
||||
|
||||
/// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures:
|
||||
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...)
|
||||
/// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression')
|
||||
|
||||
/// But we should check the number of arguments first because we don't need to do any replacements in case of
|
||||
/// s3('url' [, 'format']) or s3Cluster('cluster_name', 'url' [, 'format'])
|
||||
if (arguments->size() < url_arg_idx + 3)
|
||||
return;
|
||||
|
||||
if (arguments->size() >= url_arg_idx + 5)
|
||||
{
|
||||
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'structure', ...)
|
||||
markSecretArgument(url_arg_idx + 2);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...)
|
||||
/// We need to distinguish that from s3('url', 'format', 'structure' [, 'compression_method']).
|
||||
/// So we will check whether the argument after 'url' is a format.
|
||||
String format;
|
||||
if (!tryGetStringFromArgument(url_arg_idx + 1, &format, /* allow_identifier= */ false))
|
||||
{
|
||||
/// We couldn't evaluate the argument after 'url' so we don't know whether it is a format or `aws_access_key_id`.
|
||||
/// So it's safer to wipe the next argument just in case.
|
||||
markSecretArgument(url_arg_idx + 2); /// Wipe either `aws_secret_access_key` or `structure`.
|
||||
return;
|
||||
}
|
||||
|
||||
if (KnownFormatNames::instance().exists(format))
|
||||
return; /// The argument after 'url' is a format: s3('url', 'format', ...)
|
||||
|
||||
/// The argument after 'url' is not a format so we do our replacement:
|
||||
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) -> s3('url', 'aws_access_key_id', '[HIDDEN]', ...)
|
||||
markSecretArgument(url_arg_idx + 2);
|
||||
}
|
||||
markSecretArgument(url_arg_idx + 2);
|
||||
}
|
||||
|
||||
bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const
|
||||
@ -351,7 +341,8 @@ namespace
|
||||
/// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...)
|
||||
findMySQLFunctionSecretArguments();
|
||||
}
|
||||
else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS"))
|
||||
else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS") ||
|
||||
(engine_name == "DeltaLake") || (engine_name == "Hudi") || (engine_name == "Iceberg"))
|
||||
{
|
||||
/// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...)
|
||||
findS3TableEngineSecretArguments();
|
||||
@ -381,15 +372,29 @@ namespace
|
||||
return;
|
||||
}
|
||||
|
||||
/// We should check other arguments first because we don't need to do any replacement in case of
|
||||
/// S3('url', NOSIGN, 'format' [, 'compression'])
|
||||
/// S3('url', 'format', 'compression')
|
||||
if ((3 <= arguments->size()) && (arguments->size() <= 4))
|
||||
{
|
||||
String second_arg;
|
||||
if (tryGetStringFromArgument(1, &second_arg))
|
||||
{
|
||||
if (boost::iequals(second_arg, "NOSIGN"))
|
||||
return; /// The argument after 'url' is "NOSIGN".
|
||||
|
||||
if (arguments->size() == 3)
|
||||
{
|
||||
if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg))
|
||||
return; /// The argument after 'url' is a format: S3('url', 'format', ...)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// We replace 'aws_secret_access_key' with '[HIDDEN'] for the following signatures:
|
||||
/// S3('url', 'aws_access_key_id', 'aws_secret_access_key')
|
||||
/// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format')
|
||||
/// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression')
|
||||
|
||||
/// But we should check the number of arguments first because we don't need to do that replacements in case of
|
||||
/// S3('url' [, 'format' [, 'compression']])
|
||||
if (arguments->size() < 4)
|
||||
return;
|
||||
|
||||
markSecretArgument(2);
|
||||
}
|
||||
|
||||
@ -404,6 +409,11 @@ namespace
|
||||
/// PostgreSQL('host:port', 'database', 'user', 'password')
|
||||
findMySQLDatabaseSecretArguments();
|
||||
}
|
||||
else if (engine_name == "S3")
|
||||
{
|
||||
/// S3('url', 'access_key_id', 'secret_access_key')
|
||||
findS3DatabaseSecretArguments();
|
||||
}
|
||||
}
|
||||
|
||||
void findMySQLDatabaseSecretArguments()
|
||||
@ -420,6 +430,20 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
void findS3DatabaseSecretArguments()
|
||||
{
|
||||
if (isNamedCollectionName(0))
|
||||
{
|
||||
/// S3(named_collection, ..., secret_access_key = 'password', ...)
|
||||
findSecretNamedArgument("secret_access_key", 1);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// S3('url', 'access_key_id', 'secret_access_key')
|
||||
markSecretArgument(2);
|
||||
}
|
||||
}
|
||||
|
||||
void findBackupNameSecretArguments()
|
||||
{
|
||||
const String & engine_name = function.name;
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTSystemQuery.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <magic_enum.hpp>
|
||||
@ -86,66 +87,60 @@ void ASTSystemQuery::setTable(const String & name)
|
||||
|
||||
void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SYSTEM ";
|
||||
settings.ostr << typeToString(type) << (settings.hilite ? hilite_none : "");
|
||||
|
||||
auto print_database_table = [&]
|
||||
auto print_identifier = [&](const String & identifier) -> WriteBuffer &
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(identifier)
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
return settings.ostr;
|
||||
};
|
||||
|
||||
auto print_keyword = [&](const auto & keyword) -> WriteBuffer &
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << keyword << (settings.hilite ? hilite_none : "");
|
||||
return settings.ostr;
|
||||
};
|
||||
|
||||
auto print_database_table = [&]() -> WriteBuffer &
|
||||
{
|
||||
settings.ostr << " ";
|
||||
if (database)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getDatabase())
|
||||
<< (settings.hilite ? hilite_none : "") << ".";
|
||||
print_identifier(getDatabase()) << ".";
|
||||
}
|
||||
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getTable())
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
print_identifier(getTable());
|
||||
return settings.ostr;
|
||||
};
|
||||
|
||||
auto print_drop_replica = [&]
|
||||
{
|
||||
settings.ostr << " " << quoteString(replica);
|
||||
if (!shard.empty())
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM SHARD "
|
||||
<< (settings.hilite ? hilite_none : "") << quoteString(shard);
|
||||
}
|
||||
print_keyword(" FROM SHARD ") << quoteString(shard);
|
||||
|
||||
if (table)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM TABLE"
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
print_keyword(" FROM TABLE ");
|
||||
print_database_table();
|
||||
}
|
||||
else if (!replica_zk_path.empty())
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM ZKPATH "
|
||||
<< (settings.hilite ? hilite_none : "") << quoteString(replica_zk_path);
|
||||
print_keyword(" FROM ZKPATH ") << quoteString(replica_zk_path);
|
||||
}
|
||||
else if (database)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM DATABASE "
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getDatabase())
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
print_keyword(" FROM DATABASE ");
|
||||
print_identifier(getDatabase());
|
||||
}
|
||||
};
|
||||
|
||||
auto print_on_volume = [&]
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " ON VOLUME "
|
||||
<< (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(storage_policy)
|
||||
<< (settings.hilite ? hilite_none : "")
|
||||
<< "."
|
||||
<< (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(volume)
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
};
|
||||
|
||||
auto print_identifier = [&](const String & identifier)
|
||||
{
|
||||
settings.ostr << " " << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(identifier)
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
print_keyword(" ON VOLUME ");
|
||||
print_identifier(storage_policy) << ".";
|
||||
print_identifier(volume);
|
||||
};
|
||||
|
||||
print_keyword("SYSTEM") << " ";
|
||||
print_keyword(typeToString(type));
|
||||
if (!cluster.empty())
|
||||
formatOnCluster(settings);
|
||||
|
||||
@ -169,7 +164,10 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|
||||
|| type == Type::START_CLEANUP)
|
||||
{
|
||||
if (table)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
print_database_table();
|
||||
}
|
||||
else if (!volume.empty())
|
||||
print_on_volume();
|
||||
}
|
||||
@ -184,20 +182,35 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|
||||
|| type == Type::RESTART_DISK)
|
||||
{
|
||||
if (table)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
print_database_table();
|
||||
}
|
||||
else if (!target_model.empty())
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
print_identifier(target_model);
|
||||
}
|
||||
else if (!target_function.empty())
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
print_identifier(target_function);
|
||||
}
|
||||
else if (!disk.empty())
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
print_identifier(disk);
|
||||
}
|
||||
|
||||
if (sync_replica_mode != SyncReplicaMode::DEFAULT)
|
||||
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << magic_enum::enum_name(sync_replica_mode)
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
print_keyword(magic_enum::enum_name(sync_replica_mode));
|
||||
}
|
||||
}
|
||||
else if (type == Type::SYNC_DATABASE_REPLICA)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
print_identifier(database->as<ASTIdentifier>()->name());
|
||||
}
|
||||
else if (type == Type::DROP_REPLICA || type == Type::DROP_DATABASE_REPLICA)
|
||||
@ -206,48 +219,61 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|
||||
}
|
||||
else if (type == Type::SUSPEND)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FOR "
|
||||
<< (settings.hilite ? hilite_none : "") << seconds
|
||||
<< (settings.hilite ? hilite_keyword : "") << " SECOND"
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
print_keyword(" FOR ") << seconds;
|
||||
print_keyword(" SECOND");
|
||||
}
|
||||
else if (type == Type::DROP_FORMAT_SCHEMA_CACHE)
|
||||
{
|
||||
if (!schema_cache_format.empty())
|
||||
{
|
||||
print_keyword(" FOR ");
|
||||
print_identifier(schema_cache_format);
|
||||
}
|
||||
}
|
||||
else if (type == Type::DROP_FILESYSTEM_CACHE)
|
||||
{
|
||||
if (!filesystem_cache_name.empty())
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_none : "") << " " << filesystem_cache_name;
|
||||
settings.ostr << ' ';
|
||||
print_identifier(filesystem_cache_name);
|
||||
if (!key_to_drop.empty())
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_none : "") << " KEY " << key_to_drop;
|
||||
print_keyword(" KEY ");
|
||||
print_identifier(key_to_drop);
|
||||
if (offset_to_drop.has_value())
|
||||
settings.ostr << (settings.hilite ? hilite_none : "") << " OFFSET " << offset_to_drop.value();
|
||||
{
|
||||
print_keyword(" OFFSET ");
|
||||
settings.ostr << offset_to_drop.value();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (type == Type::DROP_SCHEMA_CACHE)
|
||||
{
|
||||
if (!schema_cache_storage.empty())
|
||||
{
|
||||
print_keyword(" FOR ");
|
||||
print_identifier(schema_cache_storage);
|
||||
}
|
||||
}
|
||||
else if (type == Type::UNFREEZE)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(backup_name);
|
||||
}
|
||||
else if (type == Type::SYNC_FILE_CACHE)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_none : "");
|
||||
print_keyword(" WITH NAME ");
|
||||
settings.ostr << quoteString(backup_name);
|
||||
}
|
||||
else if (type == Type::START_LISTEN || type == Type::STOP_LISTEN)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " "
|
||||
<< ServerType::serverTypeToString(server_type.type) << (settings.hilite ? hilite_none : "");
|
||||
settings.ostr << ' ';
|
||||
print_keyword(ServerType::serverTypeToString(server_type.type));
|
||||
|
||||
if (server_type.type == ServerType::Type::CUSTOM)
|
||||
{
|
||||
settings.ostr << " " << quoteString(server_type.custom_name);
|
||||
}
|
||||
settings.ostr << ' ' << quoteString(server_type.custom_name);
|
||||
|
||||
bool comma = false;
|
||||
|
||||
if (!server_type.exclude_types.empty())
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "")
|
||||
<< " EXCEPT" << (settings.hilite ? hilite_none : "");
|
||||
print_keyword(" EXCEPT");
|
||||
|
||||
for (auto cur_type : server_type.exclude_types)
|
||||
{
|
||||
@ -255,12 +281,12 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|
||||
continue;
|
||||
|
||||
if (comma)
|
||||
settings.ostr << ",";
|
||||
settings.ostr << ',';
|
||||
else
|
||||
comma = true;
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " "
|
||||
<< ServerType::serverTypeToString(cur_type) << (settings.hilite ? hilite_none : "");
|
||||
settings.ostr << ' ';
|
||||
print_keyword(ServerType::serverTypeToString(cur_type));
|
||||
}
|
||||
|
||||
if (server_type.exclude_types.contains(ServerType::Type::CUSTOM))
|
||||
@ -268,13 +294,12 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|
||||
for (const auto & cur_name : server_type.exclude_custom_names)
|
||||
{
|
||||
if (comma)
|
||||
settings.ostr << ",";
|
||||
settings.ostr << ',';
|
||||
else
|
||||
comma = true;
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " "
|
||||
<< ServerType::serverTypeToString(ServerType::Type::CUSTOM) << (settings.hilite ? hilite_none : "");
|
||||
|
||||
settings.ostr << ' ';
|
||||
print_keyword(ServerType::serverTypeToString(ServerType::Type::CUSTOM));
|
||||
settings.ostr << " " << quoteString(cur_name);
|
||||
}
|
||||
}
|
||||
|
@ -33,6 +33,7 @@ public:
|
||||
#endif
|
||||
DROP_FILESYSTEM_CACHE,
|
||||
DROP_SCHEMA_CACHE,
|
||||
DROP_FORMAT_SCHEMA_CACHE,
|
||||
#if USE_AWS_S3
|
||||
DROP_S3_CLIENT_CACHE,
|
||||
#endif
|
||||
@ -120,6 +121,8 @@ public:
|
||||
|
||||
String schema_cache_storage;
|
||||
|
||||
String schema_cache_format;
|
||||
|
||||
String fail_point_name;
|
||||
|
||||
SyncReplicaMode sync_replica_mode = SyncReplicaMode::DEFAULT;
|
||||
|
@ -5,6 +5,7 @@ add_headers_and_sources(clickhouse_parsers ./Access)
|
||||
add_headers_and_sources(clickhouse_parsers ./MySQL)
|
||||
add_headers_and_sources(clickhouse_parsers ./Kusto)
|
||||
add_headers_and_sources(clickhouse_parsers ./PRQL)
|
||||
add_headers_and_sources(clickhouse_parsers ./Kusto/KustoFunctions)
|
||||
add_library(clickhouse_parsers ${clickhouse_parsers_headers} ${clickhouse_parsers_sources})
|
||||
target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io clickhouse_common_access string_utils)
|
||||
if (TARGET ch_rust::prql)
|
||||
|
@ -43,7 +43,6 @@
|
||||
|
||||
#include <Interpreters/StorageID.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -27,6 +27,7 @@
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
|
||||
using namespace std::literals;
|
||||
|
||||
@ -662,6 +663,26 @@ public:
|
||||
}
|
||||
else
|
||||
{
|
||||
/// enable using subscript operator for kql_array_sort
|
||||
if (cur_op.function_name == "arrayElement" && !operands.empty())
|
||||
{
|
||||
auto* first_arg_as_node = operands.front()->as<ASTFunction>();
|
||||
if (first_arg_as_node)
|
||||
{
|
||||
if (first_arg_as_node->name == "kql_array_sort_asc" || first_arg_as_node->name == "kql_array_sort_desc")
|
||||
{
|
||||
cur_op.function_name = "tupleElement";
|
||||
cur_op.type = OperatorType::TupleElement;
|
||||
}
|
||||
else if (first_arg_as_node->name == "arrayElement" && !first_arg_as_node->arguments->children.empty())
|
||||
{
|
||||
auto *arg_inside = first_arg_as_node->arguments->children[0]->as<ASTFunction>();
|
||||
if (arg_inside && (arg_inside->name == "kql_array_sort_asc" || arg_inside->name == "kql_array_sort_desc"))
|
||||
first_arg_as_node->name = "tupleElement";
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
function = makeASTFunction(cur_op);
|
||||
|
||||
if (!popLastNOperands(function->children[0]->children, cur_op.arity))
|
||||
@ -2163,6 +2184,56 @@ private:
|
||||
bool if_permitted;
|
||||
};
|
||||
|
||||
/// Layer for table function 'kql'
|
||||
class KustoLayer : public Layer
|
||||
{
|
||||
public:
|
||||
|
||||
KustoLayer() : Layer(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true) {}
|
||||
|
||||
bool parse(IParser::Pos & pos, Expected & expected, Action & /*action*/) override
|
||||
{
|
||||
/// kql(table|project ...)
|
||||
/// 0. Parse the kql query
|
||||
/// 1. Parse closing token
|
||||
if (state == 0)
|
||||
{
|
||||
ASTPtr query;
|
||||
--pos;
|
||||
if (!ParserKQLTableFunction().parse(pos, query, expected))
|
||||
return false;
|
||||
--pos;
|
||||
pushResult(query);
|
||||
|
||||
if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
finished = true;
|
||||
state = 1;
|
||||
return true;
|
||||
}
|
||||
|
||||
if (state == 1)
|
||||
{
|
||||
if (ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected))
|
||||
{
|
||||
if (!mergeElement())
|
||||
return false;
|
||||
|
||||
finished = true;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
protected:
|
||||
bool getResultImpl(ASTPtr & node) override
|
||||
{
|
||||
node = makeASTFunction("view", std::move(elements)); // reuse view function for kql
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
std::unique_ptr<Layer> getFunctionLayer(ASTPtr identifier, bool is_table_function, bool allow_function_parameters_ = true)
|
||||
{
|
||||
@ -2199,6 +2270,8 @@ std::unique_ptr<Layer> getFunctionLayer(ASTPtr identifier, bool is_table_functio
|
||||
return std::make_unique<ViewLayer>(false);
|
||||
else if (function_name_lowercase == "viewifpermitted")
|
||||
return std::make_unique<ViewLayer>(true);
|
||||
else if (function_name_lowercase == "kql")
|
||||
return std::make_unique<KustoLayer>();
|
||||
}
|
||||
|
||||
if (function_name == "tuple")
|
||||
@ -2454,7 +2527,7 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos
|
||||
|
||||
if (layers.front()->is_table_function)
|
||||
{
|
||||
if (typeid_cast<ViewLayer *>(layers.back().get()))
|
||||
if (typeid_cast<ViewLayer *>(layers.back().get()) || typeid_cast<KustoLayer *>(layers.back().get()))
|
||||
{
|
||||
if (identifier_parser.parse(pos, tmp, expected)
|
||||
&& ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected))
|
||||
@ -2592,6 +2665,7 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos
|
||||
}
|
||||
else if (pos->type == TokenType::OpeningRoundBracket)
|
||||
{
|
||||
|
||||
if (subquery_parser.parse(pos, tmp, expected))
|
||||
{
|
||||
layers.back()->pushOperand(std::move(tmp));
|
||||
|
996
src/Parsers/Kusto/KQL_ReleaseNote.md
Normal file
996
src/Parsers/Kusto/KQL_ReleaseNote.md
Normal file
@ -0,0 +1,996 @@
|
||||
## KQL implemented features
|
||||
|
||||
# October 9, 2022
|
||||
|
||||
## operator
|
||||
- [distinct](https://learn.microsoft.com/en-us/azure/data-explorer/kusto/query/distinctoperator)
|
||||
`Customers | distinct *`
|
||||
`Customers | distinct Occupation`
|
||||
`Customers | distinct Occupation, Education`
|
||||
`Customers | where Age <30 | distinct Occupation, Education`
|
||||
`Customers | where Age <30 | order by Age| distinct Occupation, Education`
|
||||
|
||||
## String functions
|
||||
- [reverse](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/reversefunction)
|
||||
`print reverse(123)`
|
||||
`print reverse(123.34)`
|
||||
`print reverse('clickhouse')`
|
||||
`print reverse(3h)`
|
||||
`print reverse(datetime(2017-1-1 12:23:34))`
|
||||
|
||||
- [parse_command_line](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parse-command-line)
|
||||
`print parse_command_line('echo \"hello world!\" print$?', \"Windows\")`
|
||||
|
||||
- [parse_csv](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parsecsvfunction)
|
||||
`print result=parse_csv('aa,b,cc')`
|
||||
`print result_multi_record=parse_csv('record1,a,b,c\nrecord2,x,y,z')`
|
||||
|
||||
- [parse_json](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parsejsonfunction)
|
||||
`print parse_json( dynamic([1, 2, 3]))`
|
||||
`print parse_json('{"a":123.5, "b":"{\\"c\\":456}"}')`
|
||||
|
||||
- [extract_json](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractjsonfunction)
|
||||
`print extract_json( "$.a" , '{"a":123, "b":"{\\"c\\":456}"}' , typeof(int))`
|
||||
|
||||
- [parse_version](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parse-versionfunction)
|
||||
`print parse_version('1')`
|
||||
`print parse_version('1.2.3.40')`
|
||||
|
||||
## Bug fixed
|
||||
- [correct array index in expression](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1474)
|
||||
array index should start with 0
|
||||
- [Summarize should generate alias or use correct columns](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1303)
|
||||
- if bin is used , the column should be in select list if no alias include
|
||||
- if no column included in aggregate functions, ( like count() ), should has alias with fun name + '_',e.g count_
|
||||
- if column name included in aggregate functions, should have fun name + "_" + column name , like count(Age) -> count_Age
|
||||
- if argument of an aggregate functions is an exprision, Columns1 ... Columnsn should be used as alias
|
||||
```
|
||||
Customers | summarize count() by bin(Age, 10)
|
||||
┌─Age─┬─count_─┐
|
||||
│ 40 │ 2 │
|
||||
│ 20 │ 6 │
|
||||
│ 30 │ 4 │
|
||||
└─────┴────────┘
|
||||
Customers | summarize count(Age) by bin(Age, 10)
|
||||
┌─Age─┬─count_Age─┐
|
||||
│ 40 │ 2 │
|
||||
│ 20 │ 6 │
|
||||
│ 30 │ 4 │
|
||||
└─────┴───────────┘
|
||||
Customers | summarize count(Age+1) by bin(Age+1, 10)
|
||||
┌─Columns1─┬─count_─┐
|
||||
│ 40 │ 2 │
|
||||
│ 20 │ 6 │
|
||||
│ 30 │ 4 │
|
||||
└──────────┴────────┘
|
||||
```
|
||||
- [extend doesn't replace existing columns](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1246)
|
||||
|
||||
- [throw exception if use quoted string as alias](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1470)
|
||||
|
||||
- [repeat() doesn't work with count argument as negative value](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1368)
|
||||
|
||||
- [substring() doesn't work right with negative offsets](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1336)
|
||||
- [endofmonth() doesn't return correct result](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1370)
|
||||
|
||||
- [split() outputs array instead of string](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1343)
|
||||
|
||||
- [split() returns empty string when arg goes out of bound](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1328)
|
||||
|
||||
- [split() doesn't work with negative index](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1325)
|
||||
|
||||
|
||||
# September 26, 2022
|
||||
## Bug fixed :
|
||||
["select * from kql" results in syntax error](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1119)
|
||||
[Parsing ipv4 with arrayStringConcat throws exception](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1259)
|
||||
[CH Client crashes on invalid function name](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1266)
|
||||
[extract() doesn't work right with 4th argument i.e typeof()](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1327)
|
||||
[parse_ipv6_mask return incorrect results](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1050)
|
||||
[timespan returns wrong output in seconds](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1275)
|
||||
[timespan doesn't work for nanoseconds and tick](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1298)
|
||||
[totimespan() doesn't work for nanoseconds and tick timespan unit](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1301)
|
||||
[data types should throw exception in certain cases](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1112)
|
||||
[decimal does not support scientific notation](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1197)
|
||||
[extend statement causes client core dumping](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1260)
|
||||
[extend crashes with array sorting](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1247)
|
||||
[Core dump happens when WHERE keyword doesn't follow field name](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1335)
|
||||
[Null values are missing in the result of `make_list_with_nulls'](https://github.ibm.com/ClickHouse/issue-repo/issues/1009)
|
||||
[trim functions use non-unique aliases](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1111)
|
||||
[format_ipv4_mask returns incorrect mask value](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1039)
|
||||
|
||||
# September 12, 2022
|
||||
## Extend operator
|
||||
https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extendoperator
|
||||
`T | extend T | extend duration = endTime - startTime`
|
||||
`T | project endTime, startTime | extend duration = endTime - startTime`
|
||||
## Array functions
|
||||
- [array_reverse](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/array-reverse-function)
|
||||
`print array_reverse(dynamic(["this", "is", "an", "example"])) == dynamic(["example","an","is","this"])`
|
||||
|
||||
- [array_rotate_left](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/array_rotate_leftfunction)
|
||||
`print array_rotate_left(dynamic([1,2,3,4,5]), 2) == dynamic([3,4,5,1,2])`
|
||||
`print array_rotate_left(dynamic([1,2,3,4,5]), -2) == dynamic([4,5,1,2,3])`
|
||||
|
||||
- [array_rotate_right](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/array_rotate_rightfunction)
|
||||
`print array_rotate_right(dynamic([1,2,3,4,5]), -2) == dynamic([3,4,5,1,2])`
|
||||
`print array_rotate_right(dynamic([1,2,3,4,5]), 2) == dynamic([4,5,1,2,3])`
|
||||
|
||||
- [array_shift_left](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/array_shift_leftfunction)
|
||||
`print array_shift_left(dynamic([1,2,3,4,5]), 2) == dynamic([3,4,5,null,null])`
|
||||
`print array_shift_left(dynamic([1,2,3,4,5]), -2) == dynamic([null,null,1,2,3])`
|
||||
`print array_shift_left(dynamic([1,2,3,4,5]), 2, -1) == dynamic([3,4,5,-1,-1])`
|
||||
`print array_shift_left(dynamic(['a', 'b', 'c']), 2) == dynamic(['c','',''])`
|
||||
|
||||
- [array_shift_right](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/array_shift_rightfunction)
|
||||
`print array_shift_right(dynamic([1,2,3,4,5]), -2) == dynamic([3,4,5,null,null])`
|
||||
`print array_shift_right(dynamic([1,2,3,4,5]), 2) == dynamic([null,null,1,2,3])`
|
||||
`print array_shift_right(dynamic([1,2,3,4,5]), -2, -1) == dynamic([3,4,5,-1,-1])`
|
||||
`print array_shift_right(dynamic(['a', 'b', 'c']), -2) == dynamic(['c','',''])`
|
||||
|
||||
- [pack_array](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/packarrayfunction)
|
||||
`print x = 1, y = x * 2, z = y * 2, pack_array(x,y,z)`
|
||||
|
||||
Please note that only arrays of elements of the same type may be created at this time. The underlying reasons are explained under the release note section of the `dynamic` data type.
|
||||
|
||||
- [repeat](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/repeatfunction)
|
||||
`print repeat(1, 0) == dynamic([])`
|
||||
`print repeat(1, 3) == dynamic([1, 1, 1])`
|
||||
`print repeat("asd", 3) == dynamic(['asd', 'asd', 'asd'])`
|
||||
`print repeat(timespan(1d), 3) == dynamic([86400, 86400, 86400])`
|
||||
`print repeat(true, 3) == dynamic([true, true, true])`
|
||||
|
||||
- [zip](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/zipfunction)
|
||||
`print zip(dynamic([1,3,5]), dynamic([2,4,6]))`
|
||||
|
||||
Please note that only arrays of the same type are supported in our current implementation. The underlying reasons are explained under the release note section of the `dynamic` data type.
|
||||
|
||||
## Data types
|
||||
- [dynamic](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/dynamic)
|
||||
`print isnull(dynamic(null))`
|
||||
`print dynamic(1) == 1`
|
||||
`print dynamic(timespan(1d)) == 86400`
|
||||
`print dynamic([1, 2, 3])`
|
||||
`print dynamic([[1], [2], [3]])`
|
||||
`print dynamic(['a', "b", 'c'])`
|
||||
|
||||
According to the KQL specifications `dynamic` is a literal, which means that no function calls are permitted. Expressions producing literals such as `datetime` and `timespan` and their aliases (ie. `date` and `time`, respectively) along with nested `dynamic` literals are allowed.
|
||||
|
||||
Please note that our current implementation supports only scalars and arrays made up of elements of the same type. Support for mixed types and property bags is deferred for now, based on our understanding of the required effort and discussion with representatives of the QRadar team.
|
||||
|
||||
## Mathematical functions
|
||||
- [isnan](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/isnanfunction)
|
||||
`print isnan(double(nan)) == true`
|
||||
`print isnan(4.2) == false`
|
||||
`print isnan(4) == false`
|
||||
`print isnan(real(+inf)) == false`
|
||||
|
||||
## Set functions
|
||||
Please note that functions returning arrays with set semantics may return them in any particular order, which may be subject to change in the future.
|
||||
|
||||
- [jaccard_index](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/jaccard-index-function)
|
||||
`print jaccard_index(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3, 4, 4, 4])) == 0.75`
|
||||
`print jaccard_index(dynamic([1, 2, 3]), dynamic([])) == 0`
|
||||
`print jaccard_index(dynamic([]), dynamic([1, 2, 3, 4])) == 0`
|
||||
`print isnan(jaccard_index(dynamic([]), dynamic([])))`
|
||||
`print jaccard_index(dynamic([1, 2, 3]), dynamic([4, 5, 6, 7])) == 0`
|
||||
`print jaccard_index(dynamic(['a', 's', 'd']), dynamic(['f', 'd', 's', 'a'])) == 0.75`
|
||||
`print jaccard_index(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])) == 0.25`
|
||||
|
||||
- [set_difference](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/setdifferencefunction)
|
||||
`print set_difference(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])) == dynamic([])`
|
||||
`print array_sort_asc(set_difference(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[1] == dynamic([4, 5, 6])`
|
||||
`print set_difference(dynamic([4]), dynamic([1, 2, 3])) == dynamic([4])`
|
||||
`print array_sort_asc(set_difference(dynamic([1, 2, 3, 4, 5]), dynamic([5]), dynamic([2, 4])))[1] == dynamic([1, 3])`
|
||||
`print array_sort_asc(set_difference(dynamic([1, 2, 3]), dynamic([])))[1] == dynamic([1, 2, 3])`
|
||||
`print array_sort_asc(set_difference(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])))[1] == dynamic(['d', 's'])`
|
||||
`print array_sort_asc(set_difference(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])))[1] == dynamic(['Chewbacca', 'Han Solo'])`
|
||||
|
||||
- [set_has_element](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/sethaselementfunction)
|
||||
`print set_has_element(dynamic(["this", "is", "an", "example"]), "example") == true`
|
||||
`print set_has_element(dynamic(["this", "is", "an", "example"]), "test") == false`
|
||||
`print set_has_element(dynamic([1, 2, 3]), 2) == true`
|
||||
`print set_has_element(dynamic([1, 2, 3, 4.2]), 4) == false`
|
||||
|
||||
- [set_intersect](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/setintersectfunction)
|
||||
`print array_sort_asc(set_intersect(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])))[1] == dynamic([1, 2, 3])`
|
||||
`print array_sort_asc(set_intersect(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[1] == dynamic([1, 2, 3])`
|
||||
`print set_intersect(dynamic([4]), dynamic([1, 2, 3])) == dynamic([])`
|
||||
`print set_intersect(dynamic([1, 2, 3, 4, 5]), dynamic([1, 3, 5]), dynamic([2, 5])) == dynamic([5])`
|
||||
`print set_intersect(dynamic([1, 2, 3]), dynamic([])) == dynamic([])`
|
||||
`print set_intersect(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])) == dynamic(['a'])`
|
||||
`print set_intersect(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])) == dynamic(['Darth Vader'])`
|
||||
|
||||
- [set_union](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/setunionfunction)
|
||||
`print array_sort_asc(set_union(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])))[1] == dynamic([1, 2, 3])`
|
||||
`print array_sort_asc(set_union(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[1] == dynamic([1, 2, 3, 4, 5, 6])`
|
||||
`print array_sort_asc(set_union(dynamic([4]), dynamic([1, 2, 3])))[1] == dynamic([1, 2, 3, 4])`
|
||||
`print array_sort_asc(set_union(dynamic([1, 3, 4]), dynamic([5]), dynamic([2, 4])))[1] == dynamic([1, 2, 3, 4, 5])`
|
||||
`print array_sort_asc(set_union(dynamic([1, 2, 3]), dynamic([])))[1] == dynamic([1, 2, 3])`
|
||||
`print array_sort_asc(set_union(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])))[1] == dynamic(['a', 'd', 'f', 's'])`
|
||||
`print array_sort_asc(set_union(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])))[1] == dynamic(['Chewbacca', 'Darth Sidious', 'Darth Vader', 'Han Solo'])`
|
||||
|
||||
# August 29, 2022
|
||||
|
||||
## **mv-expand operator**
|
||||
https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/mvexpandoperator
|
||||
Note: **expand on array columns only**
|
||||
- test cases
|
||||
```
|
||||
CREATE TABLE T
|
||||
(
|
||||
a UInt8,
|
||||
b Array(String),
|
||||
c Array(Int8),
|
||||
d Array(Int8)
|
||||
) ENGINE = Memory;
|
||||
|
||||
INSERT INTO T VALUES (1, ['Salmon', 'Steak','Chicken'],[1,2,3,4],[5,6,7,8])
|
||||
|
||||
T | mv-expand c
|
||||
T | mv-expand c, d
|
||||
T | mv-expand b | mv-expand c
|
||||
T | mv-expand c to typeof(bool)
|
||||
T | mv-expand with_itemindex=index b, c, d
|
||||
T | mv-expand array_concat(c,d)
|
||||
T | mv-expand x = c, y = d
|
||||
T | mv-expand xy = array_concat(c, d)
|
||||
T | mv-expand with_itemindex=index c,d to typeof(bool)
|
||||
```
|
||||
|
||||
## **make-series operator**
|
||||
https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/make-seriesoperator
|
||||
|
||||
- test case make-series on datetime column
|
||||
```
|
||||
CREATE TABLE T
|
||||
(
|
||||
Supplier Nullable(String),
|
||||
Fruit String ,
|
||||
Price Float64,
|
||||
Purchase Date
|
||||
) ENGINE = Memory;
|
||||
|
||||
INSERT INTO T VALUES ('Aldi','Apple',4,'2016-09-10');
|
||||
INSERT INTO T VALUES ('Costco','Apple',2,'2016-09-11');
|
||||
INSERT INTO T VALUES ('Aldi','Apple',6,'2016-09-10');
|
||||
INSERT INTO T VALUES ('Costco','Snargaluff',100,'2016-09-12');
|
||||
INSERT INTO T VALUES ('Aldi','Apple',7,'2016-09-12');
|
||||
INSERT INTO T VALUES ('Aldi','Snargaluff',400,'2016-09-11');
|
||||
INSERT INTO T VALUES ('Costco','Snargaluff',104,'2016-09-12');
|
||||
INSERT INTO T VALUES ('Aldi','Apple',5,'2016-09-12');
|
||||
INSERT INTO T VALUES ('Aldi','Snargaluff',600,'2016-09-11');
|
||||
INSERT INTO T VALUES ('Costco','Snargaluff',200,'2016-09-10');
|
||||
```
|
||||
Have from and to
|
||||
```
|
||||
T | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) to datetime(2016-09-13) step 1d by Supplier, Fruit
|
||||
```
|
||||
Has from , without to
|
||||
```
|
||||
T | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) step 1d by Supplier, Fruit
|
||||
```
|
||||
Without from , has to
|
||||
```
|
||||
T | make-series PriceAvg = avg(Price) default=0 on Purchase to datetime(2016-09-13) step 1d by Supplier, Fruit
|
||||
```
|
||||
Without from , without to
|
||||
```
|
||||
T | make-series PriceAvg = avg(Price) default=0 on Purchase step 1d by Supplier, Fruit
|
||||
```
|
||||
Without by clause
|
||||
```
|
||||
T | make-series PriceAvg = avg(Price) default=0 on Purchase step 1d
|
||||
```
|
||||
Without aggregation alias
|
||||
```
|
||||
T | make-series avg(Price) default=0 on Purchase step 1d by Supplier, Fruit
|
||||
```
|
||||
Has group expression alias
|
||||
```
|
||||
T | make-series avg(Price) default=0 on Purchase step 1d by Supplier_Name = Supplier, Fruit
|
||||
```
|
||||
Use different step value
|
||||
```
|
||||
T | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) to datetime(2016-09-13) step 3d by Supplier, Fruit
|
||||
```
|
||||
- test case make-series on numeric column
|
||||
```
|
||||
CREATE TABLE T2
|
||||
(
|
||||
Supplier Nullable(String),
|
||||
Fruit String ,
|
||||
Price Int32,
|
||||
Purchase Int32
|
||||
) ENGINE = Memory;
|
||||
|
||||
INSERT INTO T2 VALUES ('Aldi','Apple',4,10);
|
||||
INSERT INTO T2 VALUES ('Costco','Apple',2,11);
|
||||
INSERT INTO T2 VALUES ('Aldi','Apple',6,10);
|
||||
INSERT INTO T2 VALUES ('Costco','Snargaluff',100,12);
|
||||
INSERT INTO T2 VALUES ('Aldi','Apple',7,12);
|
||||
INSERT INTO T2 VALUES ('Aldi','Snargaluff',400,11);
|
||||
INSERT INTO T2 VALUES ('Costco','Snargaluff',104,12);
|
||||
INSERT INTO T2 VALUES ('Aldi','Apple',5,12);
|
||||
INSERT INTO T2 VALUES ('Aldi','Snargaluff',600,11);
|
||||
INSERT INTO T2 VALUES ('Costco','Snargaluff',200,10);
|
||||
```
|
||||
Have from and to
|
||||
```
|
||||
T2 | make-series PriceAvg=avg(Price) default=0 on Purchase from 10 to 15 step 1.0 by Supplier, Fruit;
|
||||
```
|
||||
Has from , without to
|
||||
```
|
||||
T2 | make-series PriceAvg=avg(Price) default=0 on Purchase from 10 step 1.0 by Supplier, Fruit;
|
||||
```
|
||||
Without from , has to
|
||||
```
|
||||
T2 | make-series PriceAvg=avg(Price) default=0 on Purchase to 18 step 4.0 by Supplier, Fruit;
|
||||
```
|
||||
Without from , without to
|
||||
```
|
||||
T2 | make-series PriceAvg=avg(Price) default=0 on Purchase step 2.0 by Supplier, Fruit;
|
||||
```
|
||||
Without by clause
|
||||
```
|
||||
T2 | make-series PriceAvg=avg(Price) default=0 on Purchase step 2.0;
|
||||
```
|
||||
|
||||
## Aggregate Functions
|
||||
- [bin](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binfunction)
|
||||
`print bin(4.5, 1)`
|
||||
`print bin(time(16d), 7d)`
|
||||
`print bin(datetime(1970-05-11 13:45:07), 1d)`
|
||||
- [stdev](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/stdev-aggfunction)
|
||||
`Customers | summarize t = stdev(Age) by FirstName`
|
||||
|
||||
- [stdevif](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/stdevif-aggfunction)
|
||||
`Customers | summarize t = stdevif(Age, Age < 10) by FirstName`
|
||||
|
||||
- [binary_all_and](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-all-and-aggfunction)
|
||||
`Customers | summarize t = binary_all_and(Age) by FirstName`
|
||||
|
||||
- [binary_all_or](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-all-or-aggfunction)
|
||||
`Customers | summarize t = binary_all_or(Age) by FirstName`
|
||||
|
||||
- [binary_all_xor](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-all-xor-aggfunction)
|
||||
`Customers | summarize t = binary_all_xor(Age) by FirstName`
|
||||
|
||||
- [percentiles](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/percentiles-aggfunction)
|
||||
`Customers | summarize percentiles(Age, 30, 40, 50, 60, 70) by FirstName`
|
||||
|
||||
- [percentilesw](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/percentiles-aggfunction)
|
||||
`DataTable | summarize t = percentilesw(Bucket, Frequency, 50, 75, 99.9)`
|
||||
|
||||
- [percentile](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/percentiles-aggfunction)
|
||||
`Customers | summarize t = percentile(Age, 50) by FirstName`
|
||||
|
||||
- [percentilew](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/percentiles-aggfunction)
|
||||
`DataTable | summarize t = percentilew(Bucket, Frequency, 50)`
|
||||
|
||||
## Dynamic functions
|
||||
- [array_sort_asc](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arraysortascfunction)
|
||||
**Only support the constant dynamic array.**
|
||||
**Returns an array. So, each element of the input has to be of same datatype.**
|
||||
`print t = array_sort_asc(dynamic([null, 'd', 'a', 'c', 'c']))`
|
||||
`print t = array_sort_asc(dynamic([4, 1, 3, 2]))`
|
||||
`print t = array_sort_asc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))`
|
||||
`print t = array_sort_asc(dynamic(['q', 'p', 'r']), dynamic(['clickhouse','hello', 'world']))`
|
||||
`print t = array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , false)`
|
||||
`print t = array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2)`
|
||||
`print t = array_sort_asc( dynamic([null, 'd', null, null, 'a', 'c', 'c', null, null, null]) , false)`
|
||||
`print t = array_sort_asc( dynamic([null, null, null]) , false)`
|
||||
`print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30]), 1 > 2)`
|
||||
`print t = array_sort_asc(dynamic([2, 1, null,3]), dynamic([20, 10, 40, 30, 50, 3]), 1 > 2)`
|
||||
|
||||
- [array_sort_desc](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arraysortdescfunction) **(only support the constant dynamic array)**
|
||||
|
||||
`print t = array_sort_desc(dynamic([null, 'd', 'a', 'c', 'c']))`
|
||||
`print t = array_sort_desc(dynamic([4, 1, 3, 2]))`
|
||||
`print t = array_sort_desc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))`
|
||||
`print t = array_sort_desc(dynamic(['q', 'p', 'r']), dynamic(['clickhouse','hello', 'world']))`
|
||||
`print t = array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , false)`
|
||||
`print t = array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2)`
|
||||
`print t = array_sort_desc( dynamic([null, 'd', null, null, 'a', 'c', 'c', null, null, null]) , false)`
|
||||
`print t = array_sort_desc( dynamic([null, null, null]) , false)`
|
||||
`print t = array_sort_desc(dynamic([2, 1, null, 3]), dynamic([20, 10, 40, 30]), 1 > 2)`
|
||||
`print t = array_sort_desc(dynamic([2, 1, null,3, null]), dynamic([20, 10, 40, 30, 50, 3]), 1 > 2)`
|
||||
|
||||
- [array_concat](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arrayconcatfunction)
|
||||
`print array_concat(dynamic([1, 2, 3]), dynamic([4, 5]), dynamic([6, 7, 8, 9])) == dynamic([1, 2, 3, 4, 5, 6, 7, 8, 9])`
|
||||
|
||||
- [array_iff / array_iif](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arrayifffunction)
|
||||
`print array_iif(dynamic([true, false, true]), dynamic([1, 2, 3]), dynamic([4, 5, 6])) == dynamic([1, 5, 3])`
|
||||
`print array_iif(dynamic([true, false, true]), dynamic([1, 2, 3, 4]), dynamic([4, 5, 6])) == dynamic([1, 5, 3])`
|
||||
`print array_iif(dynamic([true, false, true, false]), dynamic([1, 2, 3, 4]), dynamic([4, 5, 6])) == dynamic([1, 5, 3, null])`
|
||||
`print array_iif(dynamic([1, 0, -1, 44, 0]), dynamic([1, 2, 3, 4]), dynamic([4, 5, 6])) == dynamic([1, 5, 3, 4, null])`
|
||||
|
||||
- [array_slice](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arrayslicefunction)
|
||||
`print array_slice(dynamic([1,2,3]), 1, 2) == dynamic([2, 3])`
|
||||
`print array_slice(dynamic([1,2,3,4,5]), 2, -1) == dynamic([3, 4, 5])`
|
||||
`print array_slice(dynamic([1,2,3,4,5]), -3, -2) == dynamic([3, 4])`
|
||||
|
||||
- [array_split](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arraysplitfunction)
|
||||
`print array_split(dynamic([1,2,3,4,5]), 2) == dynamic([[1,2],[3,4,5]])`
|
||||
`print array_split(dynamic([1,2,3,4,5]), dynamic([1,3])) == dynamic([[1],[2,3],[4,5]])`
|
||||
|
||||
## DateTimeFunctions
|
||||
|
||||
- [ago](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/agofunction)
|
||||
`print ago(2h)`
|
||||
|
||||
- [endofday](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/endofdayfunction)
|
||||
`print endofday(datetime(2017-01-01 10:10:17), -1)`
|
||||
`print endofday(datetime(2017-01-01 10:10:17), 1)`
|
||||
`print endofday(datetime(2017-01-01 10:10:17))`
|
||||
|
||||
- [endofmonth](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/endofmonthfunction)
|
||||
`print endofmonth(datetime(2017-01-01 10:10:17), -1)`
|
||||
`print endofmonth(datetime(2017-01-01 10:10:17), 1)`
|
||||
`print endofmonth(datetime(2017-01-01 10:10:17))`
|
||||
|
||||
- [endofweek](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/endofweekfunction)
|
||||
`print endofweek(datetime(2017-01-01 10:10:17), 1)`
|
||||
`print endofweek(datetime(2017-01-01 10:10:17), -1)`
|
||||
`print endofweek(datetime(2017-01-01 10:10:17))`
|
||||
|
||||
- [endofyear](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/endofyearfunction)
|
||||
`print endofyear(datetime(2017-01-01 10:10:17), -1)`
|
||||
`print endofyear(datetime(2017-01-01 10:10:17), 1)`
|
||||
`print endofyear(datetime(2017-01-01 10:10:17))`
|
||||
|
||||
- [make_datetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/make-datetimefunction)
|
||||
`print make_datetime(2017,10,01)`
|
||||
`print make_datetime(2017,10,01,12,10)`
|
||||
`print make_datetime(2017,10,01,12,11,0.1234567)`
|
||||
|
||||
- [datetime_diff](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/datetime-difffunction)
|
||||
`print datetime_diff('year',datetime(2017-01-01),datetime(2000-12-31))`
|
||||
`print datetime_diff('quarter',datetime(2017-07-01),datetime(2017-03-30))`
|
||||
`print datetime_diff('minute',datetime(2017-10-30 23:05:01),datetime(2017-10-30 23:00:59))`
|
||||
|
||||
- [unixtime_microseconds_todatetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/unixtime-microseconds-todatetimefunction)
|
||||
`print unixtime_microseconds_todatetime(1546300800000000)`
|
||||
|
||||
- [unixtime_milliseconds_todatetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/unixtime-milliseconds-todatetimefunction)
|
||||
`print unixtime_milliseconds_todatetime(1546300800000)`
|
||||
|
||||
- [unixtime_nanoseconds_todatetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/unixtime-nanoseconds-todatetimefunction)
|
||||
`print unixtime_nanoseconds_todatetime(1546300800000000000)`
|
||||
|
||||
- [datetime_part](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/datetime-partfunction)
|
||||
`print datetime_part('day', datetime(2017-10-30 01:02:03.7654321))`
|
||||
|
||||
- [datetime_add](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/datetime-addfunction)
|
||||
`print datetime_add('day',1,datetime(2017-10-30 01:02:03.7654321))`
|
||||
|
||||
- [format_timespan](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/format-timespanfunction)
|
||||
`print format_timespan(time(1d), 'd-[hh:mm:ss]')`
|
||||
`print format_timespan(time('12:30:55.123'), 'ddddd-[hh:mm:ss.ffff]')`
|
||||
|
||||
- [format_datetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/format-datetimefunction)
|
||||
`print format_datetime(todatetime('2009-06-15T13:45:30.6175425'), 'yy-M-dd [H:mm:ss.fff]')`
|
||||
`print format_datetime(datetime(2015-12-14 02:03:04.12345), 'y-M-d h:m:s tt')`
|
||||
|
||||
- [todatetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/todatetimefunction)
|
||||
`print todatetime('2014-05-25T08:20:03.123456Z')`
|
||||
`print todatetime('2014-05-25 20:03.123')`
|
||||
|
||||
- [totimespan] (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/totimespanfunction)
|
||||
` print totimespan('0.01:34:23')`
|
||||
`print totimespan(1d)`
|
||||
|
||||
# August 15, 2022
|
||||
**double quote support**
|
||||
``print res = strcat("double ","quote")``
|
||||
## Aggregate functions
|
||||
- [bin_at](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binatfunction)
|
||||
`print res = bin_at(6.5, 2.5, 7)`
|
||||
`print res = bin_at(1h, 1d, 12h)`
|
||||
`print res = bin_at(datetime(2017-05-15 10:20:00.0), 1d, datetime(1970-01-01 12:00:00.0))`
|
||||
`print res = bin_at(datetime(2017-05-17 10:20:00.0), 7d, datetime(2017-06-04 00:00:00.0))`
|
||||
|
||||
- [array_index_of](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arrayindexoffunction)
|
||||
*Supports only basic lookup. Do not support start_index, length and occurrence*
|
||||
`print output = array_index_of(dynamic(['John', 'Denver', 'Bob', 'Marley']), 'Marley')`
|
||||
`print output = array_index_of(dynamic([1, 2, 3]), 2)`
|
||||
- [array_sum](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/array-sum-function)
|
||||
`print output = array_sum(dynamic([2, 5, 3]))`
|
||||
`print output = array_sum(dynamic([2.5, 5.5, 3]))`
|
||||
- [array_length](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/arraylengthfunction)
|
||||
`print output = array_length(dynamic(['John', 'Denver', 'Bob', 'Marley']))`
|
||||
`print output = array_length(dynamic([1, 2, 3]))`
|
||||
|
||||
## Conversion
|
||||
- [tobool / toboolean](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/toboolfunction)
|
||||
`print tobool(true) == true`
|
||||
`print toboolean(false) == false`
|
||||
`print tobool(0) == false`
|
||||
`print toboolean(19819823) == true`
|
||||
`print tobool(-2) == true`
|
||||
`print isnull(toboolean('a'))`
|
||||
`print tobool('true') == true`
|
||||
`print toboolean('false') == false`
|
||||
|
||||
- [todouble / toreal](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/todoublefunction)
|
||||
`print todouble(4) == 4`
|
||||
`print toreal(4.2) == 4.2`
|
||||
`print isnull(todouble('a'))`
|
||||
`print toreal('-0.3') == -0.3`
|
||||
|
||||
- [toint](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/tointfunction)
|
||||
`print isnull(toint('a'))`
|
||||
`print toint(4) == 4`
|
||||
`print toint('4') == 4`
|
||||
`print isnull(toint(4.2))`
|
||||
|
||||
- [tostring](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/tostringfunction)
|
||||
`print tostring(123) == '123'`
|
||||
`print tostring('asd') == 'asd'`
|
||||
|
||||
## Data Types
|
||||
- [dynamic](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/dynamic)
|
||||
*Supports only 1D array*
|
||||
`print output = dynamic(['a', 'b', 'c'])`
|
||||
`print output = dynamic([1, 2, 3])`
|
||||
|
||||
- [bool,boolean](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/bool)
|
||||
`print bool(1)`
|
||||
`print boolean(0)`
|
||||
|
||||
- [datetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/datetime)
|
||||
`print datetime(2015-12-31 23:59:59.9)`
|
||||
`print datetime('2015-12-31 23:59:59.9')`
|
||||
`print datetime("2015-12-31:)`
|
||||
|
||||
- [guid](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/guid)
|
||||
`print guid(74be27de-1e4e-49d9-b579-fe0b331d3642)`
|
||||
`print guid('74be27de-1e4e-49d9-b579-fe0b331d3642')`
|
||||
`print guid('74be27de1e4e49d9b579fe0b331d3642')`
|
||||
|
||||
- [int](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/int)
|
||||
`print int(1)`
|
||||
|
||||
- [long](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/long)
|
||||
`print long(16)`
|
||||
|
||||
- [real](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/real)
|
||||
`print real(1)`
|
||||
|
||||
- [timespan ,time](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/scalar-data-types/timespan)
|
||||
**Note** the timespan is used for calculating datatime, so the output is in seconds. e.g. time(1h) = 3600
|
||||
`print 1d`
|
||||
`print 30m`
|
||||
`print time('0.12:34:56.7')`
|
||||
`print time(2h)`
|
||||
`print timespan(2h)`
|
||||
|
||||
|
||||
## StringFunctions
|
||||
|
||||
- [base64_encode_fromguid](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/base64-encode-fromguid-function)
|
||||
`print Quine = base64_encode_fromguid('ae3133f2-6e22-49ae-b06a-16e6a9b212eb')`
|
||||
- [base64_decode_toarray](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/base64_decode_toarrayfunction)
|
||||
`print base64_decode_toarray('S3VzdG8=')`
|
||||
- [base64_decode_toguid](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/base64-decode-toguid-function)
|
||||
`print base64_decode_toguid('YWUzMTMzZjItNmUyMi00OWFlLWIwNmEtMTZlNmE5YjIxMmVi')`
|
||||
- [replace_regex](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/replace-regex-function)
|
||||
`print replace_regex('Hello, World!', '.', '\\0\\0')`
|
||||
- [has_any_index](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-any-index-function)
|
||||
`print idx = has_any_index('this is an example', dynamic(['this', 'example']))`
|
||||
- [translate](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/translatefunction)
|
||||
`print translate('krasp', 'otsku', 'spark')`
|
||||
- [trim](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/trimfunction)
|
||||
`print trim('--', '--https://bing.com--')`
|
||||
- [trim_end](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/trimendfunction)
|
||||
`print trim_end('.com', 'bing.com')`
|
||||
- [trim_start](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/trimstartfunction)
|
||||
`print trim_start('[^\\w]+', strcat('- ','Te st1','// $'))`
|
||||
|
||||
## DateTimeFunctions
|
||||
- [startofyear](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/startofyearfunction)
|
||||
`print startofyear(datetime(2017-01-01 10:10:17), -1)`
|
||||
`print startofyear(datetime(2017-01-01 10:10:17), 0)`
|
||||
`print startofyear(datetime(2017-01-01 10:10:17), 1)`
|
||||
- [weekofyear](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/weekofyearfunction)
|
||||
`print week_of_year(datetime(2020-12-31))`
|
||||
`print week_of_year(datetime(2020-06-15))`
|
||||
`print week_of_year(datetime(1970-01-01))`
|
||||
`print week_of_year(datetime(2000-01-01))`
|
||||
|
||||
- [startofweek](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/startofweekfunction)
|
||||
`print startofweek(datetime(2017-01-01 10:10:17), -1)`
|
||||
`print startofweek(datetime(2017-01-01 10:10:17), 0)`
|
||||
`print startofweek(datetime(2017-01-01 10:10:17), 1)`
|
||||
|
||||
- [startofmonth](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/startofmonthfunction)
|
||||
`print startofmonth(datetime(2017-01-01 10:10:17), -1)`
|
||||
`print startofmonth(datetime(2017-01-01 10:10:17), 0)`
|
||||
`print startofmonth(datetime(2017-01-01 10:10:17), 1)`
|
||||
|
||||
- [startofday](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/startofdayfunction)
|
||||
`print startofday(datetime(2017-01-01 10:10:17), -1)`
|
||||
`print startofday(datetime(2017-01-01 10:10:17), 0)`
|
||||
`print startofday(datetime(2017-01-01 10:10:17), 1)`
|
||||
|
||||
- [monthofyear](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/monthofyearfunction)
|
||||
`print monthofyear(datetime("2015-12-14"))`
|
||||
|
||||
- [hourofday](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/hourofdayfunction)
|
||||
`print hourofday(datetime(2015-12-14 18:54:00))`
|
||||
|
||||
- [getyear](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/getyearfunction)
|
||||
`print getyear(datetime(2015-10-12))`
|
||||
|
||||
- [getmonth](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/getmonthfunction)
|
||||
`print getmonth(datetime(2015-10-12))`
|
||||
|
||||
- [dayofyear](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/dayofyearfunction)
|
||||
`print dayofyear(datetime(2015-12-14))`
|
||||
|
||||
- [dayofmonth](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/dayofmonthfunction)
|
||||
`print (datetime(2015-12-14))`
|
||||
|
||||
- [unixtime_seconds_todatetime](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/unixtime-seconds-todatetimefunction)
|
||||
`print unixtime_seconds_todatetime(1546300800)`
|
||||
|
||||
- [dayofweek](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/dayofweekfunction)
|
||||
`print dayofweek(datetime(2015-12-20))`
|
||||
|
||||
- [now](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/nowfunction)
|
||||
`print now()`
|
||||
`print now(2d)`
|
||||
`print now(-2h)`
|
||||
`print now(5microseconds)`
|
||||
`print now(5seconds)`
|
||||
`print now(6minutes)`
|
||||
`print now(-2d) `
|
||||
`print now(time(1d))`
|
||||
|
||||
|
||||
## Binary functions
|
||||
- [binary_and](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-andfunction)
|
||||
`print binary_and(15, 3) == 3`
|
||||
`print binary_and(1, 2) == 0`
|
||||
- [binary_not](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-notfunction)
|
||||
`print binary_not(1) == -2`
|
||||
- [binary_or](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-orfunction)
|
||||
`print binary_or(3, 8) == 11`
|
||||
`print binary_or(1, 2) == 3`
|
||||
- [binary_shift_left](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-shift-leftfunction)
|
||||
`print binary_shift_left(1, 1) == 2`
|
||||
`print binary_shift_left(1, 64) == 1`
|
||||
- [binary_shift_right](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-shift-rightfunction)
|
||||
`print binary_shift_right(1, 1) == 0`
|
||||
`print binary_shift_right(1, 64) == 1`
|
||||
- [binary_xor](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/binary-xorfunction)
|
||||
`print binary_xor(1, 3) == 2`
|
||||
- [bitset_count_ones](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/bitset-count-onesfunction)
|
||||
`print bitset_count_ones(42) == 3`
|
||||
|
||||
## IP functions
|
||||
- [format_ipv4](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/format-ipv4-function)
|
||||
`print format_ipv4('192.168.1.255', 24) == '192.168.1.0'`
|
||||
`print format_ipv4(3232236031, 24) == '192.168.1.0'`
|
||||
- [format_ipv4_mask](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/format-ipv4-mask-function)
|
||||
`print format_ipv4_mask('192.168.1.255', 24) == '192.168.1.0/24'`
|
||||
`print format_ipv4_mask(3232236031, 24) == '192.168.1.0/24'`
|
||||
- [ipv4_compare](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-comparefunction)
|
||||
`print ipv4_compare('127.0.0.1', '127.0.0.1') == 0`
|
||||
`print ipv4_compare('192.168.1.1', '192.168.1.255') < 0`
|
||||
`print ipv4_compare('192.168.1.1/24', '192.168.1.255/24') == 0`
|
||||
`print ipv4_compare('192.168.1.1', '192.168.1.255', 24) == 0`
|
||||
- [ipv4_is_match](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-is-matchfunction)
|
||||
`print ipv4_is_match('127.0.0.1', '127.0.0.1') == true`
|
||||
`print ipv4_is_match('192.168.1.1', '192.168.1.255') == false`
|
||||
`print ipv4_is_match('192.168.1.1/24', '192.168.1.255/24') == true`
|
||||
`print ipv4_is_match('192.168.1.1', '192.168.1.255', 24) == true`
|
||||
- [ipv6_compare](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv6-comparefunction)
|
||||
`print ipv6_compare('::ffff:7f00:1', '127.0.0.1') == 0`
|
||||
`print ipv6_compare('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995') < 0`
|
||||
`print ipv6_compare('192.168.1.1/24', '192.168.1.255/24') == 0`
|
||||
`print ipv6_compare('fe80::85d:e82c:9446:7994/127', 'fe80::85d:e82c:9446:7995/127') == 0`
|
||||
`print ipv6_compare('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995', 127) == 0`
|
||||
- [ipv6_is_match](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv6-is-matchfunction)
|
||||
`print ipv6_is_match('::ffff:7f00:1', '127.0.0.1') == true`
|
||||
`print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995') == false`
|
||||
`print ipv6_is_match('192.168.1.1/24', '192.168.1.255/24') == true`
|
||||
`print ipv6_is_match('fe80::85d:e82c:9446:7994/127', 'fe80::85d:e82c:9446:7995/127') == true`
|
||||
`print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995', 127) == true`
|
||||
- [parse_ipv4_mask](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parse-ipv4-maskfunction)
|
||||
`print parse_ipv4_mask('127.0.0.1', 24) == 2130706432`
|
||||
`print parse_ipv4_mask('192.1.168.2', 31) == 3221334018`
|
||||
`print parse_ipv4_mask('192.1.168.3', 31) == 3221334018`
|
||||
`print parse_ipv4_mask('127.2.3.4', 32) == 2130838276`
|
||||
- [parse_ipv6_mask](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parse-ipv6-maskfunction)
|
||||
`print parse_ipv6_mask('127.0.0.1', 24) == '0000:0000:0000:0000:0000:ffff:7f00:0000'`
|
||||
`print parse_ipv6_mask('fe80::85d:e82c:9446:7994', 120) == 'fe80:0000:0000:0000:085d:e82c:9446:7900'`
|
||||
|
||||
# August 1, 2022
|
||||
|
||||
**The config setting to allow modify dialect setting**.
|
||||
- Set dialect setting in server configuration XML at user level(` users.xml `). This sets the ` dialect ` at server startup and CH will do query parsing for all users with ` default ` profile according to dialect value.
|
||||
|
||||
For example:
|
||||
` <profiles>
|
||||
<!-- Default settings. -->
|
||||
<default>
|
||||
<load_balancing>random</load_balancing>
|
||||
<dialect>kusto</dialect>
|
||||
</default> `
|
||||
|
||||
- Query can be executed with HTTP client as below once dialect is set in users.xml
|
||||
` echo "KQL query" | curl -sS "http://localhost:8123/?" --data-binary @- `
|
||||
|
||||
- To execute the query using clickhouse-client , Update clickhouse-client.xml as below and connect clickhouse-client with --config-file option (` clickhouse-client --config-file=<config-file path> `)
|
||||
|
||||
` <config>
|
||||
<dialect>kusto</dialect>
|
||||
</config> `
|
||||
|
||||
OR
|
||||
pass dialect setting with '--'. For example :
|
||||
` clickhouse-client --dialect='kusto' -q "KQL query" `
|
||||
|
||||
- **strcmp** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/strcmpfunction)
|
||||
`print strcmp('abc','ABC')`
|
||||
|
||||
- **parse_url** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parseurlfunction)
|
||||
`print Result = parse_url('scheme://username:password@www.google.com:1234/this/is/a/path?k1=v1&k2=v2#fragment')`
|
||||
|
||||
- **parse_urlquery** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parseurlqueryfunction)
|
||||
`print Result = parse_urlquery('k1=v1&k2=v2&k3=v3')`
|
||||
|
||||
- **print operator** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/printoperator)
|
||||
`print x=1, s=strcat('Hello', ', ', 'World!')`
|
||||
|
||||
- **Aggregate Functions:**
|
||||
- [make_list()](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/makelist-aggfunction)
|
||||
`Customers | summarize t = make_list(FirstName) by FirstName`
|
||||
`Customers | summarize t = make_list(FirstName, 10) by FirstName`
|
||||
- [make_list_if()](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/makelistif-aggfunction)
|
||||
`Customers | summarize t = make_list_if(FirstName, Age > 10) by FirstName`
|
||||
`Customers | summarize t = make_list_if(FirstName, Age > 10, 10) by FirstName`
|
||||
- [make_list_with_nulls()](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/make-list-with-nulls-aggfunction)
|
||||
`Customers | summarize t = make_list_with_nulls(Age) by FirstName`
|
||||
- [make_set()](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/makeset-aggfunction)
|
||||
`Customers | summarize t = make_set(FirstName) by FirstName`
|
||||
`Customers | summarize t = make_set(FirstName, 10) by FirstName`
|
||||
- [make_set_if()](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/makesetif-aggfunction)
|
||||
`Customers | summarize t = make_set_if(FirstName, Age > 10) by FirstName`
|
||||
`Customers | summarize t = make_set_if(FirstName, Age > 10, 10) by FirstName`
|
||||
|
||||
## IP functions
|
||||
|
||||
- **The following functions now support arbitrary expressions as their argument:**
|
||||
- [ipv4_is_private](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-is-privatefunction)
|
||||
- [ipv4_is_in_range](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-is-in-range-function)
|
||||
- [ipv4_netmask_suffix](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-netmask-suffix-function)
|
||||
|
||||
# July 17, 2022
|
||||
|
||||
## Renamed dialect from sql_dialect to dialect
|
||||
|
||||
`set dialect='clickhouse'`
|
||||
`set dialect='kusto'`
|
||||
|
||||
## IP functions
|
||||
- [parse_ipv4](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parse-ipv4function)
|
||||
`"Customers | project parse_ipv4('127.0.0.1')"`
|
||||
- [parse_ipv6](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/parse-ipv6function)
|
||||
`"Customers | project parse_ipv6('127.0.0.1')"`
|
||||
|
||||
Please note that the functions listed below only take constant parameters for now. Further improvement is to be expected to support expressions.
|
||||
|
||||
- [ipv4_is_private](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-is-privatefunction)
|
||||
`"Customers | project ipv4_is_private('192.168.1.6/24')"`
|
||||
`"Customers | project ipv4_is_private('192.168.1.6')"`
|
||||
- [ipv4_is_in_range](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-is-in-range-function)
|
||||
`"Customers | project ipv4_is_in_range('127.0.0.1', '127.0.0.1')"`
|
||||
`"Customers | project ipv4_is_in_range('192.168.1.6', '192.168.1.1/24')"`
|
||||
- [ipv4_netmask_suffix](https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/ipv4-netmask-suffix-function)
|
||||
`"Customers | project ipv4_netmask_suffix('192.168.1.1/24')"`
|
||||
`"Customers | project ipv4_netmask_suffix('192.168.1.1')"`
|
||||
|
||||
## string functions
|
||||
- **support subquery for `in` orerator** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/in-cs-operator)
|
||||
(subquery need to be wrapped with bracket inside bracket)
|
||||
|
||||
`Customers | where Age in ((Customers|project Age|where Age < 30))`
|
||||
Note: case-insensitive not supported yet
|
||||
- **has_all** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-all-operator)
|
||||
`Customers|where Occupation has_any ('Skilled','abcd')`
|
||||
note : subquery not supported yet
|
||||
- **has _any** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-anyoperator)
|
||||
`Customers|where Occupation has_all ('Skilled','abcd')`
|
||||
note : subquery not supported yet
|
||||
- **countof** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/countoffunction)
|
||||
`Customers | project countof('The cat sat on the mat', 'at')`
|
||||
`Customers | project countof('The cat sat on the mat', 'at', 'normal')`
|
||||
`Customers | project countof('The cat sat on the mat', 'at', 'regex')`
|
||||
- **extract** ( https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractfunction)
|
||||
`Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 0, 'The price of PINEAPPLE ice cream is 20')`
|
||||
`Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 1, 'The price of PINEAPPLE ice cream is 20')`
|
||||
`Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 20')`
|
||||
`Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 3, 'The price of PINEAPPLE ice cream is 20')`
|
||||
`Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 20', typeof(real))`
|
||||
|
||||
- **extract_all** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractallfunction)
|
||||
|
||||
`Customers | project extract_all('(\\w)(\\w+)(\\w)','The price of PINEAPPLE ice cream is 20')`
|
||||
note: captureGroups not supported yet
|
||||
|
||||
- **split** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/splitfunction)
|
||||
`Customers | project split('aa_bb', '_')`
|
||||
`Customers | project split('aaa_bbb_ccc', '_', 1)`
|
||||
`Customers | project split('', '_')`
|
||||
`Customers | project split('a__b', '_')`
|
||||
`Customers | project split('aabbcc', 'bb')`
|
||||
|
||||
- **strcat_delim** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/strcat-delimfunction)
|
||||
`Customers | project strcat_delim('-', '1', '2', 'A') , 1s)`
|
||||
`Customers | project strcat_delim('-', '1', '2', strcat('A','b'))`
|
||||
note: only support string now.
|
||||
|
||||
- **indexof** (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/indexoffunction)
|
||||
`Customers | project indexof('abcdefg','cde')`
|
||||
`Customers | project indexof('abcdefg','cde',2)`
|
||||
`Customers | project indexof('abcdefg','cde',6)`
|
||||
note: length and occurrence not supported yet
|
||||
|
||||
|
||||
|
||||
|
||||
# July 4, 2022
|
||||
|
||||
## sql_dialect
|
||||
|
||||
- default is `clickhouse`
|
||||
`set sql_dialect='clickhouse'`
|
||||
- only process kql
|
||||
`set sql_dialect='kusto'`
|
||||
|
||||
## KQL() function
|
||||
|
||||
- create table
|
||||
`CREATE TABLE kql_table4 ENGINE = Memory AS select *, now() as new_column From kql(Customers | project LastName,Age);`
|
||||
verify the content of `kql_table`
|
||||
`select * from kql_table`
|
||||
|
||||
- insert into table
|
||||
create a tmp table:
|
||||
```
|
||||
CREATE TABLE temp
|
||||
(
|
||||
FirstName Nullable(String),
|
||||
LastName String,
|
||||
Age Nullable(UInt8)
|
||||
) ENGINE = Memory;
|
||||
```
|
||||
`INSERT INTO temp select * from kql(Customers|project FirstName,LastName,Age);`
|
||||
verify the content of `temp`
|
||||
`select * from temp`
|
||||
|
||||
- Select from kql()
|
||||
`Select * from kql(Customers|project FirstName)`
|
||||
|
||||
## KQL operators:
|
||||
- Tabular expression statements
|
||||
`Customers`
|
||||
- Select Column
|
||||
`Customers | project FirstName,LastName,Occupation`
|
||||
- Limit returned results
|
||||
`Customers | project FirstName,LastName,Occupation | take 1 | take 3`
|
||||
- sort, order
|
||||
`Customers | order by Age desc , FirstName asc`
|
||||
- Filter
|
||||
`Customers | where Occupation == 'Skilled Manual'`
|
||||
- summarize
|
||||
`Customers |summarize max(Age) by Occupation`
|
||||
|
||||
## KQL string operators and functions
|
||||
- contains
|
||||
`Customers |where Education contains 'degree'`
|
||||
- !contains
|
||||
`Customers |where Education !contains 'degree'`
|
||||
- contains_cs
|
||||
`Customers |where Education contains 'Degree'`
|
||||
- !contains_cs
|
||||
`Customers |where Education !contains 'Degree'`
|
||||
- endswith
|
||||
`Customers | where FirstName endswith 'RE'`
|
||||
- !endswith
|
||||
`Customers | where !FirstName endswith 'RE'`
|
||||
- endswith_cs
|
||||
`Customers | where FirstName endswith_cs 're'`
|
||||
- !endswith_cs
|
||||
`Customers | where FirstName !endswith_cs 're'`
|
||||
- ==
|
||||
`Customers | where Occupation == 'Skilled Manual'`
|
||||
- !=
|
||||
`Customers | where Occupation != 'Skilled Manual'`
|
||||
- has
|
||||
`Customers | where Occupation has 'skilled'`
|
||||
- !has
|
||||
`Customers | where Occupation !has 'skilled'`
|
||||
- has_cs
|
||||
`Customers | where Occupation has 'Skilled'`
|
||||
- !has_cs
|
||||
`Customers | where Occupation !has 'Skilled'`
|
||||
- hasprefix
|
||||
`Customers | where Occupation hasprefix_cs 'Ab'`
|
||||
- !hasprefix
|
||||
`Customers | where Occupation !hasprefix_cs 'Ab'`
|
||||
- hasprefix_cs
|
||||
`Customers | where Occupation hasprefix_cs 'ab'`
|
||||
- !hasprefix_cs
|
||||
`Customers | where Occupation! hasprefix_cs 'ab'`
|
||||
- hassuffix
|
||||
`Customers | where Occupation hassuffix 'Ent'`
|
||||
- !hassuffix
|
||||
`Customers | where Occupation !hassuffix 'Ent'`
|
||||
- hassuffix_cs
|
||||
`Customers | where Occupation hassuffix 'ent'`
|
||||
- !hassuffix_cs
|
||||
`Customers | where Occupation hassuffix 'ent'`
|
||||
- in
|
||||
`Customers |where Education in ('Bachelors','High School')`
|
||||
- !in
|
||||
`Customers | where Education !in ('Bachelors','High School')`
|
||||
- matches regex
|
||||
`Customers | where FirstName matches regex 'P.*r'`
|
||||
- startswith
|
||||
`Customers | where FirstName startswith 'pet'`
|
||||
- !startswith
|
||||
`Customers | where FirstName !startswith 'pet'`
|
||||
- startswith_cs
|
||||
`Customers | where FirstName startswith_cs 'pet'`
|
||||
- !startswith_cs
|
||||
`Customers | where FirstName !startswith_cs 'pet'`
|
||||
|
||||
- base64_encode_tostring()
|
||||
`Customers | project base64_encode_tostring('Kusto1') | take 1`
|
||||
- base64_decode_tostring()
|
||||
`Customers | project base64_decode_tostring('S3VzdG8x') | take 1`
|
||||
- isempty()
|
||||
`Customers | where isempty(LastName)`
|
||||
- isnotempty()
|
||||
`Customers | where isnotempty(LastName)`
|
||||
- isnotnull()
|
||||
`Customers | where isnotnull(FirstName)`
|
||||
- isnull()
|
||||
`Customers | where isnull(FirstName)`
|
||||
- url_decode()
|
||||
`Customers | project url_decode('https%3A%2F%2Fwww.test.com%2Fhello%20word') | take 1`
|
||||
- url_encode()
|
||||
`Customers | project url_encode('https://www.test.com/hello word') | take 1`
|
||||
- substring()
|
||||
`Customers | project name_abbr = strcat(substring(FirstName,0,3), ' ', substring(LastName,2))`
|
||||
- strcat()
|
||||
`Customers | project name = strcat(FirstName, ' ', LastName)`
|
||||
- strlen()
|
||||
`Customers | project FirstName, strlen(FirstName)`
|
||||
- strrep()
|
||||
`Customers | project strrep(FirstName,2,'_')`
|
||||
- toupper()
|
||||
`Customers | project toupper(FirstName)`
|
||||
- tolower()
|
||||
`Customers | project tolower(FirstName)`
|
||||
|
||||
## Aggregate Functions
|
||||
- arg_max()
|
||||
- arg_min()
|
||||
- avg()
|
||||
- avgif()
|
||||
- count()
|
||||
- countif()
|
||||
- max()
|
||||
- maxif()
|
||||
- min()
|
||||
- minif()
|
||||
- sum()
|
||||
- sumif()
|
||||
- dcount()
|
||||
- dcountif()
|
||||
- bin
|
||||
|
382
src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp
Normal file
382
src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp
Normal file
@ -0,0 +1,382 @@
|
||||
#include "KQLFunctionFactory.h"
|
||||
#include <Parsers/Kusto/ParserKQLOperators.h>
|
||||
#include <Parsers/Kusto/Utilities.h>
|
||||
#include <Parsers/Kusto/ParserKQLDateTypeTimespan.h>
|
||||
#include <boost/lexical_cast.hpp>
|
||||
#include <magic_enum.hpp>
|
||||
#include <pcg_random.hpp>
|
||||
#include <Poco/String.h>
|
||||
#include <format>
|
||||
#include <numeric>
|
||||
#include <stack>
|
||||
|
||||
namespace DB::ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int UNKNOWN_FUNCTION;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
constexpr DB::TokenType determineClosingPair(const DB::TokenType token_type)
|
||||
{
|
||||
if (token_type == DB::TokenType::OpeningCurlyBrace)
|
||||
return DB::TokenType::ClosingCurlyBrace;
|
||||
else if (token_type == DB::TokenType::OpeningRoundBracket)
|
||||
return DB::TokenType::ClosingRoundBracket;
|
||||
else if (token_type == DB::TokenType::OpeningSquareBracket)
|
||||
return DB::TokenType::ClosingSquareBracket;
|
||||
|
||||
throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unhandled token: {}", magic_enum::enum_name(token_type));
|
||||
}
|
||||
|
||||
constexpr bool isClosingBracket(const DB::TokenType token_type)
|
||||
{
|
||||
return token_type == DB::TokenType::ClosingCurlyBrace || token_type == DB::TokenType::ClosingRoundBracket
|
||||
|| token_type == DB::TokenType::ClosingSquareBracket;
|
||||
}
|
||||
|
||||
constexpr bool isOpeningBracket(const DB::TokenType token_type)
|
||||
{
|
||||
return token_type == DB::TokenType::OpeningCurlyBrace || token_type == DB::TokenType::OpeningRoundBracket
|
||||
|| token_type == DB::TokenType::OpeningSquareBracket;
|
||||
}
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
bool IParserKQLFunction::convert(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return wrapConvertImpl(
|
||||
pos,
|
||||
IncreaseDepthTag{},
|
||||
[&]
|
||||
{
|
||||
bool res = convertImpl(out, pos);
|
||||
if (!res)
|
||||
out = "";
|
||||
return res;
|
||||
});
|
||||
}
|
||||
|
||||
bool IParserKQLFunction::directMapping(
|
||||
String & out, IParser::Pos & pos, const std::string_view ch_fn, const Interval & argument_count_interval)
|
||||
{
|
||||
const auto fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
out.append(ch_fn.data(), ch_fn.length());
|
||||
out.push_back('(');
|
||||
|
||||
int argument_count = 0;
|
||||
const auto begin = pos;
|
||||
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
if (pos != begin)
|
||||
out.append(", ");
|
||||
|
||||
if (const auto argument = getOptionalArgument(fn_name, pos))
|
||||
{
|
||||
++argument_count;
|
||||
out.append(*argument);
|
||||
}
|
||||
|
||||
if (pos->type == TokenType::ClosingRoundBracket)
|
||||
{
|
||||
if (!argument_count_interval.IsWithinBounds(argument_count))
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"{}: between {} and {} arguments are expected, but {} were provided",
|
||||
fn_name,
|
||||
argument_count_interval.Min(),
|
||||
argument_count_interval.Max(),
|
||||
argument_count);
|
||||
|
||||
out.push_back(')');
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
out.clear();
|
||||
pos = begin;
|
||||
return false;
|
||||
}
|
||||
|
||||
String IParserKQLFunction::generateUniqueIdentifier()
|
||||
{
|
||||
// This particular random generator hits each number exactly once before looping over.
|
||||
// Because of this, it's sufficient for queries consisting of up to 2^16 (= 65536) distinct function calls.
|
||||
// Reference: https://www.pcg-random.org/using-pcg-cpp.html#insecure-generators
|
||||
static pcg32_once_insecure random_generator;
|
||||
return std::to_string(random_generator());
|
||||
}
|
||||
|
||||
String IParserKQLFunction::getArgument(const String & function_name, DB::IParser::Pos & pos, const ArgumentState argument_state)
|
||||
{
|
||||
if (auto optional_argument = getOptionalArgument(function_name, pos, argument_state))
|
||||
return std::move(*optional_argument);
|
||||
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Required argument was not provided in {}", function_name);
|
||||
}
|
||||
|
||||
std::vector<std::string> IParserKQLFunction::getArguments(
|
||||
const String & function_name, DB::IParser::Pos & pos, const ArgumentState argument_state, const Interval & argument_count_interval)
|
||||
{
|
||||
std::vector<std::string> arguments;
|
||||
while (auto argument = getOptionalArgument(function_name, pos, argument_state))
|
||||
{
|
||||
arguments.push_back(std::move(*argument));
|
||||
}
|
||||
if (!argument_count_interval.IsWithinBounds(static_cast<int>(arguments.size())))
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"{}: between {} and {} arguments are expected, but {} were provided",
|
||||
function_name,
|
||||
argument_count_interval.Min(),
|
||||
argument_count_interval.Max(),
|
||||
arguments.size());
|
||||
|
||||
return arguments;
|
||||
}
|
||||
|
||||
String IParserKQLFunction::getConvertedArgument(const String & fn_name, IParser::Pos & pos)
|
||||
{
|
||||
int32_t round_bracket_count = 0, square_bracket_count = 0;
|
||||
if (pos->type == TokenType::ClosingRoundBracket || pos->type == TokenType::ClosingSquareBracket)
|
||||
return {};
|
||||
|
||||
if (pos->isEnd() || pos->type == TokenType::PipeMark || pos->type == TokenType::Semicolon)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Need more argument(s) in function: {}", fn_name);
|
||||
|
||||
std::vector<String> tokens;
|
||||
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
if (pos->type == TokenType::OpeningRoundBracket)
|
||||
++round_bracket_count;
|
||||
if (pos->type == TokenType::ClosingRoundBracket)
|
||||
--round_bracket_count;
|
||||
|
||||
if (pos->type == TokenType::OpeningSquareBracket)
|
||||
++square_bracket_count;
|
||||
if (pos->type == TokenType::ClosingSquareBracket)
|
||||
--square_bracket_count;
|
||||
|
||||
if (!KQLOperators::convert(tokens, pos))
|
||||
{
|
||||
if (pos->type == TokenType::BareWord)
|
||||
{
|
||||
tokens.push_back(IParserKQLFunction::getExpression(pos));
|
||||
}
|
||||
else if (
|
||||
pos->type == TokenType::Comma || pos->type == TokenType::ClosingRoundBracket
|
||||
|| pos->type == TokenType::ClosingSquareBracket)
|
||||
{
|
||||
if (pos->type == TokenType::Comma)
|
||||
break;
|
||||
if (pos->type == TokenType::ClosingRoundBracket && round_bracket_count == -1)
|
||||
break;
|
||||
if (pos->type == TokenType::ClosingSquareBracket && square_bracket_count == 0)
|
||||
break;
|
||||
tokens.push_back(String(pos->begin, pos->end));
|
||||
}
|
||||
else
|
||||
{
|
||||
String token;
|
||||
if (pos->type == TokenType::QuotedIdentifier)
|
||||
token = "'" + escapeSingleQuotes(String(pos->begin + 1, pos->end - 1)) + "'";
|
||||
else if (pos->type == TokenType::OpeningSquareBracket)
|
||||
{
|
||||
++pos;
|
||||
String array_index;
|
||||
while (!pos->isEnd() && pos->type != TokenType::ClosingSquareBracket)
|
||||
{
|
||||
array_index += getExpression(pos);
|
||||
++pos;
|
||||
}
|
||||
token = std::format("[ {0} >=0 ? {0} + 1 : {0}]", array_index);
|
||||
}
|
||||
else
|
||||
token = String(pos->begin, pos->end);
|
||||
|
||||
tokens.push_back(token);
|
||||
}
|
||||
}
|
||||
|
||||
++pos;
|
||||
if (pos->type == TokenType::Comma || pos->type == TokenType::ClosingRoundBracket || pos->type == TokenType::ClosingSquareBracket)
|
||||
{
|
||||
if (pos->type == TokenType::Comma)
|
||||
break;
|
||||
if (pos->type == TokenType::ClosingRoundBracket && round_bracket_count == -1)
|
||||
break;
|
||||
if (pos->type == TokenType::ClosingSquareBracket && square_bracket_count == 0)
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
String converted_arg;
|
||||
for (const auto & token : tokens)
|
||||
converted_arg.append((converted_arg.empty() ? "" : " ") + token);
|
||||
|
||||
return converted_arg;
|
||||
}
|
||||
|
||||
std::optional<String>
|
||||
IParserKQLFunction::getOptionalArgument(const String & function_name, DB::IParser::Pos & pos, const ArgumentState argument_state)
|
||||
{
|
||||
if (const auto type = pos->type; type != DB::TokenType::Comma && type != DB::TokenType::OpeningRoundBracket)
|
||||
return {};
|
||||
|
||||
++pos;
|
||||
if (const auto type = pos->type; type == DB::TokenType::ClosingRoundBracket || type == DB::TokenType::ClosingSquareBracket)
|
||||
return {};
|
||||
|
||||
if (argument_state == ArgumentState::Parsed)
|
||||
return getConvertedArgument(function_name, pos);
|
||||
|
||||
if (argument_state != ArgumentState::Raw)
|
||||
throw Exception(
|
||||
ErrorCodes::NOT_IMPLEMENTED,
|
||||
"Argument extraction is not implemented for {}::{}",
|
||||
magic_enum::enum_type_name<ArgumentState>(),
|
||||
magic_enum::enum_name(argument_state));
|
||||
|
||||
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)))
|
||||
{
|
||||
const auto token_type = pos->type;
|
||||
if (isOpeningBracket(token_type))
|
||||
scopes.push(token_type);
|
||||
else if (isClosingBracket(token_type))
|
||||
{
|
||||
if (scopes.empty() || determineClosingPair(scopes.top()) != token_type)
|
||||
throw Exception(
|
||||
DB::ErrorCodes::SYNTAX_ERROR, "Unmatched token: {} when parsing {}", magic_enum::enum_name(token_type), function_name);
|
||||
|
||||
scopes.pop();
|
||||
}
|
||||
|
||||
++pos;
|
||||
}
|
||||
|
||||
return std::string(begin, pos->begin);
|
||||
}
|
||||
|
||||
String IParserKQLFunction::getKQLFunctionName(IParser::Pos & pos)
|
||||
{
|
||||
String fn_name(pos->begin, pos->end);
|
||||
++pos;
|
||||
if (pos->type != TokenType::OpeningRoundBracket)
|
||||
{
|
||||
--pos;
|
||||
return "";
|
||||
}
|
||||
return fn_name;
|
||||
}
|
||||
|
||||
String IParserKQLFunction::kqlCallToExpression(
|
||||
const std::string_view function_name, const std::initializer_list<const std::string_view> params, const uint32_t max_depth)
|
||||
{
|
||||
return kqlCallToExpression(function_name, std::span(params), max_depth);
|
||||
}
|
||||
|
||||
String IParserKQLFunction::kqlCallToExpression(
|
||||
const std::string_view function_name, const std::span<const std::string_view> params, const uint32_t max_depth)
|
||||
{
|
||||
const auto params_str = std::accumulate(
|
||||
std::cbegin(params),
|
||||
std::cend(params),
|
||||
String(),
|
||||
[](String acc, const std::string_view param)
|
||||
{
|
||||
if (!acc.empty())
|
||||
acc.append(", ");
|
||||
|
||||
acc.append(param.data(), param.length());
|
||||
return acc;
|
||||
});
|
||||
|
||||
const auto kql_call = std::format("{}({})", function_name, params_str);
|
||||
DB::Tokens call_tokens(kql_call.c_str(), kql_call.c_str() + kql_call.length());
|
||||
DB::IParser::Pos tokens_pos(call_tokens, max_depth);
|
||||
return DB::IParserKQLFunction::getExpression(tokens_pos);
|
||||
}
|
||||
|
||||
void IParserKQLFunction::validateEndOfFunction(const String & fn_name, IParser::Pos & pos)
|
||||
{
|
||||
if (pos->type != TokenType::ClosingRoundBracket)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Too many arguments in function: {}", fn_name);
|
||||
}
|
||||
|
||||
String IParserKQLFunction::getExpression(IParser::Pos & pos)
|
||||
{
|
||||
String arg(pos->begin, pos->end);
|
||||
auto parseConstTimespan = [&]()
|
||||
{
|
||||
ParserKQLDateTypeTimespan time_span;
|
||||
ASTPtr node;
|
||||
Expected expected;
|
||||
|
||||
if (time_span.parse(pos, node, expected))
|
||||
arg = boost::lexical_cast<std::string>(time_span.toSeconds());
|
||||
};
|
||||
|
||||
if (pos->type == TokenType::BareWord)
|
||||
{
|
||||
const auto fun = KQLFunctionFactory::get(arg);
|
||||
if (String new_arg; fun && fun->convert(new_arg, pos))
|
||||
{
|
||||
validateEndOfFunction(arg, pos);
|
||||
arg = std::move(new_arg);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!fun)
|
||||
{
|
||||
++pos;
|
||||
if (pos->type == TokenType::OpeningRoundBracket)
|
||||
{
|
||||
if (Poco::toLower(arg) != "and" && Poco::toLower(arg) != "or")
|
||||
throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "{} is not a supported kusto function", arg);
|
||||
}
|
||||
--pos;
|
||||
}
|
||||
|
||||
parseConstTimespan();
|
||||
}
|
||||
}
|
||||
else if (pos->type == TokenType::ErrorWrongNumber)
|
||||
parseConstTimespan();
|
||||
else if (pos->type == TokenType::QuotedIdentifier)
|
||||
arg = "'" + escapeSingleQuotes(String(pos->begin + 1, pos->end - 1)) + "'";
|
||||
else if (pos->type == TokenType::OpeningSquareBracket)
|
||||
{
|
||||
++pos;
|
||||
String array_index;
|
||||
while (!pos->isEnd() && pos->type != TokenType::ClosingSquareBracket)
|
||||
{
|
||||
array_index += getExpression(pos);
|
||||
++pos;
|
||||
}
|
||||
arg = std::format("[ {0} >=0 ? {0} + 1 : {0}]", array_index);
|
||||
}
|
||||
|
||||
return arg;
|
||||
}
|
||||
|
||||
String IParserKQLFunction::escapeSingleQuotes(const String & input)
|
||||
{
|
||||
String output;
|
||||
for (const auto & ch : input)
|
||||
{
|
||||
if (ch == '\'')
|
||||
output += ch;
|
||||
output += ch;
|
||||
}
|
||||
return output;
|
||||
}
|
||||
}
|
91
src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.h
Normal file
91
src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.h
Normal file
@ -0,0 +1,91 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
|
||||
#include <span>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Interval
|
||||
{
|
||||
public:
|
||||
using Representation = int;
|
||||
|
||||
Interval(const Representation min_, const Representation max_) : max(max_), min(min_) { }
|
||||
|
||||
Representation Max() const { return max; }
|
||||
Representation Min() const { return min; }
|
||||
bool IsWithinBounds(const Representation value) const { return min <= value && value <= max; }
|
||||
|
||||
static constexpr auto max_bound = std::numeric_limits<Representation>::max();
|
||||
static constexpr auto min_bound = std::numeric_limits<Representation>::min();
|
||||
|
||||
private:
|
||||
Representation max = max_bound;
|
||||
Representation min = min_bound;
|
||||
};
|
||||
|
||||
class IParserKQLFunction
|
||||
{
|
||||
public:
|
||||
enum class ArgumentState
|
||||
{
|
||||
Parsed,
|
||||
Raw
|
||||
};
|
||||
|
||||
template <typename F>
|
||||
ALWAYS_INLINE static bool wrapConvertImpl(IParser::Pos & pos, const F & func)
|
||||
{
|
||||
IParser::Pos begin = pos;
|
||||
bool res = func();
|
||||
if (!res)
|
||||
pos = begin;
|
||||
return res;
|
||||
}
|
||||
|
||||
struct IncreaseDepthTag
|
||||
{
|
||||
};
|
||||
|
||||
template <typename F>
|
||||
ALWAYS_INLINE static bool wrapConvertImpl(IParser::Pos & pos, IncreaseDepthTag, const F & func)
|
||||
{
|
||||
IParser::Pos begin = pos;
|
||||
pos.increaseDepth();
|
||||
bool res = func();
|
||||
pos.decreaseDepth();
|
||||
if (!res)
|
||||
pos = begin;
|
||||
return res;
|
||||
}
|
||||
|
||||
bool convert(String & out, IParser::Pos & pos);
|
||||
virtual const char * getName() const = 0;
|
||||
virtual ~IParserKQLFunction() = default;
|
||||
|
||||
static String generateUniqueIdentifier();
|
||||
static String getArgument(const String & function_name, DB::IParser::Pos & pos, ArgumentState argument_state = ArgumentState::Parsed);
|
||||
static std::vector<std::string> getArguments(
|
||||
const String & function_name,
|
||||
DB::IParser::Pos & pos,
|
||||
ArgumentState argument_state = ArgumentState::Parsed,
|
||||
const Interval & argument_count_interval = {0, Interval::max_bound});
|
||||
static String getConvertedArgument(const String & fn_name, IParser::Pos & pos);
|
||||
static String getExpression(IParser::Pos & pos);
|
||||
static String getKQLFunctionName(IParser::Pos & pos);
|
||||
static std::optional<String>
|
||||
getOptionalArgument(const String & function_name, DB::IParser::Pos & pos, ArgumentState argument_state = ArgumentState::Parsed);
|
||||
static String
|
||||
kqlCallToExpression(std::string_view function_name, std::initializer_list<const std::string_view> params, uint32_t max_depth);
|
||||
static String kqlCallToExpression(std::string_view function_name, std::span<const std::string_view> params, uint32_t max_depth);
|
||||
static String escapeSingleQuotes(const String & input);
|
||||
|
||||
protected:
|
||||
virtual bool convertImpl(String & out, IParser::Pos & pos) = 0;
|
||||
|
||||
static bool directMapping(
|
||||
String & out, IParser::Pos & pos, std::string_view ch_fn, const Interval & argument_count_interval = {0, Interval::max_bound});
|
||||
static void validateEndOfFunction(const String & fn_name, IParser::Pos & pos);
|
||||
};
|
||||
}
|
488
src/Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.cpp
Normal file
488
src/Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.cpp
Normal file
@ -0,0 +1,488 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLIPFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLStringFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ArgMax::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "argMax");
|
||||
}
|
||||
|
||||
bool ArgMin::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "argMin");
|
||||
}
|
||||
|
||||
bool Avg::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "avg");
|
||||
}
|
||||
|
||||
bool AvgIf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "avgIf");
|
||||
}
|
||||
|
||||
bool BinaryAllAnd::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "groupBitAnd");
|
||||
}
|
||||
|
||||
bool BinaryAllOr::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "groupBitOr");
|
||||
}
|
||||
|
||||
bool BinaryAllXor::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "groupBitXor");
|
||||
}
|
||||
|
||||
bool BuildSchema::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool Count::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "count");
|
||||
}
|
||||
|
||||
bool CountIf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "countIf");
|
||||
}
|
||||
|
||||
bool DCount::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
String value = getConvertedArgument(fn_name, pos);
|
||||
|
||||
out = "count(DISTINCT " + value + ")";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool DCountIf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
String value = getConvertedArgument(fn_name, pos);
|
||||
++pos;
|
||||
String condition = getConvertedArgument(fn_name, pos);
|
||||
out = "countIf (DISTINCT " + value + ", " + condition + ")";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool MakeBag::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool MakeBagIf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool MakeList::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
const auto expr = getConvertedArgument(fn_name, pos);
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
const auto max_size = getConvertedArgument(fn_name, pos);
|
||||
out = "groupArrayIf(" + max_size + ")(" + expr + " , " + expr + " IS NOT NULL)";
|
||||
}
|
||||
else
|
||||
out = "groupArrayIf(" + expr + " , " + expr + " IS NOT NULL)";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool MakeListIf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
const auto expr = getConvertedArgument(fn_name, pos);
|
||||
++pos;
|
||||
const auto predicate = getConvertedArgument(fn_name, pos);
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
const auto max_size = getConvertedArgument(fn_name, pos);
|
||||
out = "groupArrayIf(" + max_size + ")(" + expr + " , " + predicate + " )";
|
||||
}
|
||||
else
|
||||
out = "groupArrayIf(" + expr + " , " + predicate + " )";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool MakeListWithNulls::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
const auto column_name = getConvertedArgument(fn_name, pos);
|
||||
out = "arrayConcat(groupArray(" + column_name + "), arrayMap(x -> null, range(0, toUInt32(count(*)-length( groupArray(" + column_name
|
||||
+ "))),1)))";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool MakeSet::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
const auto expr = getConvertedArgument(fn_name, pos);
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
const auto max_size = getConvertedArgument(fn_name, pos);
|
||||
out = "groupUniqArray(" + max_size + ")(" + expr + ")";
|
||||
}
|
||||
else
|
||||
out = "groupUniqArray(" + expr + ")";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool MakeSetIf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
const auto expr = getConvertedArgument(fn_name, pos);
|
||||
++pos;
|
||||
const auto predicate = getConvertedArgument(fn_name, pos);
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
const auto max_size = getConvertedArgument(fn_name, pos);
|
||||
out = "groupUniqArrayIf(" + max_size + ")(" + expr + " , " + predicate + " )";
|
||||
}
|
||||
else
|
||||
out = "groupUniqArrayIf(" + expr + " , " + predicate + " )";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Max::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "max");
|
||||
}
|
||||
|
||||
bool MaxIf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "maxIf");
|
||||
}
|
||||
|
||||
bool Min::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "min");
|
||||
}
|
||||
|
||||
bool MinIf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "minIf");
|
||||
}
|
||||
|
||||
bool Percentile::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String column_name = getConvertedArgument(fn_name, pos);
|
||||
trim(column_name);
|
||||
|
||||
++pos;
|
||||
String value = getConvertedArgument(fn_name, pos);
|
||||
trim(value);
|
||||
|
||||
out = "quantile(" + value + "/100)(" + column_name + ")";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Percentilew::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String bucket_column = getConvertedArgument(fn_name, pos);
|
||||
trim(bucket_column);
|
||||
|
||||
++pos;
|
||||
String frequency_column = getConvertedArgument(fn_name, pos);
|
||||
trim(frequency_column);
|
||||
|
||||
++pos;
|
||||
String value = getConvertedArgument(fn_name, pos);
|
||||
trim(value);
|
||||
|
||||
out = "quantileExactWeighted(" + value + "/100)(" + bucket_column + "," + frequency_column + ")";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Percentiles::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String column_name = getConvertedArgument(fn_name, pos);
|
||||
trim(column_name);
|
||||
String expr = "quantiles(";
|
||||
String value;
|
||||
while (pos->type != TokenType::ClosingRoundBracket)
|
||||
{
|
||||
if (pos->type != TokenType::Comma)
|
||||
{
|
||||
value = String(pos->begin, pos->end);
|
||||
expr = expr + value + "/100";
|
||||
++pos;
|
||||
if (pos->type != TokenType::ClosingRoundBracket)
|
||||
expr += ", ";
|
||||
}
|
||||
else
|
||||
++pos;
|
||||
}
|
||||
out = expr + ")(" + column_name + ")";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool PercentilesArray::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String column_name = getConvertedArgument(fn_name, pos);
|
||||
trim(column_name);
|
||||
String expr = "quantiles(";
|
||||
String value;
|
||||
while (pos->type != TokenType::ClosingRoundBracket)
|
||||
{
|
||||
if (pos->type != TokenType::Comma && String(pos->begin, pos->end) != "dynamic" && pos->type != TokenType::OpeningRoundBracket
|
||||
&& pos->type != TokenType::OpeningSquareBracket && pos->type != TokenType::ClosingSquareBracket)
|
||||
{
|
||||
value = String(pos->begin, pos->end);
|
||||
expr = expr + value + "/100";
|
||||
|
||||
if (pos->type != TokenType::Comma && pos->type != TokenType::OpeningRoundBracket && pos->type != TokenType::OpeningSquareBracket
|
||||
&& pos->type != TokenType::ClosingSquareBracket)
|
||||
expr += ", ";
|
||||
++pos;
|
||||
}
|
||||
else
|
||||
{
|
||||
++pos;
|
||||
}
|
||||
}
|
||||
++pos;
|
||||
if (pos->type != TokenType::ClosingRoundBracket)
|
||||
--pos;
|
||||
|
||||
expr.pop_back();
|
||||
expr.pop_back();
|
||||
expr = expr + ")(" + column_name + ")";
|
||||
out = expr;
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Percentilesw::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String bucket_column = getConvertedArgument(fn_name, pos);
|
||||
trim(bucket_column);
|
||||
|
||||
++pos;
|
||||
String frequency_column = getConvertedArgument(fn_name, pos);
|
||||
trim(frequency_column);
|
||||
|
||||
String expr = "quantilesExactWeighted(";
|
||||
String value;
|
||||
|
||||
while (pos->type != TokenType::ClosingRoundBracket)
|
||||
{
|
||||
if (pos->type != TokenType::Comma)
|
||||
{
|
||||
value = String(pos->begin, pos->end);
|
||||
expr = expr + value + "/100";
|
||||
++pos;
|
||||
if (pos->type != TokenType::ClosingRoundBracket)
|
||||
expr += ", ";
|
||||
}
|
||||
else
|
||||
++pos;
|
||||
}
|
||||
expr = expr + ")(" + bucket_column + "," + frequency_column + ")";
|
||||
out = expr;
|
||||
return true;
|
||||
}
|
||||
|
||||
bool PercentileswArray::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String bucket_column = getConvertedArgument(fn_name, pos);
|
||||
trim(bucket_column);
|
||||
|
||||
++pos;
|
||||
String frequency_column = getConvertedArgument(fn_name, pos);
|
||||
trim(frequency_column);
|
||||
|
||||
String expr = "quantilesExactWeighted(";
|
||||
String value;
|
||||
while (pos->type != TokenType::ClosingRoundBracket)
|
||||
{
|
||||
if (pos->type != TokenType::Comma && String(pos->begin, pos->end) != "dynamic" && pos->type != TokenType::OpeningRoundBracket
|
||||
&& pos->type != TokenType::OpeningSquareBracket && pos->type != TokenType::ClosingSquareBracket)
|
||||
{
|
||||
value = String(pos->begin, pos->end);
|
||||
expr = expr + value + "/100";
|
||||
|
||||
if (pos->type != TokenType::Comma && pos->type != TokenType::OpeningRoundBracket && pos->type != TokenType::OpeningSquareBracket
|
||||
&& pos->type != TokenType::ClosingSquareBracket)
|
||||
expr += ", ";
|
||||
++pos;
|
||||
}
|
||||
else
|
||||
{
|
||||
++pos;
|
||||
}
|
||||
}
|
||||
++pos;
|
||||
if (pos->type != TokenType::ClosingRoundBracket)
|
||||
--pos;
|
||||
|
||||
expr.pop_back();
|
||||
expr.pop_back();
|
||||
expr = expr + ")(" + bucket_column + "," + frequency_column + ")";
|
||||
out = expr;
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Stdev::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
const auto expr = getConvertedArgument(fn_name, pos);
|
||||
out = "sqrt(varSamp(" + expr + "))";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool StdevIf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
const auto expr = getConvertedArgument(fn_name, pos);
|
||||
if (pos->type != TokenType::Comma)
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const auto predicate = getConvertedArgument(fn_name, pos);
|
||||
out = "sqrt(varSampIf(" + expr + ", " + predicate + "))";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Sum::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "sum");
|
||||
}
|
||||
|
||||
bool SumIf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "sumIf");
|
||||
}
|
||||
|
||||
bool TakeAny::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool TakeAnyIf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool Variance::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool VarianceIf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
}
|
267
src/Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.h
Normal file
267
src/Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.h
Normal file
@ -0,0 +1,267 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
namespace DB
|
||||
{
|
||||
class ArgMax : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "arg_max()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ArgMin : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "arg_min()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Avg : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "avg()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class AvgIf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "avgif()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class BinaryAllAnd : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "binary_all_and()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class BinaryAllOr : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "binary_all_or()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class BinaryAllXor : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "binary_all_xor()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class BuildSchema : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "buildschema()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Count : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "count()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class CountIf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "countif()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DCount : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "dcount()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DCountIf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "dcountif()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class MakeBag : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "make_bag()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class MakeBagIf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "make_bag_if()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class MakeList : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "make_list()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class MakeListIf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "make_list_if()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class MakeListWithNulls : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "make_list_with_nulls()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class MakeSet : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "make_set()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class MakeSetIf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "make_set_if()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Max : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "max()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class MaxIf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "maxif()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Min : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "min()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class MinIf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "minif()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Percentile : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "percentile()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Percentilew : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "percentilew()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Percentiles : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "percentiles()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class PercentilesArray : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "percentiles_array()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Percentilesw : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "percentilesw()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class PercentileswArray : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "percentilesw_array()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Stdev : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "stdev()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class StdevIf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "stdevif()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Sum : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "sum()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SumIf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "sumif()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class TakeAny : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "take_any()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class TakeAnyIf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "take_anyif()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Variance : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "variance()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class VarianceIf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "varianceif()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
|
||||
}
|
99
src/Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.cpp
Normal file
99
src/Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.cpp
Normal file
@ -0,0 +1,99 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLIPFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLStringFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
|
||||
#include <format>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool BinaryAnd::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto lhs = getArgument(function_name, pos);
|
||||
const auto rhs = getArgument(function_name, pos);
|
||||
out = std::format("bitAnd(cast({0}, 'Int64'), cast({1}, 'Int64'))", lhs, rhs);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool BinaryNot::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto value = getArgument(function_name, pos);
|
||||
out = std::format("bitNot(cast({0}, 'Int64'))", value);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool BinaryOr::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto lhs = getArgument(function_name, pos);
|
||||
const auto rhs = getArgument(function_name, pos);
|
||||
out = std::format("bitOr(cast({0}, 'Int64'), cast({1}, 'Int64'))", lhs, rhs);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool BinaryShiftLeft::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto value = getArgument(function_name, pos);
|
||||
const auto count = getArgument(function_name, pos);
|
||||
out = std::format("if({1} < 0, null, bitShiftLeft(cast({0}, 'Int64'), {1}))", value, count);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool BinaryShiftRight::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto value = getArgument(function_name, pos);
|
||||
const auto count = getArgument(function_name, pos);
|
||||
out = std::format("if({1} < 0, null, bitShiftRight(cast({0}, 'Int64'), {1}))", value, count);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool BinaryXor::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto lhs = getArgument(function_name, pos);
|
||||
const auto rhs = getArgument(function_name, pos);
|
||||
out = std::format("bitXor(cast({0}, 'Int64'), cast({1}, 'Int64'))", lhs, rhs);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool BitsetCountOnes::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "bitCount");
|
||||
}
|
||||
|
||||
}
|
56
src/Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.h
Normal file
56
src/Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.h
Normal file
@ -0,0 +1,56 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
namespace DB
|
||||
{
|
||||
class BinaryAnd : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "binary_and()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class BinaryNot : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "binary_not()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class BinaryOr : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "binary_or()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class BinaryShiftLeft : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "binary_shift_left()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class BinaryShiftRight : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "binary_shift_right()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class BinaryXor : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "binary_xor()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class BitsetCountOnes : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "bitset_count_ones()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
}
|
170
src/Parsers/Kusto/KustoFunctions/KQLCastingFunctions.cpp
Normal file
170
src/Parsers/Kusto/KustoFunctions/KQLCastingFunctions.cpp
Normal file
@ -0,0 +1,170 @@
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h>
|
||||
|
||||
#include <format>
|
||||
#include <regex>
|
||||
#include <Poco/String.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
bool ToBool::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto param = getArgument(function_name, pos);
|
||||
out = std::format(
|
||||
"multiIf(toString({0}) = 'true', true, "
|
||||
"toString({0}) = 'false', false, toInt64OrNull(toString({0})) != 0)",
|
||||
param,
|
||||
generateUniqueIdentifier());
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ToDateTime::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto param = getArgument(function_name, pos);
|
||||
|
||||
out = std::format("parseDateTime64BestEffortOrNull(toString({0}),9,'UTC')", param);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ToDouble::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto param = getArgument(function_name, pos);
|
||||
out = std::format("toFloat64OrNull(toString({0}))", param);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ToInt::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto param = getArgument(function_name, pos);
|
||||
out = std::format("toInt32OrNull(toString({0}))", param);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ToLong::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto param = getArgument(function_name, pos);
|
||||
out = std::format("toInt64OrNull(toString({0}))", param);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ToString::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto param = getArgument(function_name, pos);
|
||||
out = std::format("ifNull(toString({0}), '')", param);
|
||||
return true;
|
||||
}
|
||||
bool ToTimeSpan::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
String arg;
|
||||
if (pos->type == TokenType::QuotedIdentifier)
|
||||
arg = String(pos->begin + 1, pos->end - 1);
|
||||
else if (pos->type == TokenType::StringLiteral)
|
||||
arg = String(pos->begin, pos->end);
|
||||
else
|
||||
arg = getConvertedArgument(function_name, pos);
|
||||
|
||||
if (pos->type == TokenType::StringLiteral || pos->type == TokenType::QuotedIdentifier)
|
||||
{
|
||||
++pos;
|
||||
try
|
||||
{
|
||||
auto result = kqlCallToExpression("time", {arg}, pos.max_depth);
|
||||
out = std::format("{}", result);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
out = "NULL";
|
||||
}
|
||||
}
|
||||
else
|
||||
out = std::format("{}", arg);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ToDecimal::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String res;
|
||||
int scale = 0;
|
||||
int precision;
|
||||
|
||||
if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral)
|
||||
{
|
||||
res = String(pos->begin + 1, pos->end - 1);
|
||||
++pos;
|
||||
precision = 34;
|
||||
}
|
||||
else
|
||||
{
|
||||
res = getConvertedArgument(fn_name, pos);
|
||||
precision = 17;
|
||||
}
|
||||
static const std::regex expr{"^[0-9]+e[+-]?[0-9]+"};
|
||||
bool is_string = std::any_of(res.begin(), res.end(), ::isalpha) && !(std::regex_match(res, expr));
|
||||
|
||||
if (is_string)
|
||||
out = "NULL";
|
||||
else if (std::regex_match(res, expr))
|
||||
{
|
||||
auto exponential_pos = res.find('e');
|
||||
if (res[exponential_pos + 1] == '+' || res[exponential_pos + 1] == '-')
|
||||
scale = std::stoi(res.substr(exponential_pos + 2, res.length()));
|
||||
else
|
||||
scale = std::stoi(res.substr(exponential_pos + 1, res.length()));
|
||||
|
||||
out = std::format("toDecimal128({}::String,{})", res, scale);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (const auto dot_pos = res.find('.'); dot_pos != String::npos)
|
||||
{
|
||||
const auto tmp = res.substr(0, dot_pos - 1);
|
||||
const auto tmp_length = static_cast<int>(std::ssize(tmp));
|
||||
scale = std::max(precision - tmp_length, 0);
|
||||
}
|
||||
if (scale < 0)
|
||||
out = "NULL";
|
||||
else
|
||||
out = std::format("toDecimal128({}::String,{})", res, scale);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
63
src/Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h
Normal file
63
src/Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h
Normal file
@ -0,0 +1,63 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ToBool : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "tobool()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ToDateTime : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "todatetime()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ToDouble : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "todouble()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ToInt : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "toint()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ToLong : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "tolong()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ToString : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "tostring()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ToTimeSpan : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "totimespan()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ToDecimal : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "todecimal()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
}
|
261
src/Parsers/Kusto/KustoFunctions/KQLDataTypeFunctions.cpp
Normal file
261
src/Parsers/Kusto/KustoFunctions/KQLDataTypeFunctions.cpp
Normal file
@ -0,0 +1,261 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDataTypeFunctions.h>
|
||||
#include <Parsers/Kusto/ParserKQLDateTypeTimespan.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
#include "Poco/String.h"
|
||||
#include <format>
|
||||
#include <regex>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
bool DatatypeBool::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "toBool");
|
||||
}
|
||||
|
||||
bool DatatypeDatetime::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
String datetime_str;
|
||||
|
||||
++pos;
|
||||
if (pos->type == TokenType::QuotedIdentifier)
|
||||
datetime_str = std::format("'{}'", String(pos->begin + 1, pos->end - 1));
|
||||
else if (pos->type == TokenType::StringLiteral)
|
||||
datetime_str = String(pos->begin, pos->end);
|
||||
else if (pos->type == TokenType::BareWord)
|
||||
{
|
||||
datetime_str = getConvertedArgument(fn_name, pos);
|
||||
if (Poco::toUpper(datetime_str) == "NULL")
|
||||
out = "NULL";
|
||||
else
|
||||
out = std::format(
|
||||
"if(toTypeName({0}) = 'Int64' OR toTypeName({0}) = 'Int32'OR toTypeName({0}) = 'Float64' OR toTypeName({0}) = 'UInt32' OR "
|
||||
" toTypeName({0}) = 'UInt64', toDateTime64({0},9,'UTC'), parseDateTime64BestEffortOrNull({0}::String,9,'UTC'))",
|
||||
datetime_str);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
auto start = pos;
|
||||
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
++pos;
|
||||
if (pos->type == TokenType::ClosingRoundBracket)
|
||||
break;
|
||||
}
|
||||
--pos;
|
||||
datetime_str = std::format("'{}'", String(start->begin, pos->end));
|
||||
}
|
||||
out = std::format("parseDateTime64BestEffortOrNull({},9,'UTC')", datetime_str);
|
||||
++pos;
|
||||
return true;
|
||||
}
|
||||
|
||||
bool DatatypeDynamic::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
static const std::unordered_set<std::string_view> ALLOWED_FUNCTIONS{"date", "datetime", "dynamic", "time", "timespan"};
|
||||
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
if (pos->type == TokenType::OpeningCurlyBrace)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Property bags are not supported for now in {}", function_name);
|
||||
|
||||
while (!pos->isEnd() && pos->type != TokenType::ClosingRoundBracket)
|
||||
{
|
||||
if (const auto token_type = pos->type; token_type == TokenType::BareWord || token_type == TokenType::Number
|
||||
|| token_type == TokenType::QuotedIdentifier || token_type == TokenType::StringLiteral)
|
||||
{
|
||||
if (const std::string_view token(pos->begin, pos->end); token_type == TokenType::BareWord && !ALLOWED_FUNCTIONS.contains(token))
|
||||
{
|
||||
++pos;
|
||||
if (pos->type != TokenType::ClosingRoundBracket && pos->type != TokenType::ClosingSquareBracket
|
||||
&& pos->type != TokenType::Comma)
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Expression {} is not supported inside {}", token, function_name);
|
||||
|
||||
--pos;
|
||||
}
|
||||
|
||||
out.append(getConvertedArgument(function_name, pos));
|
||||
}
|
||||
else
|
||||
{
|
||||
out.append(pos->begin, pos->end);
|
||||
++pos;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool DatatypeGuid::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
String guid_str;
|
||||
|
||||
++pos;
|
||||
if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral)
|
||||
guid_str = String(pos->begin + 1, pos->end - 1);
|
||||
else
|
||||
{
|
||||
auto start = pos;
|
||||
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
++pos;
|
||||
if (pos->type == TokenType::ClosingRoundBracket)
|
||||
break;
|
||||
}
|
||||
--pos;
|
||||
guid_str = String(start->begin, pos->end);
|
||||
}
|
||||
out = std::format("toUUIDOrNull('{}')", guid_str);
|
||||
++pos;
|
||||
return true;
|
||||
}
|
||||
|
||||
bool DatatypeInt::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
String guid_str;
|
||||
|
||||
++pos;
|
||||
if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "String is not parsed as int literal.");
|
||||
else
|
||||
{
|
||||
auto arg = getConvertedArgument(fn_name, pos);
|
||||
out = std::format("toInt32({})", arg);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool DatatypeLong::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "toInt64");
|
||||
}
|
||||
|
||||
bool DatatypeReal::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "String is not parsed as double literal.");
|
||||
else
|
||||
{
|
||||
auto arg = getConvertedArgument(fn_name, pos);
|
||||
out = std::format("toFloat64({})", arg);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool DatatypeString::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool DatatypeTimespan::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
ParserKQLDateTypeTimespan time_span;
|
||||
ASTPtr node;
|
||||
Expected expected;
|
||||
bool sign = false;
|
||||
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
if (pos->type == TokenType::Minus)
|
||||
{
|
||||
sign = true;
|
||||
++pos;
|
||||
}
|
||||
if (time_span.parse(pos, node, expected))
|
||||
{
|
||||
if (sign)
|
||||
out = std::format("-{}::Float64", time_span.toSeconds());
|
||||
else
|
||||
out = std::format("{}::Float64", time_span.toSeconds());
|
||||
++pos;
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not a correct timespan expression: {}", fn_name);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool DatatypeDecimal::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String arg;
|
||||
int scale = 0;
|
||||
int precision = 34;
|
||||
|
||||
if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse String as decimal Literal: {}", fn_name);
|
||||
|
||||
--pos;
|
||||
arg = getArgument(fn_name, pos);
|
||||
|
||||
//NULL expr returns NULL not exception
|
||||
static const std::regex expr{"^[0-9]+e[+-]?[0-9]+"};
|
||||
bool is_string = std::any_of(arg.begin(), arg.end(), ::isalpha) && Poco::toUpper(arg) != "NULL" && !(std::regex_match(arg, expr));
|
||||
if (is_string)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse String as decimal Literal: {}", fn_name);
|
||||
|
||||
if (std::regex_match(arg, expr))
|
||||
{
|
||||
auto exponential_pos = arg.find('e');
|
||||
if (arg[exponential_pos + 1] == '+' || arg[exponential_pos + 1] == '-')
|
||||
scale = std::stoi(arg.substr(exponential_pos + 2, arg.length()));
|
||||
else
|
||||
scale = std::stoi(arg.substr(exponential_pos + 1, arg.length()));
|
||||
|
||||
out = std::format("toDecimal128({}::String,{})", arg, scale);
|
||||
return true;
|
||||
}
|
||||
|
||||
if (const auto dot_pos = arg.find('.'); dot_pos != String::npos)
|
||||
{
|
||||
const auto length = static_cast<int>(std::ssize(arg.substr(0, dot_pos - 1)));
|
||||
scale = std::max(precision - length, 0);
|
||||
}
|
||||
if (is_string)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse String as decimal Literal: {}", fn_name);
|
||||
|
||||
if (scale < 0 || Poco::toUpper(arg) == "NULL")
|
||||
out = "NULL";
|
||||
else
|
||||
out = std::format("toDecimal128({}::String,{})", arg, scale);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
77
src/Parsers/Kusto/KustoFunctions/KQLDataTypeFunctions.h
Normal file
77
src/Parsers/Kusto/KustoFunctions/KQLDataTypeFunctions.h
Normal file
@ -0,0 +1,77 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
namespace DB
|
||||
{
|
||||
class DatatypeBool : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "bool(),boolean()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DatatypeDatetime : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "datetime(),date()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DatatypeDynamic : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "dynamic()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DatatypeGuid : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "guid()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DatatypeInt : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "int()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DatatypeLong : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "long()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DatatypeReal : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "real(),double()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DatatypeString : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "string()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DatatypeTimespan : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "timespan(), time()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DatatypeDecimal : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "decimal()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
}
|
789
src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.cpp
Normal file
789
src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.cpp
Normal file
@ -0,0 +1,789 @@
|
||||
#include <format>
|
||||
#include <regex>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLIPFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLStringFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
#include "Poco/String.h"
|
||||
namespace DB::ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool TimeSpan::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool Ago::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
if (pos->type != TokenType::ClosingRoundBracket)
|
||||
{
|
||||
const auto offset = getConvertedArgument(fn_name, pos);
|
||||
out = std::format("now64(9,'UTC') - {}", offset);
|
||||
}
|
||||
else
|
||||
out = "now64(9,'UTC')";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool DatetimeAdd::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String period = getConvertedArgument(fn_name, pos);
|
||||
//remove quotes from period.
|
||||
trim(period);
|
||||
if (period.front() == '\"' || period.front() == '\'')
|
||||
{
|
||||
//period.remove
|
||||
period.erase(0, 1); // erase the first quote
|
||||
period.erase(period.size() - 1); // erase the last quote
|
||||
}
|
||||
++pos;
|
||||
const String offset = getConvertedArgument(fn_name, pos);
|
||||
++pos;
|
||||
const String datetime = getConvertedArgument(fn_name, pos);
|
||||
|
||||
out = std::format("date_add({}, {}, {})", period, offset, datetime);
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
bool DatetimePart::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String part = Poco::toUpper(getConvertedArgument(fn_name, pos));
|
||||
trim(part);
|
||||
if (part.front() == '\"' || part.front() == '\'')
|
||||
{
|
||||
//period.remove
|
||||
part.erase(0, 1); // erase the first quote
|
||||
part.erase(part.size() - 1); // erase the last quote
|
||||
}
|
||||
String date;
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
date = getConvertedArgument(fn_name, pos);
|
||||
}
|
||||
String format;
|
||||
|
||||
if (part == "YEAR")
|
||||
format = "%G";
|
||||
else if (part == "QUARTER")
|
||||
format = "%Q";
|
||||
else if (part == "MONTH")
|
||||
format = "%m";
|
||||
else if (part == "WEEK_OF_YEAR")
|
||||
format = "%V";
|
||||
else if (part == "DAY")
|
||||
format = "%e";
|
||||
else if (part == "DAYOFYEAR")
|
||||
format = "%j";
|
||||
else if (part == "HOUR")
|
||||
format = "%H";
|
||||
else if (part == "MINUTE")
|
||||
format = "%i";
|
||||
else if (part == "SECOND")
|
||||
format = "%S";
|
||||
else
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Unexpected argument {} for {}", part, fn_name);
|
||||
|
||||
out = std::format("formatDateTime({}, '{}')", date, format);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool DatetimeDiff::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
String arguments;
|
||||
|
||||
arguments = arguments + getConvertedArgument(fn_name, pos) + ",";
|
||||
++pos;
|
||||
arguments = arguments + getConvertedArgument(fn_name, pos) + ",";
|
||||
++pos;
|
||||
arguments = arguments + getConvertedArgument(fn_name, pos);
|
||||
|
||||
out = std::format("DateDiff({}) * -1", arguments);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool DayOfMonth::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "toDayOfMonth");
|
||||
}
|
||||
|
||||
bool DayOfWeek::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
const String datetime_str = getConvertedArgument(fn_name, pos);
|
||||
|
||||
out = std::format("concat((toDayOfWeek({})%7)::String, '.00:00:00')", datetime_str);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool DayOfYear::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "toDayOfYear");
|
||||
}
|
||||
|
||||
bool EndOfMonth::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String datetime_str = getConvertedArgument(fn_name, pos);
|
||||
String offset = "0";
|
||||
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
offset = getConvertedArgument(fn_name, pos);
|
||||
if (offset.empty())
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Number of arguments do not match in function: {}", fn_name);
|
||||
}
|
||||
out = std::format(
|
||||
"toDateTime(toLastDayOfMonth(toDateTime({}, 9, 'UTC') + toIntervalMonth({})), 9, 'UTC') + toIntervalHour(23) + "
|
||||
"toIntervalMinute(59) + toIntervalSecond(60) - toIntervalMicrosecond(1)",
|
||||
datetime_str,
|
||||
toString(offset));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool EndOfDay::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String datetime_str = getConvertedArgument(fn_name, pos);
|
||||
String offset = "0";
|
||||
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
offset = getConvertedArgument(fn_name, pos);
|
||||
}
|
||||
out = std::format(
|
||||
"toDateTime(toStartOfDay({}),9,'UTC') + (INTERVAL {} +1 DAY) - (INTERVAL 1 microsecond)", datetime_str, toString(offset));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool EndOfWeek::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String datetime_str = getConvertedArgument(fn_name, pos);
|
||||
String offset = "0";
|
||||
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
offset = getConvertedArgument(fn_name, pos);
|
||||
}
|
||||
out = std::format(
|
||||
"toDateTime(toStartOfDay({}),9,'UTC') + (INTERVAL {} +1 WEEK) - (INTERVAL 1 microsecond)", datetime_str, toString(offset));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool EndOfYear::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String datetime_str = getConvertedArgument(fn_name, pos);
|
||||
|
||||
if (datetime_str.empty())
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Number of arguments do not match in function: {}", fn_name);
|
||||
|
||||
String offset = "0";
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
offset = getConvertedArgument(fn_name, pos);
|
||||
if (offset.empty())
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Number of arguments do not match in function: {}", fn_name);
|
||||
offset.erase(remove(offset.begin(), offset.end(), ' '), offset.end());
|
||||
}
|
||||
|
||||
out = std::format(
|
||||
"(((((toDateTime(toString(toLastDayOfMonth(toDateTime({0}, 9, 'UTC') + toIntervalYear({1}) + toIntervalMonth(12 - "
|
||||
"toInt8(substring(toString(toDateTime({0}, 9, 'UTC')), 6, 2))))), 9, 'UTC') + toIntervalHour(23)) + toIntervalMinute(59)) + "
|
||||
"toIntervalSecond(60)) - toIntervalMicrosecond(1)))",
|
||||
datetime_str,
|
||||
toString(offset));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool FormatDateTime::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
String formatspecifier;
|
||||
++pos;
|
||||
const auto datetime = getConvertedArgument(fn_name, pos);
|
||||
++pos;
|
||||
auto format = getConvertedArgument(fn_name, pos);
|
||||
trim(format);
|
||||
|
||||
//remove quotes and end space from format argument.
|
||||
if (format.front() == '\"' || format.front() == '\'')
|
||||
{
|
||||
format.erase(0, 1); // erase the first quote
|
||||
format.erase(format.size() - 1); // erase the last quote
|
||||
}
|
||||
|
||||
std::vector<String> res;
|
||||
getTokens(format, res);
|
||||
std::string::size_type i = 0;
|
||||
size_t decimal = 0;
|
||||
while (i < format.size())
|
||||
{
|
||||
char c = format[i];
|
||||
if (!isalpha(c))
|
||||
{
|
||||
//delimiter
|
||||
if (c == ' ' || c == '-' || c == '_' || c == '[' || c == ']' || c == '/' || c == ',' || c == '.' || c == ':')
|
||||
formatspecifier = formatspecifier + c;
|
||||
else
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Invalid format delimiter in function: {}", fn_name);
|
||||
++i;
|
||||
}
|
||||
else
|
||||
{
|
||||
//format specifier
|
||||
String arg = res.back();
|
||||
|
||||
if (arg == "y" || arg == "yy")
|
||||
formatspecifier = formatspecifier + "%y";
|
||||
else if (arg == "yyyy")
|
||||
formatspecifier = formatspecifier + "%Y";
|
||||
else if (arg == "M" || arg == "MM")
|
||||
formatspecifier = formatspecifier + "%m";
|
||||
else if (arg == "s" || arg == "ss")
|
||||
formatspecifier = formatspecifier + "%S";
|
||||
else if (arg == "m" || arg == "mm")
|
||||
formatspecifier = formatspecifier + "%i";
|
||||
else if (arg == "h" || arg == "hh")
|
||||
formatspecifier = formatspecifier + "%h";
|
||||
else if (arg == "H" || arg == "HH")
|
||||
formatspecifier = formatspecifier + "%H";
|
||||
else if (arg == "d")
|
||||
formatspecifier = formatspecifier + "%e";
|
||||
else if (arg == "dd")
|
||||
formatspecifier = formatspecifier + "%d";
|
||||
else if (arg == "tt")
|
||||
formatspecifier = formatspecifier + "%p";
|
||||
else if (arg.starts_with('f') || arg.starts_with('F'))
|
||||
decimal = arg.size();
|
||||
else
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Format specifier {} in function: {} is not supported", arg, fn_name);
|
||||
res.pop_back();
|
||||
i = i + arg.size();
|
||||
}
|
||||
}
|
||||
if (decimal > 0 && formatspecifier.find('.') != String::npos)
|
||||
{
|
||||
out = std::format(
|
||||
"concat("
|
||||
"substring(toString(formatDateTime({0} , '{1}')),1, position(toString(formatDateTime({0},'{1}')),'.')) ,"
|
||||
"substring(substring(toString({0}), position(toString({0}),'.')+1),1,{2}),"
|
||||
"substring(toString(formatDateTime({0},'{1}')), position(toString(formatDateTime({0},'{1}')), '.')+1, length "
|
||||
"(toString(formatDateTime({0},'{1}'))))) ",
|
||||
datetime,
|
||||
formatspecifier,
|
||||
decimal);
|
||||
}
|
||||
else
|
||||
out = std::format("formatDateTime({0},'{1}')", datetime, formatspecifier);
|
||||
|
||||
return true;
|
||||
}
|
||||
bool FormatTimeSpan::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
String formatspecifier;
|
||||
++pos;
|
||||
const auto datetime = getConvertedArgument(fn_name, pos);
|
||||
++pos;
|
||||
auto format = getConvertedArgument(fn_name, pos);
|
||||
size_t decimal = 0;
|
||||
trim(format);
|
||||
//remove quotes and end space from format argument.
|
||||
if (format.front() == '\"' || format.front() == '\'')
|
||||
{
|
||||
format.erase(0, 1); // erase the first quote
|
||||
format.erase(format.size() - 1); // erase the last quote
|
||||
}
|
||||
std::vector<String> res;
|
||||
getTokens(format, res);
|
||||
size_t pad = 0;
|
||||
std::string::size_type i = 0;
|
||||
|
||||
|
||||
bool is_day_in_format = false;
|
||||
String day_val = std::to_string(std::stoi(datetime) / 86400);
|
||||
bool is_hour_zero = std::stoi(datetime) % 86400 > 3600 ? false : true;
|
||||
|
||||
while (i < format.size())
|
||||
{
|
||||
char c = format[i];
|
||||
if (!isalpha(c))
|
||||
{
|
||||
//delimiter
|
||||
if (c == ' ' || c == '-' || c == '_' || c == '[' || c == ']' || c == '/' || c == ',' || c == '.' || c == ':')
|
||||
formatspecifier = formatspecifier + c;
|
||||
else
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Invalid format delimiter in function: {}", fn_name);
|
||||
++i;
|
||||
}
|
||||
else
|
||||
{
|
||||
//format specifier
|
||||
String arg = res.back();
|
||||
|
||||
if (arg == "s" || arg == "ss")
|
||||
formatspecifier = formatspecifier + "%S";
|
||||
else if (arg == "m" || arg == "mm")
|
||||
formatspecifier = formatspecifier + "%i";
|
||||
else if (arg == "h" || arg == "hh")
|
||||
{
|
||||
if (is_hour_zero) //To handle the CH limit for 12hr format(01-12). If not handled, 1.00:00:00 returned as 1.12:00:00(in 12 hr format)
|
||||
formatspecifier = formatspecifier + "%h";
|
||||
else
|
||||
formatspecifier = formatspecifier + "%H";
|
||||
}
|
||||
else if (arg == "H" || arg == "HH")
|
||||
formatspecifier = formatspecifier + "%H";
|
||||
else if (arg.starts_with('d')) //&& arg.size() >2)
|
||||
{
|
||||
pad = std::max(arg.size(), day_val.length());
|
||||
is_day_in_format = true;
|
||||
}
|
||||
else if (arg.starts_with('f') || arg.starts_with('F'))
|
||||
decimal = arg.size();
|
||||
else
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Format specifier {} in function:{} is not supported", arg, fn_name);
|
||||
res.pop_back();
|
||||
i = i + arg.size();
|
||||
}
|
||||
}
|
||||
auto last_delim = formatspecifier.substr(formatspecifier.length() - 1, formatspecifier.length());
|
||||
|
||||
if (!is_day_in_format)
|
||||
{
|
||||
if (decimal > 0)
|
||||
{
|
||||
if (format.substr(format.length() - decimal - 1, 1) == last_delim)
|
||||
out = std::format(
|
||||
"concat(substring(toString(formatDateTime(toDateTime64({0},9,'UTC') ,'{1}')),1, length(toString(formatDateTime("
|
||||
"toDateTime64({0},9,'UTC'),'{1}'))) - position("
|
||||
"reverse(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}'))),'{3}')+1),substring(SUBSTRING(toString("
|
||||
"toDateTime64({0},9,'UTC')),position(toString(toDateTime64({0},9,'UTC')),'.')+1),1,{2}))",
|
||||
datetime,
|
||||
formatspecifier,
|
||||
decimal,
|
||||
last_delim);
|
||||
else
|
||||
out = std::format(
|
||||
"concat(substring(toString(formatDateTime(toDateTime64({0},9,'UTC') ,'{1}')),1, length(toString(formatDateTime("
|
||||
"toDateTime64({0},9,'UTC'),'{1}'))) - position("
|
||||
"reverse(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}'))),'{3}')),substring(SUBSTRING(toString(toDateTime64({"
|
||||
"0},9,'UTC')),position(toString(toDateTime64({0},9,'UTC')),'.')+1),1,{2}))",
|
||||
datetime,
|
||||
formatspecifier,
|
||||
decimal,
|
||||
last_delim);
|
||||
}
|
||||
else
|
||||
out = std::format("formatDateTime(toDateTime64({0},9,'UTC'),'{1}')", datetime, formatspecifier);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (decimal > 0)
|
||||
{
|
||||
if (format.substr(format.length() - decimal - 1, 1) == last_delim)
|
||||
out = std::format(
|
||||
"concat(leftPad('{5}', {3},'0'),substring(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}')),1,"
|
||||
"length(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}'))) - position("
|
||||
"reverse(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}'))),'{4}') "
|
||||
"+1),substring(SUBSTRING(toString(toDateTime64({0},9,'UTC')),position(toString(toDateTime64({0},9,'UTC')),'.')+1),1,{2}"
|
||||
"))",
|
||||
datetime,
|
||||
formatspecifier,
|
||||
decimal,
|
||||
pad,
|
||||
last_delim,
|
||||
day_val);
|
||||
else
|
||||
out = std::format(
|
||||
"concat(leftPad('{5}', {3}, '0'),substring(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}')),1,"
|
||||
"length(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}'))) - position("
|
||||
"reverse(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}'))),'{4}')),substring(SUBSTRING(toString(toDateTime64({"
|
||||
"0},9,'UTC')),position(toString(toDateTime64({0},9,'UTC')),'.')+1),1,{2}),substring(toString(formatDateTime("
|
||||
"toDateTime64({0},9,'UTC'),'{1}')),position(toString(formatDateTime("
|
||||
"toDateTime64({0},9,'UTC'),'{1}')),'{4}'),length(toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}')))))",
|
||||
datetime,
|
||||
formatspecifier,
|
||||
decimal,
|
||||
pad,
|
||||
last_delim,
|
||||
day_val);
|
||||
}
|
||||
else if (decimal == 0)
|
||||
out = std::format(
|
||||
"concat(leftPad('{3}',{2},'0'),toString(formatDateTime(toDateTime64({0},9,'UTC'),'{1}')))",
|
||||
datetime,
|
||||
formatspecifier,
|
||||
pad,
|
||||
day_val);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool GetMonth::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "toMonth");
|
||||
}
|
||||
|
||||
bool GetYear::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "toYear");
|
||||
}
|
||||
|
||||
bool HoursOfDay::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "toHour");
|
||||
}
|
||||
|
||||
bool MakeTimeSpan::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String datetime_str;
|
||||
String hour;
|
||||
String day;
|
||||
String minute;
|
||||
String second;
|
||||
int arg_count = 0;
|
||||
std::vector<String> args;
|
||||
while (!pos->isEnd() && pos->type != TokenType::ClosingRoundBracket)
|
||||
{
|
||||
String arg = getConvertedArgument(fn_name, pos);
|
||||
args.insert(args.begin(), arg);
|
||||
if (pos->type == TokenType::Comma)
|
||||
++pos;
|
||||
++arg_count;
|
||||
}
|
||||
|
||||
if (arg_count < 2 || arg_count > 4)
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "argument count out of bound in function: {}", fn_name);
|
||||
|
||||
if (arg_count == 2)
|
||||
{
|
||||
hour = args.back();
|
||||
args.pop_back();
|
||||
minute = args.back();
|
||||
args.pop_back();
|
||||
datetime_str = hour + ":" + minute;
|
||||
}
|
||||
else if (arg_count == 3)
|
||||
{
|
||||
hour = args.back();
|
||||
args.pop_back();
|
||||
minute = args.back();
|
||||
args.pop_back();
|
||||
second = args.back();
|
||||
args.pop_back();
|
||||
|
||||
datetime_str = hour + ":" + minute + ":" + second;
|
||||
}
|
||||
else if (arg_count == 4)
|
||||
{
|
||||
day = args.back();
|
||||
args.pop_back();
|
||||
hour = args.back();
|
||||
args.pop_back();
|
||||
minute = args.back();
|
||||
args.pop_back();
|
||||
second = args.back();
|
||||
args.pop_back();
|
||||
|
||||
datetime_str = hour + ":" + minute + ":" + second;
|
||||
day = day + ".";
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "argument count out of bound in function: {}", fn_name);
|
||||
|
||||
//Add dummy yyyy-mm-dd to parse datetime in CH
|
||||
datetime_str = "0000-00-00 " + datetime_str;
|
||||
|
||||
out = std::format(
|
||||
"CONCAT('{}',toString(SUBSTRING(toString(toTime(parseDateTime64BestEffortOrNull('{}', 9,'UTC'))),12)))", day, datetime_str);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool MakeDateTime::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String arguments;
|
||||
int arg_count = 0;
|
||||
|
||||
while (!pos->isEnd() && pos->type != TokenType::ClosingRoundBracket)
|
||||
{
|
||||
String arg = getConvertedArgument(fn_name, pos);
|
||||
if (pos->type == TokenType::Comma)
|
||||
++pos;
|
||||
arguments = arguments + arg + ",";
|
||||
++arg_count;
|
||||
}
|
||||
|
||||
if (arg_count < 1 || arg_count > 7)
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "argument count out of bound in function: {}", fn_name);
|
||||
|
||||
if (arg_count < 7)
|
||||
{
|
||||
for (int i = arg_count; i < 7; ++i)
|
||||
arguments = arguments + "0,";
|
||||
}
|
||||
|
||||
arguments = arguments + "7,'UTC'";
|
||||
out = std::format("makeDateTime64({})", arguments);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Now::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
if (pos->type != TokenType::ClosingRoundBracket)
|
||||
{
|
||||
const auto offset = getConvertedArgument(fn_name, pos);
|
||||
out = std::format("now64(9,'UTC') + {}", offset);
|
||||
}
|
||||
else
|
||||
out = "now64(9,'UTC')";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool StartOfDay::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String datetime_str = getConvertedArgument(fn_name, pos);
|
||||
String offset = "0";
|
||||
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
offset = getConvertedArgument(fn_name, pos);
|
||||
}
|
||||
out = std::format("date_add(DAY,{}, parseDateTime64BestEffortOrNull(toString((toStartOfDay({}))), 9, 'UTC')) ", offset, datetime_str);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool StartOfMonth::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String datetime_str = getConvertedArgument(fn_name, pos);
|
||||
String offset = "0";
|
||||
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
offset = getConvertedArgument(fn_name, pos);
|
||||
}
|
||||
out = std::format(
|
||||
"date_add(MONTH,{}, parseDateTime64BestEffortOrNull(toString((toStartOfMonth({}))), 9, 'UTC')) ", offset, datetime_str);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool StartOfWeek::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String datetime_str = getConvertedArgument(fn_name, pos);
|
||||
String offset = "0";
|
||||
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
offset = getConvertedArgument(fn_name, pos);
|
||||
}
|
||||
out = std::format(
|
||||
"date_add(Week,{}, parseDateTime64BestEffortOrNull(toString((toStartOfWeek({}))), 9, 'UTC')) ", offset, datetime_str);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool StartOfYear::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String datetime_str = getConvertedArgument(fn_name, pos);
|
||||
String offset = "0";
|
||||
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
offset = getConvertedArgument(fn_name, pos);
|
||||
}
|
||||
out = std::format(
|
||||
"date_add(YEAR,{}, parseDateTime64BestEffortOrNull(toString((toStartOfYear({}, 'UTC'))), 9, 'UTC'))", offset, datetime_str);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool UnixTimeMicrosecondsToDateTime::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String value = getConvertedArgument(fn_name, pos);
|
||||
|
||||
out = std::format("fromUnixTimestamp64Micro({},'UTC')", value);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool UnixTimeMillisecondsToDateTime::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String value = getConvertedArgument(fn_name, pos);
|
||||
|
||||
out = std::format("fromUnixTimestamp64Milli({},'UTC')", value);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool UnixTimeNanosecondsToDateTime::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String value = getConvertedArgument(fn_name, pos);
|
||||
|
||||
out = std::format("fromUnixTimestamp64Nano({},'UTC')", value);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool UnixTimeSecondsToDateTime::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} accepts only long, int and double type of arguments", fn_name);
|
||||
|
||||
String expression = getConvertedArgument(fn_name, pos);
|
||||
out = std::format(
|
||||
" if(toTypeName({0}) = 'Int64' OR toTypeName({0}) = 'Int32'OR toTypeName({0}) = 'Float64' OR toTypeName({0}) = 'UInt32' OR "
|
||||
"toTypeName({0}) = 'UInt64', toDateTime64({0}, 9, 'UTC'), toDateTime64(throwIf(true, '{1} only accepts Int, Long and double type "
|
||||
"of arguments'), 9, 'UTC'))",
|
||||
expression,
|
||||
fn_name);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool WeekOfYear::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
const String time_str = getConvertedArgument(fn_name, pos);
|
||||
out = std::format("toWeek({},3,'UTC')", time_str);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool MonthOfYear::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "toMonth");
|
||||
}
|
||||
|
||||
}
|
250
src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.h
Normal file
250
src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.h
Normal file
@ -0,0 +1,250 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class TimeSpan : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "timespan()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
/*
|
||||
class DateTime : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "datetime()"; }
|
||||
bool convertImpl(String &out,IParser::Pos &pos) override;
|
||||
};*/
|
||||
|
||||
|
||||
class Ago : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "ago()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DatetimeAdd : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "datetime_add()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DatetimePart : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "datetime_part()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DatetimeDiff : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "datetime_diff()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DayOfMonth : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "dayofmonth()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DayOfWeek : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "dayofweek()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class DayOfYear : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "dayofyear()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class EndOfDay : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "endofday()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class EndOfMonth : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "endofmonth()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class EndOfWeek : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "endofweek()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class EndOfYear : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "endofyear()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class FormatDateTime : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "format_datetime()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class FormatTimeSpan : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "format_timespan()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class GetMonth : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "getmonth()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class GetYear : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "getyear()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class HoursOfDay : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "hourofday()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class MakeTimeSpan : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "make_timespan()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class MakeDateTime : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "make_datetime()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Now : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "now()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class StartOfDay : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "startofday()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class StartOfMonth : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "startofmonth()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class StartOfWeek : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "startofweek()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class StartOfYear : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "startofyear()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class UnixTimeMicrosecondsToDateTime : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "unixtime_microseconds_todatetime()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class UnixTimeMillisecondsToDateTime : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "unixtime_milliseconds_todatetime()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class UnixTimeNanosecondsToDateTime : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "unixtime_nanoseconds_todatetime()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class UnixTimeSecondsToDateTime : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "unixtime_seconds_todatetime()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class WeekOfYear : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "week_of_year()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class MonthOfYear : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "monthofyear()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
void inline getTokens(String format, std::vector<String> & res)
|
||||
{
|
||||
String str = format;
|
||||
String token;
|
||||
auto pos = str.find_first_not_of("abcdefghijklmnopqrstuvwxyzQWERTYUIOPASDFGHJKLZXCVBNM");
|
||||
while (pos != String::npos)
|
||||
{
|
||||
if (pos != 0)
|
||||
{
|
||||
// Found a token
|
||||
token = str.substr(0, pos);
|
||||
res.insert(res.begin(), token);
|
||||
}
|
||||
str.erase(0, pos + 1); // Always remove pos+1 to get rid of delimiter
|
||||
pos = str.find_first_not_of("abcdefghijklmnopqrstuvwxyzQWERTYUIOPASDFGHJKLZXCVBNM");
|
||||
}
|
||||
// Cover the last (or only) token
|
||||
if (str.length() > 0)
|
||||
{
|
||||
token = str;
|
||||
res.insert(res.begin(), token);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
377
src/Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.cpp
Normal file
377
src/Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.cpp
Normal file
@ -0,0 +1,377 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.h>
|
||||
|
||||
#include <format>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
bool ArrayConcat::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "arrayConcat");
|
||||
}
|
||||
|
||||
bool ArrayIif::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto conditions = getArgument(function_name, pos);
|
||||
const auto if_true = getArgument(function_name, pos);
|
||||
const auto if_false = getArgument(function_name, pos);
|
||||
|
||||
out = std::format(
|
||||
"arrayMap(x -> multiIf(toTypeName(x.1) = 'String', null, toInt64(x.1) != 0, x.2, x.3), "
|
||||
"arrayZip({0}, arrayResize({1}, length({0}), null), arrayResize({2}, length({0}), null)))",
|
||||
conditions,
|
||||
if_true,
|
||||
if_false);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ArrayIndexOf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
const auto array = getArgument(fn_name, pos);
|
||||
const auto needle = getArgument(fn_name, pos);
|
||||
out = "minus(indexOf(" + array + ", " + needle + "), 1)";
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ArrayLength::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "length");
|
||||
}
|
||||
|
||||
bool ArrayReverse::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto array = getArgument(function_name, pos);
|
||||
out = std::format("if(throwIf(not startsWith(toTypeName({0}), 'Array'), 'Only arrays are supported'), [], reverse({0}))", array);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ArrayRotateLeft::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto array = getArgument(function_name, pos);
|
||||
const auto count = getArgument(function_name, pos);
|
||||
out = std::format(
|
||||
"arrayMap(x -> {0}[moduloOrZero(x + length({0}) + moduloOrZero({1}, toInt64(length({0}))), length({0})) + 1], "
|
||||
"range(0, length({0})))",
|
||||
array,
|
||||
count);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ArrayRotateRight::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto array = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto count = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
out = kqlCallToExpression("array_rotate_left", {array, "-1 * " + count}, pos.max_depth);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ArrayShiftLeft::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto array = getArgument(function_name, pos);
|
||||
const auto count = getArgument(function_name, pos);
|
||||
const auto fill = getOptionalArgument(function_name, pos);
|
||||
out = std::format(
|
||||
"arrayResize(if({1} > 0, arraySlice({0}, {1} + 1), arrayConcat(arrayWithConstant(abs({1}), fill_value_{3}), {0})), "
|
||||
"length({0}), if(isNull({2}) and (extract(toTypeName({0}), 'Array\\((.*)\\)') as element_type_{3}) = 'String', "
|
||||
"defaultValueOfTypeName(if(element_type_{3} = 'Nothing', 'Nullable(Nothing)', element_type_{3})), {2}) as fill_value_{3})",
|
||||
array,
|
||||
count,
|
||||
fill ? *fill : "null",
|
||||
generateUniqueIdentifier());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ArrayShiftRight::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto array = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto count = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto fill = getOptionalArgument(function_name, pos, ArgumentState::Raw);
|
||||
|
||||
const auto negated_count = "-1 * " + count;
|
||||
out = kqlCallToExpression(
|
||||
"array_shift_left",
|
||||
fill ? std::initializer_list<std::string_view>{array, negated_count, *fill}
|
||||
: std::initializer_list<std::string_view>{array, negated_count},
|
||||
pos.max_depth);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ArraySlice::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto array = getArgument(function_name, pos);
|
||||
const auto start = getArgument(function_name, pos);
|
||||
const auto end = getArgument(function_name, pos);
|
||||
|
||||
out = std::format(
|
||||
"arraySlice({0}, plus(1, if({1} >= 0, {1}, arrayMax([-length({0}), {1}]) + length({0}))) as offset_{3}, "
|
||||
" plus(1, if({2} >= 0, {2}, arrayMax([-length({0}), {2}]) + length({0}))) - offset_{3} + 1)",
|
||||
array,
|
||||
start,
|
||||
end,
|
||||
generateUniqueIdentifier());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ArraySortAsc::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "kql_array_sort_asc");
|
||||
}
|
||||
|
||||
bool ArraySortDesc::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "kql_array_sort_desc");
|
||||
}
|
||||
|
||||
bool ArraySplit::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto array = getArgument(function_name, pos);
|
||||
const auto indices = getArgument(function_name, pos);
|
||||
|
||||
out = std::format(
|
||||
"if(empty(arrayMap(x -> if(x >= 0, x, arrayMax([0, x + length({0})::Int64])), flatten([{1}])) as indices_{2}), [{0}], "
|
||||
"arrayConcat([arraySlice({0}, 1, indices_{2}[1])], arrayMap(i -> arraySlice({0}, indices_{2}[i] + 1, "
|
||||
"if(i = length(indices_{2}), length({0})::Int64, indices_{2}[i + 1]::Int64) - indices_{2}[i]), "
|
||||
"range(1, length(indices_{2}) + 1))))",
|
||||
array,
|
||||
indices,
|
||||
generateUniqueIdentifier());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ArraySum::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "arraySum");
|
||||
}
|
||||
|
||||
bool BagKeys::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool BagMerge::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool BagRemoveKeys::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool JaccardIndex::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto lhs = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto rhs = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
out = std::format(
|
||||
"divide(length({0}), length({1}))",
|
||||
kqlCallToExpression("set_intersect", {lhs, rhs}, pos.max_depth),
|
||||
kqlCallToExpression("set_union", {lhs, rhs}, pos.max_depth));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Pack::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool PackAll::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool PackArray::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "array", {1, Interval::max_bound});
|
||||
}
|
||||
|
||||
bool Repeat::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
String value = getArgument(function_name, pos);
|
||||
String count = getArgument(function_name, pos);
|
||||
|
||||
value.erase(remove(value.begin(), value.end(), ' '), value.end());
|
||||
count.erase(remove(count.begin(), count.end(), ' '), count.end());
|
||||
|
||||
if (count.empty())
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "number of arguments do not match in function: {}", function_name);
|
||||
else
|
||||
out = "if(" + count + " < 0, [NULL], " + std::format("arrayWithConstant(abs({1}), {0}))", value, count);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool SetDifference::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto lhs = getArgument(function_name, pos);
|
||||
const auto rhs = std::invoke(
|
||||
[&function_name, &pos]
|
||||
{
|
||||
std::vector<String> arrays{getArgument(function_name, pos, ArgumentState::Raw)};
|
||||
while (auto next_array = getOptionalArgument(function_name, pos, ArgumentState::Raw))
|
||||
arrays.push_back(*next_array);
|
||||
|
||||
return kqlCallToExpression("set_union", std::vector<std::string_view>(arrays.cbegin(), arrays.cend()), pos.max_depth);
|
||||
});
|
||||
|
||||
out = std::format("arrayFilter(x -> not has({1}, x), arrayDistinct({0}))", lhs, rhs);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool SetHasElement::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "has");
|
||||
}
|
||||
|
||||
bool SetIntersect::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "arrayIntersect");
|
||||
}
|
||||
|
||||
bool SetUnion::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
if (!directMapping(out, pos, "arrayConcat"))
|
||||
return false;
|
||||
|
||||
out = std::format("arrayDistinct({0})", out);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool TreePath::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool Zip::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto arguments = std::invoke(
|
||||
[&function_name, &pos]
|
||||
{
|
||||
std::vector<String> result;
|
||||
while (auto argument = getOptionalArgument(function_name, pos))
|
||||
result.push_back(std::move(*argument));
|
||||
|
||||
return result;
|
||||
});
|
||||
|
||||
if (const auto size = arguments.size(); size < 2 || size > 16)
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Between 2 and 16 arguments are expected, but {} were provided", size);
|
||||
|
||||
const auto unique_identifier = generateUniqueIdentifier();
|
||||
const auto resized_arguments = std::invoke(
|
||||
[&arguments, &unique_identifier]
|
||||
{
|
||||
String lengths;
|
||||
for (int i = 0; i < std::ssize(arguments); ++i)
|
||||
{
|
||||
lengths.append(i > 0 ? ", " : "");
|
||||
lengths.append(std::format(
|
||||
"length(if(match(toTypeName({0}), 'Array\\(Nullable\\(.*\\)\\)'), {0}, "
|
||||
"cast({0}, concat('Array(Nullable(', extract(toTypeName({0}), 'Array\\((.*)\\)'), '))'))) as arg{1}_{2})",
|
||||
arguments[i],
|
||||
i,
|
||||
unique_identifier));
|
||||
}
|
||||
|
||||
auto result = std::format("arrayResize(arg0_{1}, arrayMax([{0}]) as max_length_{1}, null)", lengths, unique_identifier);
|
||||
for (int i = 1; i < std::ssize(arguments); ++i)
|
||||
result.append(std::format(", arrayResize(arg{0}_{1}, max_length_{1}, null)", i, unique_identifier));
|
||||
|
||||
return result;
|
||||
});
|
||||
|
||||
out = std::format("arrayMap(t -> [untuple(t)], arrayZip({0}))", resized_arguments);
|
||||
|
||||
return true;
|
||||
}
|
||||
}
|
203
src/Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.h
Normal file
203
src/Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.h
Normal file
@ -0,0 +1,203 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
namespace DB
|
||||
{
|
||||
class ArrayConcat : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array_concat()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ArrayIif : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array_iif()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ArrayIndexOf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array_index_of()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ArrayLength : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array_length()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ArrayReverse : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array_reverse()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ArrayRotateLeft : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array_rotate_left()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ArrayRotateRight : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array_rotate_right()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ArrayShiftLeft : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array_shift_left()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ArrayShiftRight : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array_shift_right()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ArraySlice : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array_slice()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ArraySortAsc : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array_sort_asc()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ArraySortDesc : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array_sort_desc()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ArraySplit : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array_split()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ArraySum : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "array_sum()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class BagKeys : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "bag_keys()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class BagMerge : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "bag_merge()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class BagRemoveKeys : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "bag_remove_keys()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class JaccardIndex : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "jaccard_index()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Pack : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "pack()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class PackAll : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "pack_all()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class PackArray : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "pack_array()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Repeat : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "repeat()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SetDifference : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "set_difference()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SetHasElement : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "set_has_element()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SetIntersect : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "set_intersect()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SetUnion : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "set_union()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class TreePath : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "treepath()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Zip : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "zip()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
}
|
819
src/Parsers/Kusto/KustoFunctions/KQLFunctionFactory.cpp
Normal file
819
src/Parsers/Kusto/KustoFunctions/KQLFunctionFactory.cpp
Normal file
@ -0,0 +1,819 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDataTypeFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLIPFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLMathematicalFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLStringFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
std::unordered_map<String, KQLFunctionValue> KQLFunctionFactory::kql_functions
|
||||
= {{"ago", KQLFunctionValue::ago},
|
||||
{"datetime_add", KQLFunctionValue::datetime_add},
|
||||
{"datetime_part", KQLFunctionValue::datetime_part},
|
||||
{"datetime_diff", KQLFunctionValue::datetime_diff},
|
||||
{"dayofmonth", KQLFunctionValue::dayofmonth},
|
||||
{"dayofweek", KQLFunctionValue::dayofweek},
|
||||
{"dayofyear", KQLFunctionValue::dayofyear},
|
||||
{"endofday", KQLFunctionValue::endofday},
|
||||
{"endofweek", KQLFunctionValue::endofweek},
|
||||
{"endofyear", KQLFunctionValue::endofyear},
|
||||
{"endofmonth", KQLFunctionValue::endofmonth},
|
||||
|
||||
{"format_datetime", KQLFunctionValue::format_datetime},
|
||||
{"format_timespan", KQLFunctionValue::format_timespan},
|
||||
{"getmonth", KQLFunctionValue::getmonth},
|
||||
{"getyear", KQLFunctionValue::getyear},
|
||||
{"hourofday", KQLFunctionValue::hourofday},
|
||||
{"make_timespan", KQLFunctionValue::make_timespan},
|
||||
{"make_datetime", KQLFunctionValue::make_datetime},
|
||||
{"now", KQLFunctionValue::now},
|
||||
{"startofday", KQLFunctionValue::startofday},
|
||||
{"startofmonth", KQLFunctionValue::startofmonth},
|
||||
{"startofweek", KQLFunctionValue::startofweek},
|
||||
{"startofyear", KQLFunctionValue::startofyear},
|
||||
{"todatetime", KQLFunctionValue::todatetime},
|
||||
{"totimespan", KQLFunctionValue::totimespan},
|
||||
{"unixtime_microseconds_todatetime", KQLFunctionValue::unixtime_microseconds_todatetime},
|
||||
{"unixtime_milliseconds_todatetime", KQLFunctionValue::unixtime_milliseconds_todatetime},
|
||||
{"unixtime_nanoseconds_todatetime", KQLFunctionValue::unixtime_nanoseconds_todatetime},
|
||||
{"unixtime_seconds_todatetime", KQLFunctionValue::unixtime_seconds_todatetime},
|
||||
{"week_of_year", KQLFunctionValue::week_of_year},
|
||||
{"monthofyear", KQLFunctionValue::monthofyear},
|
||||
{"base64_encode_tostring", KQLFunctionValue::base64_encode_tostring},
|
||||
{"base64_encode_fromguid", KQLFunctionValue::base64_encode_fromguid},
|
||||
{"base64_decode_tostring", KQLFunctionValue::base64_decode_tostring},
|
||||
{"base64_decode_toarray", KQLFunctionValue::base64_decode_toarray},
|
||||
{"base64_decode_toguid", KQLFunctionValue::base64_decode_toguid},
|
||||
{"countof", KQLFunctionValue::countof},
|
||||
{"extract", KQLFunctionValue::extract},
|
||||
{"extract_all", KQLFunctionValue::extract_all},
|
||||
{"extract_json", KQLFunctionValue::extract_json},
|
||||
{"extractjson", KQLFunctionValue::extract_json},
|
||||
{"has_any_index", KQLFunctionValue::has_any_index},
|
||||
{"indexof", KQLFunctionValue::indexof},
|
||||
{"isempty", KQLFunctionValue::isempty},
|
||||
{"isnan", KQLFunctionValue::isnan},
|
||||
{"isnotempty", KQLFunctionValue::isnotempty},
|
||||
{"notempty", KQLFunctionValue::isnotempty},
|
||||
{"isnotnull", KQLFunctionValue::isnotnull},
|
||||
{"notnull", KQLFunctionValue::isnotnull},
|
||||
{"isnull", KQLFunctionValue::isnull},
|
||||
{"parse_command_line", KQLFunctionValue::parse_command_line},
|
||||
{"parse_csv", KQLFunctionValue::parse_csv},
|
||||
{"parse_json", KQLFunctionValue::parse_json},
|
||||
{"parse_url", KQLFunctionValue::parse_url},
|
||||
{"parse_urlquery", KQLFunctionValue::parse_urlquery},
|
||||
{"parse_version", KQLFunctionValue::parse_version},
|
||||
{"replace_regex", KQLFunctionValue::replace_regex},
|
||||
{"reverse", KQLFunctionValue::reverse},
|
||||
{"split", KQLFunctionValue::split},
|
||||
{"strcat", KQLFunctionValue::strcat},
|
||||
{"strcat_delim", KQLFunctionValue::strcat_delim},
|
||||
{"strcmp", KQLFunctionValue::strcmp},
|
||||
{"strlen", KQLFunctionValue::strlen},
|
||||
{"strrep", KQLFunctionValue::strrep},
|
||||
{"substring", KQLFunctionValue::substring},
|
||||
{"tolower", KQLFunctionValue::tolower},
|
||||
{"toupper", KQLFunctionValue::toupper},
|
||||
{"translate", KQLFunctionValue::translate},
|
||||
{"trim", KQLFunctionValue::trim},
|
||||
{"trim_end", KQLFunctionValue::trim_end},
|
||||
{"trim_start", KQLFunctionValue::trim_start},
|
||||
{"url_decode", KQLFunctionValue::url_decode},
|
||||
{"url_encode", KQLFunctionValue::url_encode},
|
||||
|
||||
{"array_concat", KQLFunctionValue::array_concat},
|
||||
{"array_iff", KQLFunctionValue::array_iif},
|
||||
{"array_iif", KQLFunctionValue::array_iif},
|
||||
{"array_index_of", KQLFunctionValue::array_index_of},
|
||||
{"array_length", KQLFunctionValue::array_length},
|
||||
{"array_reverse", KQLFunctionValue::array_reverse},
|
||||
{"array_rotate_left", KQLFunctionValue::array_rotate_left},
|
||||
{"array_rotate_right", KQLFunctionValue::array_rotate_right},
|
||||
{"array_shift_left", KQLFunctionValue::array_shift_left},
|
||||
{"array_shift_right", KQLFunctionValue::array_shift_right},
|
||||
{"array_slice", KQLFunctionValue::array_slice},
|
||||
{"array_sort_asc", KQLFunctionValue::array_sort_asc},
|
||||
{"array_sort_desc", KQLFunctionValue::array_sort_desc},
|
||||
{"array_split", KQLFunctionValue::array_split},
|
||||
{"array_sum", KQLFunctionValue::array_sum},
|
||||
{"bag_keys", KQLFunctionValue::bag_keys},
|
||||
{"bag_merge", KQLFunctionValue::bag_merge},
|
||||
{"bag_remove_keys", KQLFunctionValue::bag_remove_keys},
|
||||
{"jaccard_index", KQLFunctionValue::jaccard_index},
|
||||
{"pack", KQLFunctionValue::pack},
|
||||
{"pack_all", KQLFunctionValue::pack_all},
|
||||
{"pack_array", KQLFunctionValue::pack_array},
|
||||
{"repeat", KQLFunctionValue::repeat},
|
||||
{"set_difference", KQLFunctionValue::set_difference},
|
||||
{"set_has_element", KQLFunctionValue::set_has_element},
|
||||
{"set_intersect", KQLFunctionValue::set_intersect},
|
||||
{"set_union", KQLFunctionValue::set_union},
|
||||
{"treepath", KQLFunctionValue::treepath},
|
||||
{"zip", KQLFunctionValue::zip},
|
||||
|
||||
{"tobool", KQLFunctionValue::tobool},
|
||||
{"toboolean", KQLFunctionValue::tobool},
|
||||
{"todouble", KQLFunctionValue::todouble},
|
||||
{"toint", KQLFunctionValue::toint},
|
||||
{"tolong", KQLFunctionValue::tolong},
|
||||
{"toreal", KQLFunctionValue::todouble},
|
||||
{"tostring", KQLFunctionValue::tostring},
|
||||
{"totimespan", KQLFunctionValue::totimespan},
|
||||
{"todecimal", KQLFunctionValue::todecimal},
|
||||
|
||||
{"arg_max", KQLFunctionValue::arg_max},
|
||||
{"arg_min", KQLFunctionValue::arg_min},
|
||||
{"avg", KQLFunctionValue::avg},
|
||||
{"avgif", KQLFunctionValue::avgif},
|
||||
{"binary_all_and", KQLFunctionValue::binary_all_and},
|
||||
{"binary_all_or", KQLFunctionValue::binary_all_or},
|
||||
{"binary_all_xor", KQLFunctionValue::binary_all_xor},
|
||||
{"buildschema", KQLFunctionValue::buildschema},
|
||||
{"count", KQLFunctionValue::count},
|
||||
{"countif", KQLFunctionValue::countif},
|
||||
{"dcount", KQLFunctionValue::dcount},
|
||||
{"dcountif", KQLFunctionValue::dcountif},
|
||||
{"make_bag", KQLFunctionValue::make_bag},
|
||||
{"make_bag_if", KQLFunctionValue::make_bag_if},
|
||||
{"make_list", KQLFunctionValue::make_list},
|
||||
{"make_list_if", KQLFunctionValue::make_list_if},
|
||||
{"make_list_with_nulls", KQLFunctionValue::make_list_with_nulls},
|
||||
{"make_set", KQLFunctionValue::make_set},
|
||||
{"make_set_if", KQLFunctionValue::make_set_if},
|
||||
{"max", KQLFunctionValue::max},
|
||||
{"maxif", KQLFunctionValue::maxif},
|
||||
{"min", KQLFunctionValue::min},
|
||||
{"minif", KQLFunctionValue::minif},
|
||||
{"percentile", KQLFunctionValue::percentile},
|
||||
{"percentilew", KQLFunctionValue::percentilew},
|
||||
{"percentiles", KQLFunctionValue::percentiles},
|
||||
{"percentiles_array", KQLFunctionValue::percentiles_array},
|
||||
{"percentilesw", KQLFunctionValue::percentilesw},
|
||||
{"percentilesw_array", KQLFunctionValue::percentilesw_array},
|
||||
{"stdev", KQLFunctionValue::stdev},
|
||||
{"stdevif", KQLFunctionValue::stdevif},
|
||||
{"sum", KQLFunctionValue::sum},
|
||||
{"sumif", KQLFunctionValue::sumif},
|
||||
{"take_any", KQLFunctionValue::take_any},
|
||||
{"take_anyif", KQLFunctionValue::take_anyif},
|
||||
{"variance", KQLFunctionValue::variance},
|
||||
{"varianceif", KQLFunctionValue::varianceif},
|
||||
|
||||
{"series_fir", KQLFunctionValue::series_fir},
|
||||
{"series_iir", KQLFunctionValue::series_iir},
|
||||
{"series_fit_line", KQLFunctionValue::series_fit_line},
|
||||
{"series_fit_line_dynamic", KQLFunctionValue::series_fit_line_dynamic},
|
||||
{"series_fit_2lines", KQLFunctionValue::series_fit_2lines},
|
||||
{"series_fit_2lines_dynamic", KQLFunctionValue::series_fit_2lines_dynamic},
|
||||
{"series_outliers", KQLFunctionValue::series_outliers},
|
||||
{"series_periods_detect", KQLFunctionValue::series_periods_detect},
|
||||
{"series_periods_validate", KQLFunctionValue::series_periods_validate},
|
||||
{"series_stats_dynamic", KQLFunctionValue::series_stats_dynamic},
|
||||
{"series_stats", KQLFunctionValue::series_stats},
|
||||
{"series_fill_backward", KQLFunctionValue::series_fill_backward},
|
||||
{"series_fill_const", KQLFunctionValue::series_fill_const},
|
||||
{"series_fill_forward", KQLFunctionValue::series_fill_forward},
|
||||
{"series_fill_linear", KQLFunctionValue::series_fill_linear},
|
||||
|
||||
{"ipv4_compare", KQLFunctionValue::ipv4_compare},
|
||||
{"ipv4_is_in_range", KQLFunctionValue::ipv4_is_in_range},
|
||||
{"ipv4_is_match", KQLFunctionValue::ipv4_is_match},
|
||||
{"ipv4_is_private", KQLFunctionValue::ipv4_is_private},
|
||||
{"ipv4_netmask_suffix", KQLFunctionValue::ipv4_netmask_suffix},
|
||||
{"parse_ipv4", KQLFunctionValue::parse_ipv4},
|
||||
{"parse_ipv4_mask", KQLFunctionValue::parse_ipv4_mask},
|
||||
{"ipv6_compare", KQLFunctionValue::ipv6_compare},
|
||||
{"ipv6_is_match", KQLFunctionValue::ipv6_is_match},
|
||||
{"parse_ipv6", KQLFunctionValue::parse_ipv6},
|
||||
{"parse_ipv6_mask", KQLFunctionValue::parse_ipv6_mask},
|
||||
{"format_ipv4", KQLFunctionValue::format_ipv4},
|
||||
{"format_ipv4_mask", KQLFunctionValue::format_ipv4_mask},
|
||||
|
||||
{"binary_and", KQLFunctionValue::binary_and},
|
||||
{"binary_not", KQLFunctionValue::binary_not},
|
||||
{"binary_or", KQLFunctionValue::binary_or},
|
||||
{"binary_shift_left", KQLFunctionValue::binary_shift_left},
|
||||
{"binary_shift_right", KQLFunctionValue::binary_shift_right},
|
||||
{"binary_xor", KQLFunctionValue::binary_xor},
|
||||
{"bitset_count_ones", KQLFunctionValue::bitset_count_ones},
|
||||
|
||||
{"bin", KQLFunctionValue::bin},
|
||||
{"bin_at", KQLFunctionValue::bin_at},
|
||||
|
||||
{"bool", KQLFunctionValue::datatype_bool},
|
||||
{"boolean", KQLFunctionValue::datatype_bool},
|
||||
{"datetime", KQLFunctionValue::datatype_datetime},
|
||||
{"date", KQLFunctionValue::datatype_datetime},
|
||||
{"dynamic", KQLFunctionValue::datatype_dynamic},
|
||||
{"guid", KQLFunctionValue::datatype_guid},
|
||||
{"int", KQLFunctionValue::datatype_int},
|
||||
{"long", KQLFunctionValue::datatype_long},
|
||||
{"real", KQLFunctionValue::datatype_real},
|
||||
{"double", KQLFunctionValue::datatype_real},
|
||||
{"string", KQLFunctionValue::datatype_string},
|
||||
{"timespan", KQLFunctionValue::datatype_timespan},
|
||||
{"time", KQLFunctionValue::datatype_timespan},
|
||||
{"decimal", KQLFunctionValue::datatype_decimal},
|
||||
{"round", KQLFunctionValue::round}
|
||||
};
|
||||
|
||||
|
||||
std::unique_ptr<IParserKQLFunction> KQLFunctionFactory::get(String & kql_function)
|
||||
{
|
||||
if (kql_functions.find(kql_function) == kql_functions.end())
|
||||
return nullptr;
|
||||
|
||||
auto kql_function_id = kql_functions[kql_function];
|
||||
switch (kql_function_id)
|
||||
{
|
||||
case KQLFunctionValue::none:
|
||||
return nullptr;
|
||||
|
||||
case KQLFunctionValue::timespan:
|
||||
return std::make_unique<TimeSpan>();
|
||||
|
||||
case KQLFunctionValue::ago:
|
||||
return std::make_unique<Ago>();
|
||||
|
||||
case KQLFunctionValue::datetime_add:
|
||||
return std::make_unique<DatetimeAdd>();
|
||||
|
||||
case KQLFunctionValue::datetime_part:
|
||||
return std::make_unique<DatetimePart>();
|
||||
|
||||
case KQLFunctionValue::datetime_diff:
|
||||
return std::make_unique<DatetimeDiff>();
|
||||
|
||||
case KQLFunctionValue::dayofmonth:
|
||||
return std::make_unique<DayOfMonth>();
|
||||
|
||||
case KQLFunctionValue::dayofweek:
|
||||
return std::make_unique<DayOfWeek>();
|
||||
|
||||
case KQLFunctionValue::dayofyear:
|
||||
return std::make_unique<DayOfYear>();
|
||||
|
||||
case KQLFunctionValue::endofday:
|
||||
return std::make_unique<EndOfDay>();
|
||||
|
||||
case KQLFunctionValue::endofweek:
|
||||
return std::make_unique<EndOfWeek>();
|
||||
|
||||
case KQLFunctionValue::endofyear:
|
||||
return std::make_unique<EndOfYear>();
|
||||
|
||||
case KQLFunctionValue::endofmonth:
|
||||
return std::make_unique<EndOfMonth>();
|
||||
|
||||
case KQLFunctionValue::monthofyear:
|
||||
return std::make_unique<MonthOfYear>();
|
||||
|
||||
case KQLFunctionValue::format_datetime:
|
||||
return std::make_unique<FormatDateTime>();
|
||||
|
||||
case KQLFunctionValue::format_timespan:
|
||||
return std::make_unique<FormatTimeSpan>();
|
||||
|
||||
case KQLFunctionValue::getmonth:
|
||||
return std::make_unique<GetMonth>();
|
||||
|
||||
case KQLFunctionValue::getyear:
|
||||
return std::make_unique<GetYear>();
|
||||
|
||||
case KQLFunctionValue::hourofday:
|
||||
return std::make_unique<HoursOfDay>();
|
||||
|
||||
case KQLFunctionValue::make_timespan:
|
||||
return std::make_unique<MakeTimeSpan>();
|
||||
|
||||
case KQLFunctionValue::make_datetime:
|
||||
return std::make_unique<MakeDateTime>();
|
||||
|
||||
case KQLFunctionValue::now:
|
||||
return std::make_unique<Now>();
|
||||
|
||||
case KQLFunctionValue::startofday:
|
||||
return std::make_unique<StartOfDay>();
|
||||
|
||||
case KQLFunctionValue::startofmonth:
|
||||
return std::make_unique<StartOfMonth>();
|
||||
|
||||
case KQLFunctionValue::startofweek:
|
||||
return std::make_unique<StartOfWeek>();
|
||||
|
||||
case KQLFunctionValue::startofyear:
|
||||
return std::make_unique<StartOfYear>();
|
||||
|
||||
case KQLFunctionValue::unixtime_microseconds_todatetime:
|
||||
return std::make_unique<UnixTimeMicrosecondsToDateTime>();
|
||||
|
||||
case KQLFunctionValue::unixtime_milliseconds_todatetime:
|
||||
return std::make_unique<UnixTimeMillisecondsToDateTime>();
|
||||
|
||||
case KQLFunctionValue::unixtime_nanoseconds_todatetime:
|
||||
return std::make_unique<UnixTimeNanosecondsToDateTime>();
|
||||
|
||||
case KQLFunctionValue::unixtime_seconds_todatetime:
|
||||
return std::make_unique<UnixTimeSecondsToDateTime>();
|
||||
|
||||
case KQLFunctionValue::week_of_year:
|
||||
return std::make_unique<WeekOfYear>();
|
||||
|
||||
case KQLFunctionValue::base64_encode_tostring:
|
||||
return std::make_unique<Base64EncodeToString>();
|
||||
|
||||
case KQLFunctionValue::base64_encode_fromguid:
|
||||
return std::make_unique<Base64EncodeFromGuid>();
|
||||
|
||||
case KQLFunctionValue::base64_decode_tostring:
|
||||
return std::make_unique<Base64DecodeToString>();
|
||||
|
||||
case KQLFunctionValue::base64_decode_toarray:
|
||||
return std::make_unique<Base64DecodeToArray>();
|
||||
|
||||
case KQLFunctionValue::base64_decode_toguid:
|
||||
return std::make_unique<Base64DecodeToGuid>();
|
||||
|
||||
case KQLFunctionValue::countof:
|
||||
return std::make_unique<CountOf>();
|
||||
|
||||
case KQLFunctionValue::extract:
|
||||
return std::make_unique<Extract>();
|
||||
|
||||
case KQLFunctionValue::extract_all:
|
||||
return std::make_unique<ExtractAll>();
|
||||
|
||||
case KQLFunctionValue::extract_json:
|
||||
return std::make_unique<ExtractJson>();
|
||||
|
||||
case KQLFunctionValue::has_any_index:
|
||||
return std::make_unique<HasAnyIndex>();
|
||||
|
||||
case KQLFunctionValue::indexof:
|
||||
return std::make_unique<IndexOf>();
|
||||
|
||||
case KQLFunctionValue::isempty:
|
||||
return std::make_unique<IsEmpty>();
|
||||
|
||||
case KQLFunctionValue::isnan:
|
||||
return std::make_unique<IsNan>();
|
||||
|
||||
case KQLFunctionValue::isnotempty:
|
||||
return std::make_unique<IsNotEmpty>();
|
||||
|
||||
case KQLFunctionValue::isnotnull:
|
||||
return std::make_unique<IsNotNull>();
|
||||
|
||||
case KQLFunctionValue::isnull:
|
||||
return std::make_unique<IsNull>();
|
||||
|
||||
case KQLFunctionValue::parse_command_line:
|
||||
return std::make_unique<ParseCommandLine>();
|
||||
|
||||
case KQLFunctionValue::parse_csv:
|
||||
return std::make_unique<ParseCSV>();
|
||||
|
||||
case KQLFunctionValue::parse_json:
|
||||
return std::make_unique<ParseJson>();
|
||||
|
||||
case KQLFunctionValue::parse_url:
|
||||
return std::make_unique<ParseURL>();
|
||||
|
||||
case KQLFunctionValue::parse_urlquery:
|
||||
return std::make_unique<ParseURLQuery>();
|
||||
|
||||
case KQLFunctionValue::parse_version:
|
||||
return std::make_unique<ParseVersion>();
|
||||
|
||||
case KQLFunctionValue::replace_regex:
|
||||
return std::make_unique<ReplaceRegex>();
|
||||
|
||||
case KQLFunctionValue::reverse:
|
||||
return std::make_unique<Reverse>();
|
||||
|
||||
case KQLFunctionValue::split:
|
||||
return std::make_unique<Split>();
|
||||
|
||||
case KQLFunctionValue::strcat:
|
||||
return std::make_unique<StrCat>();
|
||||
|
||||
case KQLFunctionValue::strcat_delim:
|
||||
return std::make_unique<StrCatDelim>();
|
||||
|
||||
case KQLFunctionValue::strcmp:
|
||||
return std::make_unique<StrCmp>();
|
||||
|
||||
case KQLFunctionValue::strlen:
|
||||
return std::make_unique<StrLen>();
|
||||
|
||||
case KQLFunctionValue::strrep:
|
||||
return std::make_unique<StrRep>();
|
||||
|
||||
case KQLFunctionValue::substring:
|
||||
return std::make_unique<SubString>();
|
||||
|
||||
case KQLFunctionValue::tolower:
|
||||
return std::make_unique<ToLower>();
|
||||
|
||||
case KQLFunctionValue::toupper:
|
||||
return std::make_unique<ToUpper>();
|
||||
|
||||
case KQLFunctionValue::translate:
|
||||
return std::make_unique<Translate>();
|
||||
|
||||
case KQLFunctionValue::trim:
|
||||
return std::make_unique<Trim>();
|
||||
|
||||
case KQLFunctionValue::trim_end:
|
||||
return std::make_unique<TrimEnd>();
|
||||
|
||||
case KQLFunctionValue::trim_start:
|
||||
return std::make_unique<TrimStart>();
|
||||
|
||||
case KQLFunctionValue::url_decode:
|
||||
return std::make_unique<URLDecode>();
|
||||
|
||||
case KQLFunctionValue::url_encode:
|
||||
return std::make_unique<URLEncode>();
|
||||
|
||||
case KQLFunctionValue::array_concat:
|
||||
return std::make_unique<ArrayConcat>();
|
||||
|
||||
case KQLFunctionValue::array_iif:
|
||||
return std::make_unique<ArrayIif>();
|
||||
|
||||
case KQLFunctionValue::array_index_of:
|
||||
return std::make_unique<ArrayIndexOf>();
|
||||
|
||||
case KQLFunctionValue::array_length:
|
||||
return std::make_unique<ArrayLength>();
|
||||
|
||||
case KQLFunctionValue::array_reverse:
|
||||
return std::make_unique<ArrayReverse>();
|
||||
|
||||
case KQLFunctionValue::array_rotate_left:
|
||||
return std::make_unique<ArrayRotateLeft>();
|
||||
|
||||
case KQLFunctionValue::array_rotate_right:
|
||||
return std::make_unique<ArrayRotateRight>();
|
||||
|
||||
case KQLFunctionValue::array_shift_left:
|
||||
return std::make_unique<ArrayShiftLeft>();
|
||||
|
||||
case KQLFunctionValue::array_shift_right:
|
||||
return std::make_unique<ArrayShiftRight>();
|
||||
|
||||
case KQLFunctionValue::array_slice:
|
||||
return std::make_unique<ArraySlice>();
|
||||
|
||||
case KQLFunctionValue::array_sort_asc:
|
||||
return std::make_unique<ArraySortAsc>();
|
||||
|
||||
case KQLFunctionValue::array_sort_desc:
|
||||
return std::make_unique<ArraySortDesc>();
|
||||
|
||||
case KQLFunctionValue::array_split:
|
||||
return std::make_unique<ArraySplit>();
|
||||
|
||||
case KQLFunctionValue::array_sum:
|
||||
return std::make_unique<ArraySum>();
|
||||
|
||||
case KQLFunctionValue::bag_keys:
|
||||
return std::make_unique<BagKeys>();
|
||||
|
||||
case KQLFunctionValue::bag_merge:
|
||||
return std::make_unique<BagMerge>();
|
||||
|
||||
case KQLFunctionValue::bag_remove_keys:
|
||||
return std::make_unique<BagRemoveKeys>();
|
||||
|
||||
case KQLFunctionValue::jaccard_index:
|
||||
return std::make_unique<JaccardIndex>();
|
||||
|
||||
case KQLFunctionValue::pack:
|
||||
return std::make_unique<Pack>();
|
||||
|
||||
case KQLFunctionValue::pack_all:
|
||||
return std::make_unique<PackAll>();
|
||||
|
||||
case KQLFunctionValue::pack_array:
|
||||
return std::make_unique<PackArray>();
|
||||
|
||||
case KQLFunctionValue::repeat:
|
||||
return std::make_unique<Repeat>();
|
||||
|
||||
case KQLFunctionValue::set_difference:
|
||||
return std::make_unique<SetDifference>();
|
||||
|
||||
case KQLFunctionValue::set_has_element:
|
||||
return std::make_unique<SetHasElement>();
|
||||
|
||||
case KQLFunctionValue::set_intersect:
|
||||
return std::make_unique<SetIntersect>();
|
||||
|
||||
case KQLFunctionValue::set_union:
|
||||
return std::make_unique<SetUnion>();
|
||||
|
||||
case KQLFunctionValue::treepath:
|
||||
return std::make_unique<TreePath>();
|
||||
|
||||
case KQLFunctionValue::zip:
|
||||
return std::make_unique<Zip>();
|
||||
|
||||
case KQLFunctionValue::tobool:
|
||||
return std::make_unique<ToBool>();
|
||||
|
||||
case KQLFunctionValue::todatetime:
|
||||
return std::make_unique<ToDateTime>();
|
||||
|
||||
case KQLFunctionValue::todouble:
|
||||
return std::make_unique<ToDouble>();
|
||||
|
||||
case KQLFunctionValue::toint:
|
||||
return std::make_unique<ToInt>();
|
||||
|
||||
case KQLFunctionValue::tolong:
|
||||
return std::make_unique<ToLong>();
|
||||
|
||||
case KQLFunctionValue::tostring:
|
||||
return std::make_unique<ToString>();
|
||||
|
||||
case KQLFunctionValue::totimespan:
|
||||
return std::make_unique<ToTimeSpan>();
|
||||
|
||||
case KQLFunctionValue::todecimal:
|
||||
return std::make_unique<ToDecimal>();
|
||||
|
||||
case KQLFunctionValue::arg_max:
|
||||
return std::make_unique<ArgMax>();
|
||||
|
||||
case KQLFunctionValue::arg_min:
|
||||
return std::make_unique<ArgMin>();
|
||||
|
||||
case KQLFunctionValue::avg:
|
||||
return std::make_unique<Avg>();
|
||||
|
||||
case KQLFunctionValue::avgif:
|
||||
return std::make_unique<AvgIf>();
|
||||
|
||||
case KQLFunctionValue::binary_all_and:
|
||||
return std::make_unique<BinaryAllAnd>();
|
||||
|
||||
case KQLFunctionValue::binary_all_or:
|
||||
return std::make_unique<BinaryAllOr>();
|
||||
|
||||
case KQLFunctionValue::binary_all_xor:
|
||||
return std::make_unique<BinaryAllXor>();
|
||||
|
||||
case KQLFunctionValue::buildschema:
|
||||
return std::make_unique<BuildSchema>();
|
||||
|
||||
case KQLFunctionValue::count:
|
||||
return std::make_unique<Count>();
|
||||
|
||||
case KQLFunctionValue::countif:
|
||||
return std::make_unique<CountIf>();
|
||||
|
||||
case KQLFunctionValue::dcount:
|
||||
return std::make_unique<DCount>();
|
||||
|
||||
case KQLFunctionValue::dcountif:
|
||||
return std::make_unique<DCountIf>();
|
||||
|
||||
case KQLFunctionValue::make_bag:
|
||||
return std::make_unique<MakeBag>();
|
||||
|
||||
case KQLFunctionValue::make_bag_if:
|
||||
return std::make_unique<MakeBagIf>();
|
||||
|
||||
case KQLFunctionValue::make_list:
|
||||
return std::make_unique<MakeList>();
|
||||
|
||||
case KQLFunctionValue::make_list_if:
|
||||
return std::make_unique<MakeListIf>();
|
||||
|
||||
case KQLFunctionValue::make_list_with_nulls:
|
||||
return std::make_unique<MakeListWithNulls>();
|
||||
|
||||
case KQLFunctionValue::make_set:
|
||||
return std::make_unique<MakeSet>();
|
||||
|
||||
case KQLFunctionValue::make_set_if:
|
||||
return std::make_unique<MakeSetIf>();
|
||||
|
||||
case KQLFunctionValue::max:
|
||||
return std::make_unique<Max>();
|
||||
|
||||
case KQLFunctionValue::maxif:
|
||||
return std::make_unique<MaxIf>();
|
||||
|
||||
case KQLFunctionValue::min:
|
||||
return std::make_unique<Min>();
|
||||
|
||||
case KQLFunctionValue::minif:
|
||||
return std::make_unique<MinIf>();
|
||||
|
||||
case KQLFunctionValue::percentile:
|
||||
return std::make_unique<Percentile>();
|
||||
|
||||
case KQLFunctionValue::percentilew:
|
||||
return std::make_unique<Percentilew>();
|
||||
|
||||
case KQLFunctionValue::percentiles:
|
||||
return std::make_unique<Percentiles>();
|
||||
|
||||
case KQLFunctionValue::percentiles_array:
|
||||
return std::make_unique<PercentilesArray>();
|
||||
|
||||
case KQLFunctionValue::percentilesw:
|
||||
return std::make_unique<Percentilesw>();
|
||||
|
||||
case KQLFunctionValue::percentilesw_array:
|
||||
return std::make_unique<PercentileswArray>();
|
||||
|
||||
case KQLFunctionValue::stdev:
|
||||
return std::make_unique<Stdev>();
|
||||
|
||||
case KQLFunctionValue::stdevif:
|
||||
return std::make_unique<StdevIf>();
|
||||
|
||||
case KQLFunctionValue::sum:
|
||||
return std::make_unique<Sum>();
|
||||
|
||||
case KQLFunctionValue::sumif:
|
||||
return std::make_unique<SumIf>();
|
||||
|
||||
case KQLFunctionValue::take_any:
|
||||
return std::make_unique<TakeAny>();
|
||||
|
||||
case KQLFunctionValue::take_anyif:
|
||||
return std::make_unique<TakeAnyIf>();
|
||||
|
||||
case KQLFunctionValue::variance:
|
||||
return std::make_unique<Variance>();
|
||||
|
||||
case KQLFunctionValue::varianceif:
|
||||
return std::make_unique<VarianceIf>();
|
||||
|
||||
case KQLFunctionValue::series_fir:
|
||||
return std::make_unique<SeriesFir>();
|
||||
|
||||
case KQLFunctionValue::series_iir:
|
||||
return std::make_unique<SeriesIir>();
|
||||
|
||||
case KQLFunctionValue::series_fit_line:
|
||||
return std::make_unique<SeriesFitLine>();
|
||||
|
||||
case KQLFunctionValue::series_fit_line_dynamic:
|
||||
return std::make_unique<SeriesFitLineDynamic>();
|
||||
|
||||
case KQLFunctionValue::series_fit_2lines:
|
||||
return std::make_unique<SeriesFit2lines>();
|
||||
|
||||
case KQLFunctionValue::series_fit_2lines_dynamic:
|
||||
return std::make_unique<SeriesFit2linesDynamic>();
|
||||
|
||||
case KQLFunctionValue::series_outliers:
|
||||
return std::make_unique<SeriesOutliers>();
|
||||
|
||||
case KQLFunctionValue::series_periods_detect:
|
||||
return std::make_unique<SeriesPeriodsDetect>();
|
||||
|
||||
case KQLFunctionValue::series_periods_validate:
|
||||
return std::make_unique<SeriesPeriodsValidate>();
|
||||
|
||||
case KQLFunctionValue::series_stats_dynamic:
|
||||
return std::make_unique<SeriesStatsDynamic>();
|
||||
|
||||
case KQLFunctionValue::series_stats:
|
||||
return std::make_unique<SeriesStats>();
|
||||
|
||||
case KQLFunctionValue::series_fill_backward:
|
||||
return std::make_unique<SeriesFillBackward>();
|
||||
|
||||
case KQLFunctionValue::series_fill_const:
|
||||
return std::make_unique<SeriesFillConst>();
|
||||
|
||||
case KQLFunctionValue::series_fill_forward:
|
||||
return std::make_unique<SeriesFillForward>();
|
||||
|
||||
case KQLFunctionValue::series_fill_linear:
|
||||
return std::make_unique<SeriesFillLinear>();
|
||||
|
||||
case KQLFunctionValue::ipv4_compare:
|
||||
return std::make_unique<Ipv4Compare>();
|
||||
|
||||
case KQLFunctionValue::ipv4_is_in_range:
|
||||
return std::make_unique<Ipv4IsInRange>();
|
||||
|
||||
case KQLFunctionValue::ipv4_is_match:
|
||||
return std::make_unique<Ipv4IsMatch>();
|
||||
|
||||
case KQLFunctionValue::ipv4_is_private:
|
||||
return std::make_unique<Ipv4IsPrivate>();
|
||||
|
||||
case KQLFunctionValue::ipv4_netmask_suffix:
|
||||
return std::make_unique<Ipv4NetmaskSuffix>();
|
||||
|
||||
case KQLFunctionValue::parse_ipv4:
|
||||
return std::make_unique<ParseIpv4>();
|
||||
|
||||
case KQLFunctionValue::parse_ipv4_mask:
|
||||
return std::make_unique<ParseIpv4Mask>();
|
||||
|
||||
case KQLFunctionValue::ipv6_compare:
|
||||
return std::make_unique<Ipv6Compare>();
|
||||
|
||||
case KQLFunctionValue::ipv6_is_match:
|
||||
return std::make_unique<Ipv6IsMatch>();
|
||||
|
||||
case KQLFunctionValue::parse_ipv6:
|
||||
return std::make_unique<ParseIpv6>();
|
||||
|
||||
case KQLFunctionValue::parse_ipv6_mask:
|
||||
return std::make_unique<ParseIpv6Mask>();
|
||||
|
||||
case KQLFunctionValue::format_ipv4:
|
||||
return std::make_unique<FormatIpv4>();
|
||||
|
||||
case KQLFunctionValue::format_ipv4_mask:
|
||||
return std::make_unique<FormatIpv4Mask>();
|
||||
|
||||
case KQLFunctionValue::binary_and:
|
||||
return std::make_unique<BinaryAnd>();
|
||||
|
||||
case KQLFunctionValue::binary_not:
|
||||
return std::make_unique<BinaryNot>();
|
||||
|
||||
case KQLFunctionValue::binary_or:
|
||||
return std::make_unique<BinaryOr>();
|
||||
|
||||
case KQLFunctionValue::binary_shift_left:
|
||||
return std::make_unique<BinaryShiftLeft>();
|
||||
|
||||
case KQLFunctionValue::binary_shift_right:
|
||||
return std::make_unique<BinaryShiftRight>();
|
||||
|
||||
case KQLFunctionValue::binary_xor:
|
||||
return std::make_unique<BinaryXor>();
|
||||
|
||||
case KQLFunctionValue::bitset_count_ones:
|
||||
return std::make_unique<BitsetCountOnes>();
|
||||
|
||||
case KQLFunctionValue::bin:
|
||||
return std::make_unique<Bin>();
|
||||
|
||||
case KQLFunctionValue::bin_at:
|
||||
return std::make_unique<BinAt>();
|
||||
|
||||
case KQLFunctionValue::datatype_bool:
|
||||
return std::make_unique<DatatypeBool>();
|
||||
|
||||
case KQLFunctionValue::datatype_datetime:
|
||||
return std::make_unique<DatatypeDatetime>();
|
||||
|
||||
case KQLFunctionValue::datatype_dynamic:
|
||||
return std::make_unique<DatatypeDynamic>();
|
||||
|
||||
case KQLFunctionValue::datatype_guid:
|
||||
return std::make_unique<DatatypeGuid>();
|
||||
|
||||
case KQLFunctionValue::datatype_int:
|
||||
return std::make_unique<DatatypeInt>();
|
||||
|
||||
case KQLFunctionValue::datatype_long:
|
||||
return std::make_unique<DatatypeLong>();
|
||||
|
||||
case KQLFunctionValue::datatype_real:
|
||||
return std::make_unique<DatatypeReal>();
|
||||
|
||||
case KQLFunctionValue::datatype_string:
|
||||
return std::make_unique<DatatypeString>();
|
||||
|
||||
case KQLFunctionValue::datatype_timespan:
|
||||
return std::make_unique<DatatypeTimespan>();
|
||||
|
||||
case KQLFunctionValue::datatype_decimal:
|
||||
return std::make_unique<DatatypeDecimal>();
|
||||
|
||||
case KQLFunctionValue::round:
|
||||
return std::make_unique<Round>();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
220
src/Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h
Normal file
220
src/Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h
Normal file
@ -0,0 +1,220 @@
|
||||
#pragma once
|
||||
|
||||
#include <unordered_map>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
namespace DB
|
||||
{
|
||||
enum class KQLFunctionValue : uint16_t
|
||||
{
|
||||
none,
|
||||
timespan,
|
||||
ago,
|
||||
datetime_add,
|
||||
datetime_part,
|
||||
datetime_diff,
|
||||
dayofmonth,
|
||||
dayofweek,
|
||||
dayofyear,
|
||||
endofday,
|
||||
endofweek,
|
||||
endofyear,
|
||||
endofmonth,
|
||||
monthofyear,
|
||||
format_datetime,
|
||||
format_timespan,
|
||||
getmonth,
|
||||
getyear,
|
||||
hourofday,
|
||||
make_timespan,
|
||||
make_datetime,
|
||||
now,
|
||||
startofday,
|
||||
startofmonth,
|
||||
startofweek,
|
||||
startofyear,
|
||||
todatetime,
|
||||
totimespan,
|
||||
unixtime_microseconds_todatetime,
|
||||
unixtime_milliseconds_todatetime,
|
||||
unixtime_nanoseconds_todatetime,
|
||||
unixtime_seconds_todatetime,
|
||||
week_of_year,
|
||||
|
||||
base64_encode_tostring,
|
||||
base64_encode_fromguid,
|
||||
base64_decode_tostring,
|
||||
base64_decode_toarray,
|
||||
base64_decode_toguid,
|
||||
countof,
|
||||
extract,
|
||||
extract_all,
|
||||
extract_json,
|
||||
has_any_index,
|
||||
indexof,
|
||||
isempty,
|
||||
isnan,
|
||||
isnotempty,
|
||||
isnotnull,
|
||||
isnull,
|
||||
parse_command_line,
|
||||
parse_csv,
|
||||
parse_json,
|
||||
parse_url,
|
||||
parse_urlquery,
|
||||
parse_version,
|
||||
replace_regex,
|
||||
reverse,
|
||||
split,
|
||||
strcat,
|
||||
strcat_delim,
|
||||
strcmp,
|
||||
strlen,
|
||||
strrep,
|
||||
substring,
|
||||
tolower,
|
||||
toupper,
|
||||
translate,
|
||||
trim,
|
||||
trim_end,
|
||||
trim_start,
|
||||
url_decode,
|
||||
url_encode,
|
||||
|
||||
array_concat,
|
||||
array_iif,
|
||||
array_index_of,
|
||||
array_length,
|
||||
array_reverse,
|
||||
array_rotate_left,
|
||||
array_rotate_right,
|
||||
array_shift_left,
|
||||
array_shift_right,
|
||||
array_slice,
|
||||
array_sort_asc,
|
||||
array_sort_desc,
|
||||
array_split,
|
||||
array_sum,
|
||||
bag_keys,
|
||||
bag_merge,
|
||||
bag_remove_keys,
|
||||
jaccard_index,
|
||||
pack,
|
||||
pack_all,
|
||||
pack_array,
|
||||
repeat,
|
||||
set_difference,
|
||||
set_has_element,
|
||||
set_intersect,
|
||||
set_union,
|
||||
treepath,
|
||||
zip,
|
||||
|
||||
tobool,
|
||||
todouble,
|
||||
toint,
|
||||
tolong,
|
||||
tostring,
|
||||
todecimal,
|
||||
|
||||
arg_max,
|
||||
arg_min,
|
||||
avg,
|
||||
avgif,
|
||||
binary_all_and,
|
||||
binary_all_or,
|
||||
binary_all_xor,
|
||||
buildschema,
|
||||
count,
|
||||
countif,
|
||||
dcount,
|
||||
dcountif,
|
||||
make_bag,
|
||||
make_bag_if,
|
||||
make_list,
|
||||
make_list_if,
|
||||
make_list_with_nulls,
|
||||
make_set,
|
||||
make_set_if,
|
||||
max,
|
||||
maxif,
|
||||
min,
|
||||
minif,
|
||||
percentile,
|
||||
percentilew,
|
||||
percentiles,
|
||||
percentiles_array,
|
||||
percentilesw,
|
||||
percentilesw_array,
|
||||
stdev,
|
||||
stdevif,
|
||||
sum,
|
||||
sumif,
|
||||
take_any,
|
||||
take_anyif,
|
||||
variance,
|
||||
varianceif,
|
||||
|
||||
series_fir,
|
||||
series_iir,
|
||||
series_fit_line,
|
||||
series_fit_line_dynamic,
|
||||
series_fit_2lines,
|
||||
series_fit_2lines_dynamic,
|
||||
series_outliers,
|
||||
series_periods_detect,
|
||||
series_periods_validate,
|
||||
series_stats_dynamic,
|
||||
series_stats,
|
||||
series_fill_backward,
|
||||
series_fill_const,
|
||||
series_fill_forward,
|
||||
series_fill_linear,
|
||||
|
||||
ipv4_compare,
|
||||
ipv4_is_in_range,
|
||||
ipv4_is_match,
|
||||
ipv4_is_private,
|
||||
ipv4_netmask_suffix,
|
||||
parse_ipv4,
|
||||
parse_ipv4_mask,
|
||||
ipv6_compare,
|
||||
ipv6_is_match,
|
||||
parse_ipv6,
|
||||
parse_ipv6_mask,
|
||||
format_ipv4,
|
||||
format_ipv4_mask,
|
||||
|
||||
binary_and,
|
||||
binary_not,
|
||||
binary_or,
|
||||
binary_shift_left,
|
||||
binary_shift_right,
|
||||
binary_xor,
|
||||
bitset_count_ones,
|
||||
|
||||
bin,
|
||||
bin_at,
|
||||
|
||||
datatype_bool,
|
||||
datatype_datetime,
|
||||
datatype_dynamic,
|
||||
datatype_guid,
|
||||
datatype_int,
|
||||
datatype_long,
|
||||
datatype_real,
|
||||
datatype_string,
|
||||
datatype_timespan,
|
||||
datatype_decimal,
|
||||
round
|
||||
};
|
||||
class KQLFunctionFactory
|
||||
{
|
||||
public:
|
||||
static std::unique_ptr<IParserKQLFunction> get(String & kql_function);
|
||||
|
||||
protected:
|
||||
static std::unordered_map<String, KQLFunctionValue> kql_functions;
|
||||
};
|
||||
|
||||
}
|
103
src/Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.cpp
Normal file
103
src/Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.cpp
Normal file
@ -0,0 +1,103 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLIPFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLStringFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.h>
|
||||
#include <Parsers/Kusto/ParserKQLDateTypeTimespan.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
#include <boost/lexical_cast.hpp>
|
||||
#include <format>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool Bin::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
double bin_size;
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String origal_expr(pos->begin, pos->end);
|
||||
String value = getConvertedArgument(fn_name, pos);
|
||||
|
||||
++pos;
|
||||
String round_to = getConvertedArgument(fn_name, pos);
|
||||
|
||||
//remove sapce between minus and number
|
||||
round_to.erase(std::remove_if(round_to.begin(), round_to.end(), isspace), round_to.end());
|
||||
|
||||
auto t = std::format("toFloat64({})", value);
|
||||
|
||||
bin_size = std::stod(round_to);
|
||||
|
||||
if (origal_expr == "datetime" || origal_expr == "date")
|
||||
{
|
||||
out = std::format("toDateTime64(toInt64({0}/{1}) * {1}, 9, 'UTC')", t, bin_size);
|
||||
}
|
||||
else if (origal_expr == "timespan" || origal_expr == "time" || ParserKQLDateTypeTimespan().parseConstKQLTimespan(origal_expr))
|
||||
{
|
||||
String bin_value = std::format("toInt64({0}/{1}) * {1}", t, bin_size);
|
||||
out = std::format(
|
||||
"concat(toString(toInt32((({}) as x) / 3600)),':', toString(toInt32(x % 3600 / 60)),':',toString(toInt32(x % 3600 % 60)))",
|
||||
bin_value);
|
||||
}
|
||||
else
|
||||
{
|
||||
out = std::format("toInt64({0} / {1}) * {1}", t, bin_size);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool BinAt::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
double bin_size;
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String origal_expr(pos->begin, pos->end);
|
||||
String expression_str = getConvertedArgument(fn_name, pos);
|
||||
|
||||
++pos;
|
||||
String bin_size_str = getConvertedArgument(fn_name, pos);
|
||||
|
||||
++pos;
|
||||
String fixed_point_str = getConvertedArgument(fn_name, pos);
|
||||
|
||||
auto t1 = std::format("toFloat64({})", fixed_point_str);
|
||||
auto t2 = std::format("toFloat64({})", expression_str);
|
||||
int dir = t2 >= t1 ? 0 : -1;
|
||||
bin_size = std::stod(bin_size_str);
|
||||
|
||||
if (origal_expr == "datetime" || origal_expr == "date")
|
||||
{
|
||||
out = std::format("toDateTime64({} + toInt64(({} - {}) / {} + {}) * {}, 9, 'UTC')", t1, t2, t1, bin_size, dir, bin_size);
|
||||
}
|
||||
else if (origal_expr == "timespan" || origal_expr == "time" || ParserKQLDateTypeTimespan().parseConstKQLTimespan(origal_expr))
|
||||
{
|
||||
String bin_value = std::format("{} + toInt64(({} - {}) / {} + {}) * {}", t1, t2, t1, bin_size, dir, bin_size);
|
||||
out = std::format(
|
||||
"concat(toString(toInt32((({}) as x) / 3600)),':', toString(toInt32(x % 3600 / 60)), ':', toString(toInt32(x % 3600 % 60)))",
|
||||
bin_value);
|
||||
}
|
||||
else
|
||||
{
|
||||
out = std::format("{} + toInt64(({} - {}) / {} + {}) * {}", t1, t2, t1, bin_size, dir, bin_size);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
21
src/Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.h
Normal file
21
src/Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.h
Normal file
@ -0,0 +1,21 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
namespace DB
|
||||
{
|
||||
class Bin : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "bin()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class BinAt : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "bin_at()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
}
|
275
src/Parsers/Kusto/KustoFunctions/KQLIPFunctions.cpp
Normal file
275
src/Parsers/Kusto/KustoFunctions/KQLIPFunctions.cpp
Normal file
@ -0,0 +1,275 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLIPFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLStringFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
|
||||
#include <format>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
bool Ipv4Compare::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto lhs = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto rhs = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto mask = getOptionalArgument(function_name, pos, ArgumentState::Parsed);
|
||||
out = std::format(
|
||||
"if(isNull({0} as lhs_ip_{5}) or isNull({1} as lhs_mask_{5}) "
|
||||
"or isNull({2} as rhs_ip_{5}) or isNull({3} as rhs_mask_{5}), null, "
|
||||
"sign(IPv4StringToNumOrNull(toString((tupleElement(IPv4CIDRToRange(assumeNotNull(lhs_ip_{5}), "
|
||||
"toUInt8(min2({4}, min2(assumeNotNull(lhs_mask_{5}), assumeNotNull(rhs_mask_{5})))) as mask_{5}), 1))))"
|
||||
" - IPv4StringToNumOrNull(toString((tupleElement(IPv4CIDRToRange(assumeNotNull(rhs_ip_{5}), mask_{5}), 1))))))",
|
||||
kqlCallToExpression("parse_ipv4", {lhs}, pos.max_depth),
|
||||
kqlCallToExpression("ipv4_netmask_suffix", {lhs}, pos.max_depth),
|
||||
kqlCallToExpression("parse_ipv4", {rhs}, pos.max_depth),
|
||||
kqlCallToExpression("ipv4_netmask_suffix", {rhs}, pos.max_depth),
|
||||
mask ? *mask : "32",
|
||||
generateUniqueIdentifier());
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Ipv4IsInRange::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto ip_address = getArgument(function_name, pos, ArgumentState::Parsed);
|
||||
const auto ip_range = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
out = std::format(
|
||||
"if(isNull(IPv4StringToNumOrNull({0}) as ip_{3}) "
|
||||
"or isNull({1} as range_start_ip_{3}) or isNull({2} as range_mask_{3}), null, "
|
||||
"bitXor(range_start_ip_{3}, bitAnd(ip_{3}, bitNot(toUInt32(intExp2(toInt32(32 - range_mask_{3})) - 1)))) = 0) ",
|
||||
ip_address,
|
||||
kqlCallToExpression("parse_ipv4", {ip_range}, pos.max_depth),
|
||||
kqlCallToExpression("ipv4_netmask_suffix", {ip_range}, pos.max_depth),
|
||||
generateUniqueIdentifier());
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Ipv4IsMatch::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto lhs = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto rhs = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto mask = getOptionalArgument(function_name, pos, ArgumentState::Raw);
|
||||
out = std::format("equals({}, 0)", kqlCallToExpression("ipv4_compare", {lhs, rhs, mask ? *mask : "32"}, pos.max_depth));
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Ipv4IsPrivate::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
static const std::array<String, 3> s_private_subnets{"10.0.0.0/8", "172.16.0.0/12", "192.168.0.0/16"};
|
||||
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto ip_address = getArgument(function_name, pos);
|
||||
const auto unique_identifier = generateUniqueIdentifier();
|
||||
|
||||
out += std::format(
|
||||
"multiIf(length(splitByChar('/', {0}) as tokens_{1}) > 2 or isNull(toIPv4OrNull(tokens_{1}[1]) as nullable_ip_{1}) "
|
||||
"or length(tokens_{1}) = 2 and isNull(toUInt8OrNull(tokens_{1}[-1]) as mask_{1}), null, "
|
||||
"ignore(assumeNotNull(nullable_ip_{1}) as ip_{1}, "
|
||||
"IPv4CIDRToRange(ip_{1}, assumeNotNull(mask_{1})) as range_{1}, IPv4NumToString(tupleElement(range_{1}, 1)) as begin_{1}, "
|
||||
"IPv4NumToString(tupleElement(range_{1}, 2)) as end_{1}), null, ",
|
||||
ip_address,
|
||||
unique_identifier);
|
||||
for (int i = 0; i < std::ssize(s_private_subnets); ++i)
|
||||
{
|
||||
if (i > 0)
|
||||
out += " or ";
|
||||
|
||||
const auto & subnet = s_private_subnets[i];
|
||||
out += std::format(
|
||||
"length(tokens_{1}) = 1 and isIPAddressInRange(IPv4NumToString(ip_{1}), '{0}') or "
|
||||
"length(tokens_{1}) = 2 and isIPAddressInRange(begin_{1}, '{0}') and isIPAddressInRange(end_{1}, '{0}')",
|
||||
subnet,
|
||||
unique_identifier);
|
||||
}
|
||||
|
||||
out.push_back(')');
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Ipv4NetmaskSuffix::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto ip_range = getArgument(function_name, pos);
|
||||
out = std::format(
|
||||
"multiIf(length(splitByChar('/', {0}) as tokens_{1}) > 2 or not isIPv4String(tokens_{1}[1]), null, "
|
||||
"length(tokens_{1}) = 1, 32, isNull(toUInt8OrNull(tokens_{1}[-1]) as mask_{1}), null, toUInt8(min2(mask_{1}, 32)))",
|
||||
ip_range,
|
||||
generateUniqueIdentifier());
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParseIpv4::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto ip_address = getArgument(function_name, pos);
|
||||
out = std::format(
|
||||
"multiIf(length(splitByChar('/', {0}) as tokens_{1}) = 1, IPv4StringToNumOrNull(tokens_{1}[1]) as ip_{1}, "
|
||||
"length(tokens_{1}) = 2 and isNotNull(ip_{1}) and isNotNull(toUInt8OrNull(tokens_{1}[-1]) as mask_{1}), "
|
||||
"IPv4StringToNumOrNull(toString(tupleElement(IPv4CIDRToRange(toIPv4(assumeNotNull(ip_{1})), assumeNotNull(mask_{1})), 1))), null)",
|
||||
ip_address,
|
||||
generateUniqueIdentifier());
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParseIpv4Mask::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto ip_address = getArgument(function_name, pos);
|
||||
const auto mask = getArgument(function_name, pos);
|
||||
out = std::format(
|
||||
"if(isNull(toIPv4OrNull({0}) as ip_{2}) or isNull(toUInt8OrNull(toString({1})) as mask_{2}), null, "
|
||||
"toUInt32(tupleElement(IPv4CIDRToRange(assumeNotNull(ip_{2}), arrayMax([0, arrayMin([32, assumeNotNull(mask_{2})])])), 1)))",
|
||||
ip_address,
|
||||
mask,
|
||||
generateUniqueIdentifier());
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Ipv6Compare::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto lhs = getArgument(function_name, pos);
|
||||
const auto rhs = getArgument(function_name, pos);
|
||||
const auto mask = getOptionalArgument(function_name, pos);
|
||||
const auto calculated_mask = mask ? *mask : "128";
|
||||
out = std::format(
|
||||
"if(length(splitByChar('/', {1}) as lhs_tokens_{0}) > 2 or length(splitByChar('/', {2}) as rhs_tokens_{0}) > 2 "
|
||||
"or isNull(IPv6StringToNumOrNull(lhs_tokens_{0}[1]) as lhs_ipv6_{0}) or length(lhs_tokens_{0}) = 2 "
|
||||
"and isNull((if(isIPv4String(lhs_tokens_{0}[1]), 96, 0) + toUInt8OrNull(lhs_tokens_{0}[-1])) as lhs_suffix_{0}) "
|
||||
"or isNull(IPv6StringToNumOrNull(rhs_tokens_{0}[1]) as rhs_ipv6_{0}) or length(rhs_tokens_{0}) = 2 "
|
||||
"and isNull((if(isIPv4String(rhs_tokens_{0}[1]), 96, 0) + toUInt8OrNull(rhs_tokens_{0}[-1])) as rhs_suffix_{0}) "
|
||||
"or isNull(toUInt8(min2({3}, min2(ifNull(lhs_suffix_{0}, 128), ifNull(rhs_suffix_{0}, 128)))) as suffix_{0}) "
|
||||
"or isNull(bitShiftLeft(bitShiftRight(bitNot(reinterpretAsFixedString(0::UInt128)), (128 - suffix_{0}) as zeroes_{0}), "
|
||||
"zeroes_{0}) as mask_{0}) or isNull(bitAnd(lhs_ipv6_{0}, mask_{0}) as lhs_base_{0}) "
|
||||
"or isNull(bitAnd(rhs_ipv6_{0}, mask_{0}) as rhs_base_{0}), null, "
|
||||
"multiIf(lhs_base_{0} < rhs_base_{0}, -1, lhs_base_{0} > rhs_base_{0}, 1, 0))",
|
||||
generateUniqueIdentifier(),
|
||||
lhs,
|
||||
rhs,
|
||||
calculated_mask);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Ipv6IsMatch::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto lhs = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto rhs = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto mask = getOptionalArgument(function_name, pos, ArgumentState::Raw);
|
||||
out = std::format("equals({}, 0)", kqlCallToExpression("ipv6_compare", {lhs, rhs, mask ? *mask : "128"}, pos.max_depth));
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParseIpv6::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto ip_address = getArgument(function_name, pos);
|
||||
out = std::format(
|
||||
"if(length(splitByChar('/', assumeNotNull({0})) as tokens_{1}) > 2 or isNull(IPv6StringToNumOrNull(tokens_{1}[1]) as ip_{1}) "
|
||||
"or length(tokens_{1}) = 2 and isNull(toUInt8OrNull(tokens_{1}[-1]) as mask_{1}), null, "
|
||||
"arrayStringConcat(flatten(extractAllGroups(lower(hex(tupleElement(IPv6CIDRToRange(assumeNotNull(ip_{1}), toUInt8(ifNull(mask_{1} "
|
||||
"+ if(isIPv4String(tokens_{1}[1]), 96, 0), 128))), 1))), '([\\da-f]{{4}})')), ':'))",
|
||||
ip_address,
|
||||
generateUniqueIdentifier());
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParseIpv6Mask::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto ip_address = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto mask = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto unique_identifier = generateUniqueIdentifier();
|
||||
out = std::format(
|
||||
"if(empty({0} as ipv4_{3}), {1}, {2})",
|
||||
kqlCallToExpression("format_ipv4", {"trim_start('::', " + ip_address + ")", mask + " - 96"}, pos.max_depth),
|
||||
kqlCallToExpression("parse_ipv6", {"strcat(tostring(parse_ipv6(" + ip_address + ")), '/', tostring(" + mask + "))"}, pos.max_depth),
|
||||
kqlCallToExpression("parse_ipv6", {"ipv4_" + unique_identifier}, pos.max_depth),
|
||||
unique_identifier);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool FormatIpv4::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto ip_address = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto mask = getOptionalArgument(function_name, pos, ArgumentState::Parsed);
|
||||
out = std::format(
|
||||
"ifNull(if(isNotNull(toUInt32OrNull(toString({0})) as param_as_uint32_{3}) and toTypeName({0}) = 'String' or ({1}) < 0 "
|
||||
"or isNull(ifNull(param_as_uint32_{3}, {2}) as ip_as_number_{3}), null, "
|
||||
"IPv4NumToString(bitAnd(ip_as_number_{3}, bitNot(toUInt32(intExp2(toInt32(32 - ({1}))) - 1))))), '')",
|
||||
ParserKQLBase::getExprFromToken(ip_address, pos.max_depth),
|
||||
mask ? *mask : "32",
|
||||
kqlCallToExpression("parse_ipv4", {"tostring(" + ip_address + ")"}, pos.max_depth),
|
||||
generateUniqueIdentifier());
|
||||
return true;
|
||||
}
|
||||
|
||||
bool FormatIpv4Mask::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto ip_address = getArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto mask = getOptionalArgument(function_name, pos, ArgumentState::Raw);
|
||||
const auto calculated_mask = mask ? *mask : "32";
|
||||
out = std::format(
|
||||
"if(empty({1} as formatted_ip_{2}) or position(toTypeName({0}), 'Int') = 0 or not {0} between 0 and 32, '', "
|
||||
"concat(formatted_ip_{2}, '/', toString(toInt64(min2({0}, ifNull({3} as suffix_{2}, 32))))))",
|
||||
ParserKQLBase::getExprFromToken(calculated_mask, pos.max_depth),
|
||||
kqlCallToExpression("format_ipv4", {ip_address, calculated_mask}, pos.max_depth),
|
||||
generateUniqueIdentifier(),
|
||||
kqlCallToExpression("ipv4_netmask_suffix", {"tostring(" + ip_address + ")"}, pos.max_depth));
|
||||
return true;
|
||||
}
|
||||
}
|
98
src/Parsers/Kusto/KustoFunctions/KQLIPFunctions.h
Normal file
98
src/Parsers/Kusto/KustoFunctions/KQLIPFunctions.h
Normal file
@ -0,0 +1,98 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
namespace DB
|
||||
{
|
||||
class Ipv4Compare : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "ipv4_compare()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Ipv4IsInRange : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "ipv4_is_in_range()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Ipv4IsMatch : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "ipv4_is_match()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Ipv4IsPrivate : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "ipv4_is_private()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Ipv4NetmaskSuffix : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "ipv4_netmask_suffix()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ParseIpv4 : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "parse_ipv4()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ParseIpv4Mask : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "parse_ipv4_mask()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Ipv6Compare : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "ipv6_compare()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Ipv6IsMatch : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "ipv6_is_match()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ParseIpv6 : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "parse_ipv6()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ParseIpv6Mask : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "parse_ipv6_mask()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class FormatIpv4 : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "format_ipv4()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class FormatIpv4Mask : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "format_ipv4_mask()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
}
|
@ -0,0 +1,23 @@
|
||||
#include "KQLMathematicalFunctions.h"
|
||||
|
||||
#include <format>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
bool IsNan::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto argument = getArgument(function_name, pos);
|
||||
out = std::format("if(toTypeName({0}) = 'Float64', isNaN({0}), throwIf(true, 'Expected argument of data type real'))", argument);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Round::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "round");
|
||||
}
|
||||
}
|
21
src/Parsers/Kusto/KustoFunctions/KQLMathematicalFunctions.h
Normal file
21
src/Parsers/Kusto/KustoFunctions/KQLMathematicalFunctions.h
Normal file
@ -0,0 +1,21 @@
|
||||
#pragma once
|
||||
|
||||
#include "IParserKQLFunction.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class IsNan : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "isnan()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Round : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "round()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
}
|
773
src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.cpp
Normal file
773
src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.cpp
Normal file
@ -0,0 +1,773 @@
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLStringFunctions.h>
|
||||
|
||||
#include <boost/algorithm/string/classification.hpp>
|
||||
#include <boost/algorithm/string/split.hpp>
|
||||
#include <Poco/String.h>
|
||||
|
||||
#include <format>
|
||||
|
||||
|
||||
namespace DB::ErrorCodes
|
||||
{
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int UNKNOWN_TYPE;
|
||||
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool Base64EncodeToString::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "base64Encode");
|
||||
}
|
||||
|
||||
bool Base64EncodeFromGuid::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto argument = getArgument(function_name, pos);
|
||||
out = std::format(
|
||||
"if(toTypeName({0}) not in ['UUID', 'Nullable(UUID)'], toString(throwIf(true, 'Expected guid as argument')), "
|
||||
"base64Encode(UUIDStringToNum(toString({0}), 2)))",
|
||||
argument,
|
||||
generateUniqueIdentifier());
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Base64DecodeToString::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "base64Decode");
|
||||
}
|
||||
|
||||
bool Base64DecodeToArray::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String str = getConvertedArgument(fn_name, pos);
|
||||
|
||||
out = std::format("arrayMap(x -> (reinterpretAsUInt8(x)), splitByRegexp ('',base64Decode({})))", str);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Base64DecodeToGuid::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const auto function_name = getKQLFunctionName(pos);
|
||||
if (function_name.empty())
|
||||
return false;
|
||||
|
||||
const auto argument = getArgument(function_name, pos);
|
||||
out = std::format("toUUIDOrNull(UUIDNumToString(toFixedString(base64Decode({}), 16), 2))", argument);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool CountOf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String source = getConvertedArgument(fn_name, pos);
|
||||
|
||||
++pos;
|
||||
const String search = getConvertedArgument(fn_name, pos);
|
||||
|
||||
String kind = "'normal'";
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
kind = getConvertedArgument(fn_name, pos);
|
||||
}
|
||||
assert(kind == "'normal'" || kind == "'regex'");
|
||||
|
||||
if (kind == "'normal'")
|
||||
out = "countSubstrings(" + source + ", " + search + ")";
|
||||
else
|
||||
out = "countMatches(" + source + ", " + search + ")";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Extract::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
ParserKeyword s_kql("typeof");
|
||||
ParserToken open_bracket(TokenType::OpeningRoundBracket);
|
||||
ParserToken close_bracket(TokenType::ClosingRoundBracket);
|
||||
Expected expected;
|
||||
|
||||
std::unordered_map<String, String> type_cast
|
||||
= {{"bool", "Boolean"},
|
||||
{"boolean", "Boolean"},
|
||||
{"datetime", "DateTime"},
|
||||
{"date", "DateTime"},
|
||||
{"guid", "UUID"},
|
||||
{"int", "Int32"},
|
||||
{"long", "Int64"},
|
||||
{"real", "Float64"},
|
||||
{"double", "Float64"},
|
||||
{"string", "String"},
|
||||
{"decimal", "Decimal"}};
|
||||
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String regex = getConvertedArgument(fn_name, pos);
|
||||
|
||||
++pos;
|
||||
size_t capture_group = stoi(getConvertedArgument(fn_name, pos));
|
||||
|
||||
++pos;
|
||||
String source = getConvertedArgument(fn_name, pos);
|
||||
|
||||
String type_literal;
|
||||
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
|
||||
if (s_kql.ignore(pos, expected))
|
||||
{
|
||||
if (!open_bracket.ignore(pos, expected))
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near typeof");
|
||||
|
||||
type_literal = String(pos->begin, pos->end);
|
||||
|
||||
if (type_cast.find(type_literal) == type_cast.end())
|
||||
throw Exception(ErrorCodes::UNKNOWN_TYPE, "{} is not a supported kusto data type for extract", type_literal);
|
||||
|
||||
type_literal = type_cast[type_literal];
|
||||
++pos;
|
||||
|
||||
if (!close_bracket.ignore(pos, expected))
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near typeof");
|
||||
}
|
||||
}
|
||||
|
||||
if (capture_group == 0)
|
||||
{
|
||||
String tmp_regex;
|
||||
for (auto c : regex)
|
||||
{
|
||||
if (c != '(' && c != ')')
|
||||
tmp_regex += c;
|
||||
}
|
||||
regex = std::move(tmp_regex);
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t group_idx = 0;
|
||||
size_t str_idx = -1;
|
||||
for (size_t i = 0; i < regex.length(); ++i)
|
||||
{
|
||||
if (regex[i] == '(')
|
||||
{
|
||||
++group_idx;
|
||||
if (group_idx == capture_group)
|
||||
{
|
||||
str_idx = i + 1;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
String tmp_regex;
|
||||
if (str_idx > 0)
|
||||
{
|
||||
for (size_t i = str_idx; i < regex.length(); ++i)
|
||||
{
|
||||
if (regex[i] == ')')
|
||||
break;
|
||||
tmp_regex += regex[i];
|
||||
}
|
||||
}
|
||||
regex = "'" + tmp_regex + "'";
|
||||
}
|
||||
|
||||
out = "extract(" + source + ", " + regex + ")";
|
||||
|
||||
if (type_literal == "Decimal")
|
||||
{
|
||||
out = std::format("countSubstrings({0}, '.') > 1 ? NULL: {0}, length(substr({0}, position({0},'.') + 1)))", out);
|
||||
out = std::format("toDecimal128OrNull({0})", out);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (type_literal == "Boolean")
|
||||
out = std::format("toInt64OrNull({})", out);
|
||||
|
||||
if (!type_literal.empty())
|
||||
out = "accurateCastOrNull(" + out + ", '" + type_literal + "')";
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ExtractAll::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String regex = getConvertedArgument(fn_name, pos);
|
||||
|
||||
++pos;
|
||||
const String second_arg = getConvertedArgument(fn_name, pos);
|
||||
|
||||
String third_arg;
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
third_arg = getConvertedArgument(fn_name, pos);
|
||||
}
|
||||
|
||||
if (!third_arg.empty()) // currently the captureGroups not supported
|
||||
return false;
|
||||
|
||||
out = "extractAllGroups(" + second_arg + ", " + regex + ")";
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ExtractJson::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String datatype = "String";
|
||||
ParserKeyword s_kql("typeof");
|
||||
ParserToken open_bracket(TokenType::OpeningRoundBracket);
|
||||
ParserToken close_bracket(TokenType::ClosingRoundBracket);
|
||||
Expected expected;
|
||||
|
||||
std::unordered_map<String, String> type_cast
|
||||
= {{"bool", "Boolean"},
|
||||
{"boolean", "Boolean"},
|
||||
{"datetime", "DateTime"},
|
||||
{"date", "DateTime"},
|
||||
{"dynamic", "Array"},
|
||||
{"guid", "UUID"},
|
||||
{"int", "Int32"},
|
||||
{"long", "Int64"},
|
||||
{"real", "Float64"},
|
||||
{"double", "Float64"},
|
||||
{"string", "String"},
|
||||
{"decimal", "Decimal"}};
|
||||
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String json_datapath = getConvertedArgument(fn_name, pos);
|
||||
++pos;
|
||||
const String json_datasource = getConvertedArgument(fn_name, pos);
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
if (s_kql.ignore(pos, expected))
|
||||
{
|
||||
if (!open_bracket.ignore(pos, expected))
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near typeof");
|
||||
|
||||
datatype = String(pos->begin, pos->end);
|
||||
|
||||
if (type_cast.find(datatype) == type_cast.end())
|
||||
throw Exception(ErrorCodes::UNKNOWN_TYPE, "{} is not a supported kusto data type for {}", datatype, fn_name);
|
||||
datatype = type_cast[datatype];
|
||||
++pos;
|
||||
|
||||
if (!close_bracket.ignore(pos, expected))
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near typeof");
|
||||
}
|
||||
}
|
||||
const auto json_val = std::format("JSON_VALUE({0},{1})", json_datasource, json_datapath);
|
||||
|
||||
if (datatype == "Decimal")
|
||||
{
|
||||
out = std::format("countSubstrings({0}, '.') > 1 ? NULL: length(substr({0}, position({0},'.') + 1)))", json_val);
|
||||
out = std::format("toDecimal128OrNull({0}::String, {1})", json_val, out);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (datatype == "Boolean")
|
||||
out = std::format("toInt64OrNull({})", json_val);
|
||||
|
||||
if (!datatype.empty())
|
||||
out = std::format("accurateCastOrNull({},'{}')", json_val, datatype);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool HasAnyIndex::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String source = getConvertedArgument(fn_name, pos);
|
||||
|
||||
++pos;
|
||||
const String lookup = getConvertedArgument(fn_name, pos);
|
||||
String src_array = std::format("splitByChar(' ',{})", source);
|
||||
out = std::format(
|
||||
"if(empty({1}), -1, indexOf(arrayMap(x->(x in {0}), if(empty({1}),[''], arrayMap(x->(toString(x)),{1}))),1) - 1)",
|
||||
src_array,
|
||||
lookup);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool IndexOf::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
int start_index = 0, length = -1, occurrence = 1;
|
||||
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String source = getConvertedArgument(fn_name, pos);
|
||||
|
||||
++pos;
|
||||
const String lookup = getConvertedArgument(fn_name, pos);
|
||||
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
start_index = stoi(getConvertedArgument(fn_name, pos));
|
||||
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
length = stoi(getConvertedArgument(fn_name, pos));
|
||||
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
occurrence = stoi(getConvertedArgument(fn_name, pos));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (pos->type == TokenType::ClosingRoundBracket)
|
||||
{
|
||||
if (occurrence < 0 || length < -1)
|
||||
out = "";
|
||||
else if (length == -1)
|
||||
out = "position(" + source + ", " + lookup + ", " + std::to_string(start_index + 1) + ") - 1";
|
||||
else
|
||||
{
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
bool IsEmpty::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "empty");
|
||||
}
|
||||
|
||||
bool IsNotEmpty::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "notEmpty");
|
||||
}
|
||||
|
||||
bool IsNotNull::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "isNotNull");
|
||||
}
|
||||
|
||||
bool ParseCommandLine::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String json_string = getConvertedArgument(fn_name, pos);
|
||||
|
||||
++pos;
|
||||
const String type = getConvertedArgument(fn_name, pos);
|
||||
|
||||
if (type != "'windows'")
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Supported type argument is windows for {}", fn_name);
|
||||
|
||||
out = std::format(
|
||||
"if(empty({0}) OR hasAll(splitByChar(' ', {0}) , ['']) , arrayMap(x->null, splitByChar(' ', '')), splitByChar(' ', {0}))",
|
||||
json_string);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool IsNull::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "isNull");
|
||||
}
|
||||
|
||||
bool ParseCSV::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String csv_string = getConvertedArgument(fn_name, pos);
|
||||
|
||||
out = std::format(
|
||||
"if(position({0} ,'\n')::UInt8, (splitByChar(',', substring({0}, 1, position({0},'\n') -1))), (splitByChar(',', substring({0}, 1, "
|
||||
"length({0})))))",
|
||||
csv_string);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParseJson::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
if (String(pos->begin, pos->end) == "dynamic")
|
||||
{
|
||||
--pos;
|
||||
auto arg = getArgument(fn_name, pos);
|
||||
auto result = kqlCallToExpression("dynamic", {arg}, pos.max_depth);
|
||||
out = std::format("{}", result);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto arg = getConvertedArgument(fn_name, pos);
|
||||
out = std::format("if (isValidJSON({0}) , JSON_QUERY({0}, '$') , toJSONString({0}))", arg);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParseURL::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String url = getConvertedArgument(fn_name, pos);
|
||||
|
||||
const String scheme = std::format(R"(concat('"Scheme":"', protocol({0}),'"'))", url);
|
||||
const String host = std::format(R"(concat('"Host":"', domain({0}),'"'))", url);
|
||||
const String port = std::format(R"(concat('"Port":"', toString(port({0})),'"'))", url);
|
||||
const String path = std::format(R"(concat('"Path":"', path({0}),'"'))", url);
|
||||
const String username_pwd = std::format("netloc({0})", url);
|
||||
const String query_string = std::format("queryString({0})", url);
|
||||
const String fragment = std::format(R"(concat('"Fragment":"',fragment({0}),'"'))", url);
|
||||
const String username = std::format(
|
||||
R"(concat('"Username":"', arrayElement(splitByChar(':',arrayElement(splitByChar('@',{0}) ,1)),1),'"'))", username_pwd);
|
||||
const String password = std::format(
|
||||
R"(concat('"Password":"', arrayElement(splitByChar(':',arrayElement(splitByChar('@',{0}) ,1)),2),'"'))", username_pwd);
|
||||
const String query_parameters = std::format(
|
||||
R"(concat('"Query Parameters":', concat('{{"', replace(replace({}, '=', '":"'),'&','","') ,'"}}')))", query_string);
|
||||
|
||||
out = std::format(
|
||||
"concat('{{',{},',',{},',',{},',',{},',',{},',',{},',',{},',',{},'}}')",
|
||||
scheme,
|
||||
host,
|
||||
port,
|
||||
path,
|
||||
username,
|
||||
password,
|
||||
query_parameters,
|
||||
fragment);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParseURLQuery::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
++pos;
|
||||
const String query = getConvertedArgument(fn_name, pos);
|
||||
|
||||
const String query_string = std::format("if (position({},'?') > 0, queryString({}), {})", query, query, query);
|
||||
const String query_parameters = std::format(
|
||||
R"(concat('"Query Parameters":', concat('{{"', replace(replace({}, '=', '":"'),'&','","') ,'"}}')))", query_string);
|
||||
out = std::format("concat('{{',{},'}}')", query_parameters);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParseVersion::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
String arg;
|
||||
++pos;
|
||||
arg = getConvertedArgument(fn_name, pos);
|
||||
out = std::format(
|
||||
"length(splitByChar('.', {0})) > 4 OR length(splitByChar('.', {0})) < 1 OR match({0}, '.*[a-zA-Z]+.*') = 1 ? "
|
||||
"toDecimal128OrNull('NULL' , 0) : toDecimal128OrNull(substring(arrayStringConcat(arrayMap(x -> leftPad(x, 8, '0'), arrayMap(x -> "
|
||||
"if(empty(x), '0', x), arrayResize(splitByChar('.', {0}), 4)))), 8),0)",
|
||||
arg);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ReplaceRegex::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "replaceRegexpAll");
|
||||
}
|
||||
|
||||
bool Reverse::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
|
||||
auto arg = getConvertedArgument(fn_name, pos);
|
||||
|
||||
out = std::format("reverse(accurateCastOrNull({} , 'String'))", arg);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Split::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String source = getConvertedArgument(fn_name, pos);
|
||||
|
||||
++pos;
|
||||
const String delimiter = getConvertedArgument(fn_name, pos);
|
||||
auto split_res = std::format("empty({0}) ? splitByString(' ' , {1}) : splitByString({0} , {1})", delimiter, source);
|
||||
int requested_index = -1;
|
||||
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
auto arg = getConvertedArgument(fn_name, pos);
|
||||
// remove space between minus and value
|
||||
arg.erase(remove_if(arg.begin(), arg.end(), isspace), arg.end());
|
||||
requested_index = std::stoi(arg);
|
||||
requested_index += 1;
|
||||
out = std::format(
|
||||
"multiIf(length({0}) >= {1} AND {1} > 0, arrayPushBack([],arrayElement({0}, {1})), {1}=0, {0}, arrayPushBack([] "
|
||||
",arrayElement(NULL,1)))",
|
||||
split_res,
|
||||
requested_index);
|
||||
}
|
||||
else
|
||||
out = split_res;
|
||||
return true;
|
||||
}
|
||||
|
||||
bool StrCat::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "concat");
|
||||
}
|
||||
|
||||
bool StrCatDelim::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String delimiter = getConvertedArgument(fn_name, pos);
|
||||
|
||||
int arg_count = 0;
|
||||
String args;
|
||||
|
||||
while (!pos->isEnd() && pos->type != TokenType::Semicolon && pos->type != TokenType::ClosingRoundBracket)
|
||||
{
|
||||
++pos;
|
||||
String arg = getConvertedArgument(fn_name, pos);
|
||||
if (args.empty())
|
||||
args = "concat(" + arg;
|
||||
else
|
||||
args = args + ", " + delimiter + ", " + arg;
|
||||
++arg_count;
|
||||
}
|
||||
args += ")";
|
||||
|
||||
if (arg_count < 2 || arg_count > 64)
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "argument count out of bound in function: {}", fn_name);
|
||||
|
||||
out = std::move(args);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool StrCmp::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String string1 = getConvertedArgument(fn_name, pos);
|
||||
++pos;
|
||||
const String string2 = getConvertedArgument(fn_name, pos);
|
||||
|
||||
out = std::format("multiIf({0} == {1}, 0, {0} < {1}, -1, 1)", string1, string2);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool StrLen::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "lengthUTF8");
|
||||
}
|
||||
|
||||
bool StrRep::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
const String value = getConvertedArgument(fn_name, pos);
|
||||
|
||||
++pos;
|
||||
const String multiplier = getConvertedArgument(fn_name, pos);
|
||||
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
const String delimiter = getConvertedArgument(fn_name, pos);
|
||||
const String repeated_str = "repeat(concat(" + value + "," + delimiter + ")," + multiplier + ")";
|
||||
out = "substr(" + repeated_str + ", 1, length(" + repeated_str + ") - length(" + delimiter + "))";
|
||||
}
|
||||
else
|
||||
out = "repeat(" + value + ", " + multiplier + ")";
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool SubString::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String source = getConvertedArgument(fn_name, pos);
|
||||
|
||||
++pos;
|
||||
String starting_index = getConvertedArgument(fn_name, pos);
|
||||
|
||||
if (pos->type == TokenType::Comma)
|
||||
{
|
||||
++pos;
|
||||
auto length = getConvertedArgument(fn_name, pos);
|
||||
|
||||
if (starting_index.empty())
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "number of arguments do not match in function: {}", fn_name);
|
||||
else
|
||||
out = "if(toInt64(length(" + source + ")) <= 0, '', substr(" + source + ", " + "((" + starting_index + "% toInt64(length("
|
||||
+ source + ")) + toInt64(length(" + source + "))) % toInt64(length(" + source + "))) + 1, " + length + ") )";
|
||||
}
|
||||
else
|
||||
out = "if(toInt64(length(" + source + ")) <= 0, '', substr(" + source + "," + "((" + starting_index + "% toInt64(length(" + source
|
||||
+ ")) + toInt64(length(" + source + "))) % toInt64(length(" + source + "))) + 1))";
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ToLower::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "lower");
|
||||
}
|
||||
|
||||
bool ToUpper::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "upper");
|
||||
}
|
||||
|
||||
bool Translate::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
String from = getConvertedArgument(fn_name, pos);
|
||||
++pos;
|
||||
String to = getConvertedArgument(fn_name, pos);
|
||||
++pos;
|
||||
String source = getConvertedArgument(fn_name, pos);
|
||||
|
||||
String len_diff = std::format("length({}) - length({})", from, to);
|
||||
String to_str = std::format(
|
||||
"multiIf(length({1}) = 0, {0}, {2} > 0, concat({1},repeat(substr({1},length({1}),1),toUInt16({2}))),{2} < 0, "
|
||||
"substr({1},1,length({0})),{1})",
|
||||
from,
|
||||
to,
|
||||
len_diff);
|
||||
out = std::format("if (length({3}) = 0,'',translate({0},{1},{2}))", source, from, to_str, to);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Trim::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
const auto regex = getArgument(fn_name, pos, ArgumentState::Raw);
|
||||
const auto source = getArgument(fn_name, pos, ArgumentState::Raw);
|
||||
out = kqlCallToExpression("trim_start", {regex, std::format("trim_end({0}, {1})", regex, source)}, pos.max_depth);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool TrimEnd::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
const auto regex = getArgument(fn_name, pos);
|
||||
const auto source = getArgument(fn_name, pos);
|
||||
out = std::format("replaceRegexpOne({0}, concat({1}, '$'), '')", source, regex);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool TrimStart::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
const String fn_name = getKQLFunctionName(pos);
|
||||
if (fn_name.empty())
|
||||
return false;
|
||||
|
||||
const auto regex = getArgument(fn_name, pos);
|
||||
const auto source = getArgument(fn_name, pos);
|
||||
out = std::format("replaceRegexpOne({0}, concat('^', {1}), '')", source, regex);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool URLDecode::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "decodeURLComponent");
|
||||
}
|
||||
|
||||
bool URLEncode::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
return directMapping(out, pos, "encodeURLComponent");
|
||||
}
|
||||
|
||||
}
|
275
src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h
Normal file
275
src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.h
Normal file
@ -0,0 +1,275 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
#include <base/extended_types.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Base64EncodeToString : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "base64_encode_tostring()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Base64EncodeFromGuid : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "base64_encode_fromguid()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Base64DecodeToString : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "base64_decode_tostring()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Base64DecodeToArray : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "base64_decode_toarray()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Base64DecodeToGuid : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "base64_decode_toguid()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class CountOf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "countof()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Extract : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "extract()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ExtractAll : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "extract_all()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ExtractJson : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "extract_json(), extractjson()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class HasAnyIndex : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "has_any_index()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class IndexOf : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "indexof()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class IsEmpty : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "isempty()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class IsNotEmpty : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "isnotempty()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class IsNotNull : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "isnotnull()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class IsNull : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "isnull()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ParseCommandLine : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "parse_command_line()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ParseCSV : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "parse_csv()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ParseJson : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "parse_json()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ParseURL : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "parse_url()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ParseURLQuery : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "parse_urlquery()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ParseVersion : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "parse_version()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ReplaceRegex : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "replace_regex()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Reverse : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "reverse()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Split : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "split()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class StrCat : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "strcat()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class StrCatDelim : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "strcat_delim()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class StrCmp : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "strcmp()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class StrLen : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "strlen()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class StrRep : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "strrep()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SubString : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "substring()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ToLower : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "tolower()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class ToUpper : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "toupper()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Translate : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "translate()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class Trim : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "trim()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class TrimEnd : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "trim_end()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class TrimStart : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "trim_start()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class URLDecode : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "url_decode()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class URLEncode : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "url_encode()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
}
|
126
src/Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.cpp
Normal file
126
src/Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.cpp
Normal file
@ -0,0 +1,126 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLAggregationFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLBinaryFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLCastingFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLDynamicFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLGeneralFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLIPFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLStringFunctions.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool SeriesFir::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool SeriesIir::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool SeriesFitLine::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool SeriesFitLineDynamic::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool SeriesFit2lines::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool SeriesFit2linesDynamic::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool SeriesOutliers::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool SeriesPeriodsDetect::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool SeriesPeriodsValidate::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool SeriesStatsDynamic::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool SeriesStats::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool SeriesFillBackward::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool SeriesFillConst::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool SeriesFillForward::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool SeriesFillLinear::convertImpl(String & out, IParser::Pos & pos)
|
||||
{
|
||||
String res = String(pos->begin, pos->end);
|
||||
out = res;
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
112
src/Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.h
Normal file
112
src/Parsers/Kusto/KustoFunctions/KQLTimeSeriesFunctions.h
Normal file
@ -0,0 +1,112 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
namespace DB
|
||||
{
|
||||
class SeriesFir : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_fir()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SeriesIir : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_iir()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SeriesFitLine : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_fit_line()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SeriesFitLineDynamic : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_fit_line_dynamic()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SeriesFit2lines : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_fit_2lines()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SeriesFit2linesDynamic : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_fit_2lines_dynamic()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SeriesOutliers : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_outliers()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SeriesPeriodsDetect : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_periods_detect()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SeriesPeriodsValidate : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_periods_validate()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SeriesStatsDynamic : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_stats_dynamic()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SeriesStats : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_stats()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SeriesFillBackward : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_fill_backward()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SeriesFillConst : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_fill_const()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SeriesFillForward : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_fill_forward()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
class SeriesFillLinear : public IParserKQLFunction
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "series_fill_linear()"; }
|
||||
bool convertImpl(String & out, IParser::Pos & pos) override;
|
||||
};
|
||||
|
||||
}
|
202
src/Parsers/Kusto/ParserKQLDateTypeTimespan.cpp
Normal file
202
src/Parsers/Kusto/ParserKQLDateTypeTimespan.cpp
Normal file
@ -0,0 +1,202 @@
|
||||
#include <cmath>
|
||||
#include <cstdlib>
|
||||
#include <format>
|
||||
#include <unordered_map>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLDateTypeTimespan.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLDateTypeTimespan ::parseImpl(Pos & pos, [[maybe_unused]] ASTPtr & node, Expected & expected)
|
||||
{
|
||||
String token;
|
||||
const char * current_word = pos->begin;
|
||||
expected.add(pos, current_word);
|
||||
|
||||
if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral)
|
||||
token = String(pos->begin + 1, pos->end - 1);
|
||||
else
|
||||
token = String(pos->begin, pos->end);
|
||||
if (!parseConstKQLTimespan(token))
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
double ParserKQLDateTypeTimespan::toSeconds()
|
||||
{
|
||||
switch (time_span_unit)
|
||||
{
|
||||
case KQLTimespanUint::day:
|
||||
return time_span * 24 * 60 * 60;
|
||||
case KQLTimespanUint::hour:
|
||||
return time_span * 60 * 60;
|
||||
case KQLTimespanUint::minute:
|
||||
return time_span * 60;
|
||||
case KQLTimespanUint::second:
|
||||
return time_span;
|
||||
case KQLTimespanUint::millisec:
|
||||
return time_span / 1000.0;
|
||||
case KQLTimespanUint::microsec:
|
||||
return time_span / 1000000.0;
|
||||
case KQLTimespanUint::nanosec:
|
||||
return time_span / 1000000000.0;
|
||||
case KQLTimespanUint::tick:
|
||||
return time_span / 10000000.0;
|
||||
}
|
||||
}
|
||||
|
||||
bool ParserKQLDateTypeTimespan ::parseConstKQLTimespan(const String & text)
|
||||
{
|
||||
std::unordered_map<String, KQLTimespanUint> timespan_suffixes
|
||||
= {{"d", KQLTimespanUint::day},
|
||||
{"day", KQLTimespanUint::day},
|
||||
{"days", KQLTimespanUint::day},
|
||||
{"h", KQLTimespanUint::hour},
|
||||
{"hr", KQLTimespanUint::hour},
|
||||
{"hrs", KQLTimespanUint::hour},
|
||||
{"hour", KQLTimespanUint::hour},
|
||||
{"hours", KQLTimespanUint::hour},
|
||||
{"m", KQLTimespanUint::minute},
|
||||
{"min", KQLTimespanUint::minute},
|
||||
{"minute", KQLTimespanUint::minute},
|
||||
{"minutes", KQLTimespanUint::minute},
|
||||
{"s", KQLTimespanUint::second},
|
||||
{"sec", KQLTimespanUint::second},
|
||||
{"second", KQLTimespanUint::second},
|
||||
{"seconds", KQLTimespanUint::second},
|
||||
{"ms", KQLTimespanUint::millisec},
|
||||
{"milli", KQLTimespanUint::millisec},
|
||||
{"millis", KQLTimespanUint::millisec},
|
||||
{"millisec", KQLTimespanUint::millisec},
|
||||
{"millisecond", KQLTimespanUint::millisec},
|
||||
{"milliseconds", KQLTimespanUint::millisec},
|
||||
{"micro", KQLTimespanUint::microsec},
|
||||
{"micros", KQLTimespanUint::microsec},
|
||||
{"microsec", KQLTimespanUint::microsec},
|
||||
{"microsecond", KQLTimespanUint::microsec},
|
||||
{"microseconds", KQLTimespanUint::microsec},
|
||||
{"nano", KQLTimespanUint::nanosec},
|
||||
{"nanos", KQLTimespanUint::nanosec},
|
||||
{"nanosec", KQLTimespanUint::nanosec},
|
||||
{"nanosecond", KQLTimespanUint::nanosec},
|
||||
{"nanoseconds", KQLTimespanUint::nanosec},
|
||||
{"tick", KQLTimespanUint::tick},
|
||||
{"ticks", KQLTimespanUint::tick}};
|
||||
|
||||
int days = 0, hours = 0, minutes = 0, seconds = 0, sec_scale_len = 0;
|
||||
double nanoseconds = 00.00;
|
||||
|
||||
const char * ptr = text.c_str();
|
||||
bool sign = false;
|
||||
|
||||
auto scan_digit = [&](const char * start) -> int
|
||||
{
|
||||
const auto * index = start;
|
||||
while (isdigit(*index))
|
||||
++index;
|
||||
return index > start ? static_cast<int>(index - start) : -1;
|
||||
};
|
||||
if (*ptr == '-')
|
||||
{
|
||||
sign = true;
|
||||
++ptr;
|
||||
}
|
||||
auto number_len = scan_digit(ptr);
|
||||
if (number_len <= 0)
|
||||
return false;
|
||||
|
||||
days = std::stoi(String(ptr, ptr + number_len));
|
||||
|
||||
if (*(ptr + number_len) == '.')
|
||||
{
|
||||
auto fraction_len = scan_digit(ptr + number_len + 1);
|
||||
if (fraction_len >= 0)
|
||||
{
|
||||
hours = std::stoi(String(ptr + number_len + 1, ptr + number_len + 1 + fraction_len));
|
||||
number_len += fraction_len + 1;
|
||||
}
|
||||
}
|
||||
else if (*(ptr + number_len) == '\0')
|
||||
{
|
||||
if (sign)
|
||||
time_span = -(std::stoi(String(ptr, ptr + number_len))) * 86400;
|
||||
else
|
||||
time_span = std::stoi(String(ptr, ptr + number_len)) * 86400;
|
||||
|
||||
time_span_unit = KQLTimespanUint::second;
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
hours = days;
|
||||
days = 0;
|
||||
}
|
||||
|
||||
if (*(ptr + number_len) != ':')
|
||||
{
|
||||
String timespan_suffix(ptr + number_len, ptr + text.size());
|
||||
|
||||
trim(timespan_suffix);
|
||||
if (timespan_suffixes.find(timespan_suffix) == timespan_suffixes.end())
|
||||
return false;
|
||||
|
||||
time_span = std::stod(String(ptr, ptr + number_len));
|
||||
time_span_unit = timespan_suffixes[timespan_suffix];
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
if (hours > 23)
|
||||
return false;
|
||||
|
||||
auto min_len = scan_digit(ptr + number_len + 1);
|
||||
if (min_len < 0)
|
||||
return false;
|
||||
|
||||
minutes = std::stoi(String(ptr + number_len + 1, ptr + number_len + 1 + min_len));
|
||||
if (minutes > 59)
|
||||
return false;
|
||||
|
||||
number_len += min_len + 1;
|
||||
if (*(ptr + number_len) == ':')
|
||||
{
|
||||
auto sec_len = scan_digit(ptr + number_len + 1);
|
||||
if (sec_len > 0)
|
||||
{
|
||||
seconds = std::stoi(String(ptr + number_len + 1, ptr + number_len + 1 + sec_len));
|
||||
if (seconds > 59)
|
||||
return false;
|
||||
|
||||
number_len += sec_len + 1;
|
||||
if (*(ptr + number_len) == '.')
|
||||
{
|
||||
sec_scale_len = scan_digit(ptr + number_len + 1);
|
||||
if (sec_scale_len > 0)
|
||||
{
|
||||
nanoseconds = std::stoi(String(ptr + number_len + 1, ptr + number_len + 1 + sec_scale_len));
|
||||
|
||||
if (nanoseconds > 1000000000)
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
auto exponent = 9 - sec_scale_len; // max supported length of fraction of seconds is 9
|
||||
nanoseconds = nanoseconds * pow(10, exponent);
|
||||
|
||||
if (sign)
|
||||
time_span = -(days * 86400 + hours * 3600 + minutes * 60 + seconds + (nanoseconds / 1000000000));
|
||||
else
|
||||
time_span = days * 86400 + hours * 3600 + minutes * 60 + seconds + (nanoseconds / 1000000000);
|
||||
|
||||
time_span_unit = KQLTimespanUint::second;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
35
src/Parsers/Kusto/ParserKQLDateTypeTimespan.h
Normal file
35
src/Parsers/Kusto/ParserKQLDateTypeTimespan.h
Normal file
@ -0,0 +1,35 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserKQLDateTypeTimespan : public ParserKQLBase
|
||||
{
|
||||
public:
|
||||
enum class KQLTimespanUint : uint8_t
|
||||
{
|
||||
day,
|
||||
hour,
|
||||
minute,
|
||||
second,
|
||||
millisec,
|
||||
microsec,
|
||||
nanosec,
|
||||
tick
|
||||
};
|
||||
bool parseConstKQLTimespan(const String & text);
|
||||
double toSeconds();
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "KQLDateTypeTimespan"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
private:
|
||||
double time_span;
|
||||
KQLTimespanUint time_span_unit;
|
||||
};
|
||||
|
||||
}
|
26
src/Parsers/Kusto/ParserKQLDistinct.cpp
Normal file
26
src/Parsers/Kusto/ParserKQLDistinct.cpp
Normal file
@ -0,0 +1,26 @@
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/Kusto/ParserKQLDistinct.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLDistinct::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr select_expression_list;
|
||||
String expr;
|
||||
|
||||
expr = getExprFromToken(pos);
|
||||
|
||||
Tokens tokens(expr.c_str(), expr.c_str() + expr.size());
|
||||
IParser::Pos new_pos(tokens, pos.max_depth);
|
||||
|
||||
if (!ParserNotEmptyExpressionList(false).parse(new_pos, select_expression_list, expected))
|
||||
return false;
|
||||
|
||||
node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_expression_list));
|
||||
node->as<ASTSelectQuery>()->distinct = true;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
16
src/Parsers/Kusto/ParserKQLDistinct.h
Normal file
16
src/Parsers/Kusto/ParserKQLDistinct.h
Normal file
@ -0,0 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserKQLDistinct : public ParserKQLBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "KQL distinct"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
90
src/Parsers/Kusto/ParserKQLExtend.cpp
Normal file
90
src/Parsers/Kusto/ParserKQLExtend.cpp
Normal file
@ -0,0 +1,90 @@
|
||||
#include <format>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLExtend.h>
|
||||
#include <Parsers/Kusto/ParserKQLMakeSeries.h>
|
||||
#include <Parsers/Kusto/ParserKQLOperators.h>
|
||||
#include <Parsers/Kusto/ParserKQLProject.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/ParserSelectQuery.h>
|
||||
#include <Parsers/ParserTablesInSelectQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
bool ParserKQLExtend ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr select_query;
|
||||
int32_t new_column_index = 1;
|
||||
|
||||
String extend_expr = getExprFromToken(pos);
|
||||
|
||||
String except_str;
|
||||
String new_extend_str;
|
||||
Tokens ntokens(extend_expr.c_str(), extend_expr.c_str() + extend_expr.size());
|
||||
IParser::Pos npos(ntokens, pos.max_depth);
|
||||
|
||||
String alias;
|
||||
|
||||
auto apply_alias = [&]
|
||||
{
|
||||
if (alias.empty())
|
||||
{
|
||||
alias = std::format("Column{}", new_column_index);
|
||||
++new_column_index;
|
||||
new_extend_str += " AS";
|
||||
}
|
||||
else
|
||||
except_str = except_str.empty() ? " except " + alias : except_str + " except " + alias;
|
||||
|
||||
new_extend_str = new_extend_str + " " + alias;
|
||||
|
||||
alias.clear();
|
||||
};
|
||||
|
||||
int32_t round_bracket_count = 0;
|
||||
int32_t square_bracket_count = 0;
|
||||
while (!npos->isEnd())
|
||||
{
|
||||
if (npos->type == TokenType::OpeningRoundBracket)
|
||||
++round_bracket_count;
|
||||
if (npos->type == TokenType::OpeningSquareBracket)
|
||||
++square_bracket_count;
|
||||
if (npos->type == TokenType::ClosingRoundBracket)
|
||||
--round_bracket_count;
|
||||
if (npos->type == TokenType::ClosingSquareBracket)
|
||||
--square_bracket_count;
|
||||
|
||||
auto expr = String(npos->begin, npos->end);
|
||||
if (expr == "AS")
|
||||
{
|
||||
++npos;
|
||||
alias = String(npos->begin, npos->end);
|
||||
}
|
||||
|
||||
if (npos->type == TokenType::Comma && square_bracket_count == 0 && round_bracket_count == 0)
|
||||
{
|
||||
apply_alias();
|
||||
new_extend_str += ", ";
|
||||
}
|
||||
else
|
||||
new_extend_str = new_extend_str.empty() ? expr : new_extend_str + " " + expr;
|
||||
|
||||
++npos;
|
||||
}
|
||||
apply_alias();
|
||||
|
||||
String expr = std::format("SELECT * {}, {} from prev", except_str, new_extend_str);
|
||||
Tokens tokens(expr.c_str(), expr.c_str() + expr.size());
|
||||
IParser::Pos new_pos(tokens, pos.max_depth);
|
||||
|
||||
if (!ParserSelectQuery().parse(new_pos, select_query, expected))
|
||||
return false;
|
||||
if (!setSubQuerySource(select_query, node, false, false))
|
||||
return false;
|
||||
|
||||
node = select_query;
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
17
src/Parsers/Kusto/ParserKQLExtend.h
Normal file
17
src/Parsers/Kusto/ParserKQLExtend.h
Normal file
@ -0,0 +1,17 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLProject.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserKQLExtend : public ParserKQLBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "KQL extend"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
@ -1,19 +1,19 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLFilter.h>
|
||||
#include <Parsers/Kusto/ParserKQLOperators.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLFilter :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
bool ParserKQLFilter::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
String expr = getExprFromToken(pos);
|
||||
ASTPtr where_expression;
|
||||
|
||||
Tokens token_filter(expr.c_str(), expr.c_str()+expr.size());
|
||||
Tokens token_filter(expr.c_str(), expr.c_str() + expr.size());
|
||||
IParser::Pos pos_filter(token_filter, pos.max_depth);
|
||||
if (!ParserExpressionWithOptionalAlias(false).parse(pos_filter, where_expression, expected))
|
||||
return false;
|
||||
|
@ -1,15 +1,13 @@
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLLimit.h>
|
||||
#include <Parsers/ParserTablesInSelectQuery.h>
|
||||
#include <cstdlib>
|
||||
#include <format>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLLimit.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLLimit :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
bool ParserKQLLimit::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr limit_length;
|
||||
|
||||
|
311
src/Parsers/Kusto/ParserKQLMVExpand.cpp
Normal file
311
src/Parsers/Kusto/ParserKQLMVExpand.cpp
Normal file
@ -0,0 +1,311 @@
|
||||
#include <format>
|
||||
#include <unordered_map>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLMVExpand.h>
|
||||
#include <Parsers/Kusto/ParserKQLMakeSeries.h>
|
||||
#include <Parsers/Kusto/ParserKQLOperators.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/ParserSelectQuery.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
#include <Parsers/ParserTablesInSelectQuery.h>
|
||||
|
||||
namespace DB::ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_TYPE;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::unordered_map<String, String> ParserKQLMVExpand::type_cast
|
||||
= {{"bool", "Boolean"},
|
||||
{"boolean", "Boolean"},
|
||||
{"datetime", "DateTime"},
|
||||
{"date", "DateTime"},
|
||||
{"guid", "UUID"},
|
||||
{"int", "Int32"},
|
||||
{"long", "Int64"},
|
||||
{"real", "Float64"},
|
||||
{"double", "Float64"},
|
||||
{"string", "String"}};
|
||||
|
||||
bool ParserKQLMVExpand::parseColumnArrayExprs(ColumnArrayExprs & column_array_exprs, Pos & pos, Expected & expected)
|
||||
{
|
||||
ParserToken equals(TokenType::Equals);
|
||||
ParserToken open_bracket(TokenType::OpeningRoundBracket);
|
||||
ParserToken close_bracket(TokenType::ClosingRoundBracket);
|
||||
ParserToken comma(TokenType::Comma);
|
||||
|
||||
ParserKeyword s_to("to");
|
||||
ParserKeyword s_type("typeof");
|
||||
uint16_t bracket_count = 0;
|
||||
Pos expr_begin_pos = pos;
|
||||
Pos expr_end_pos = pos;
|
||||
|
||||
String alias;
|
||||
String column_array_expr;
|
||||
String to_type;
|
||||
--expr_end_pos;
|
||||
|
||||
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
if (pos->type == TokenType::OpeningRoundBracket)
|
||||
++bracket_count;
|
||||
|
||||
if (pos->type == TokenType::ClosingRoundBracket)
|
||||
--bracket_count;
|
||||
|
||||
if (String(pos->begin, pos->end) == "=")
|
||||
{
|
||||
--pos;
|
||||
alias = String(pos->begin, pos->end);
|
||||
++pos;
|
||||
++pos;
|
||||
expr_begin_pos = pos;
|
||||
}
|
||||
|
||||
auto add_columns = [&]
|
||||
{
|
||||
column_array_expr = getExprFromToken(String(expr_begin_pos->begin, expr_end_pos->end), pos.max_depth);
|
||||
|
||||
if (alias.empty())
|
||||
{
|
||||
alias = expr_begin_pos == expr_end_pos ? column_array_expr : String(expr_begin_pos->begin, expr_begin_pos->end) + "_";
|
||||
}
|
||||
column_array_exprs.push_back(ColumnArrayExpr(alias, column_array_expr, to_type));
|
||||
};
|
||||
|
||||
if (s_to.ignore(pos, expected))
|
||||
{
|
||||
--pos;
|
||||
--pos;
|
||||
expr_end_pos = pos;
|
||||
++pos;
|
||||
++pos;
|
||||
|
||||
column_array_expr = String(expr_begin_pos->begin, expr_end_pos->end);
|
||||
|
||||
if (!s_type.ignore(pos, expected))
|
||||
return false;
|
||||
if (!open_bracket.ignore(pos, expected))
|
||||
return false;
|
||||
to_type = String(pos->begin, pos->end);
|
||||
|
||||
if (type_cast.find(to_type) == type_cast.end())
|
||||
throw Exception(ErrorCodes::UNKNOWN_TYPE, "{} is not a supported kusto data type for mv-expand", to_type);
|
||||
|
||||
++pos;
|
||||
if (!close_bracket.ignore(pos, expected))
|
||||
return false;
|
||||
--pos;
|
||||
}
|
||||
|
||||
if ((pos->type == TokenType::Comma && bracket_count == 0) || String(pos->begin, pos->end) == "limit"
|
||||
|| pos->type == TokenType::Semicolon)
|
||||
{
|
||||
if (column_array_expr.empty())
|
||||
{
|
||||
expr_end_pos = pos;
|
||||
--expr_end_pos;
|
||||
}
|
||||
add_columns();
|
||||
expr_begin_pos = pos;
|
||||
expr_end_pos = pos;
|
||||
++expr_begin_pos;
|
||||
|
||||
alias.clear();
|
||||
column_array_expr.clear();
|
||||
to_type.clear();
|
||||
|
||||
if (pos->type == TokenType::Semicolon)
|
||||
break;
|
||||
}
|
||||
|
||||
if (String(pos->begin, pos->end) == "limit")
|
||||
break;
|
||||
if (!pos->isEnd())
|
||||
++pos;
|
||||
if (pos->isEnd() || pos->type == TokenType::PipeMark || pos->type == TokenType::Semicolon)
|
||||
{
|
||||
if (expr_end_pos < expr_begin_pos)
|
||||
{
|
||||
expr_end_pos = pos;
|
||||
--expr_end_pos;
|
||||
}
|
||||
add_columns();
|
||||
break;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserKQLMVExpand::parserMVExpand(KQLMVExpand & kql_mv_expand, Pos & pos, Expected & expected)
|
||||
{
|
||||
ParserKeyword s_bagexpansion("bagexpansion");
|
||||
ParserKeyword s_kind("kind");
|
||||
ParserKeyword s_with_itemindex("with_itemindex");
|
||||
ParserKeyword s_limit("limit");
|
||||
|
||||
ParserToken equals(TokenType::Equals);
|
||||
ParserToken comma(TokenType::Comma);
|
||||
|
||||
auto & column_array_exprs = kql_mv_expand.column_array_exprs;
|
||||
auto & bagexpansion = kql_mv_expand.bagexpansion;
|
||||
auto & with_itemindex = kql_mv_expand.with_itemindex;
|
||||
auto & limit = kql_mv_expand.limit;
|
||||
|
||||
if (s_bagexpansion.ignore(pos, expected))
|
||||
{
|
||||
if (!equals.ignore(pos, expected))
|
||||
return false;
|
||||
bagexpansion = String(pos->begin, pos->end);
|
||||
++pos;
|
||||
}
|
||||
else if (s_kind.ignore(pos, expected))
|
||||
{
|
||||
if (!equals.ignore(pos, expected))
|
||||
return false;
|
||||
bagexpansion = String(pos->begin, pos->end);
|
||||
++pos;
|
||||
}
|
||||
|
||||
if (s_with_itemindex.ignore(pos, expected))
|
||||
{
|
||||
if (!equals.ignore(pos, expected))
|
||||
return false;
|
||||
with_itemindex = String(pos->begin, pos->end);
|
||||
++pos;
|
||||
}
|
||||
|
||||
if (!parseColumnArrayExprs(column_array_exprs, pos, expected))
|
||||
return false;
|
||||
|
||||
if (s_limit.ignore(pos, expected))
|
||||
limit = String(pos->begin, pos->end);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserKQLMVExpand::genQuery(KQLMVExpand & kql_mv_expand, ASTPtr & select_node, int32_t max_depth)
|
||||
{
|
||||
String expand_str;
|
||||
String cast_type_column_remove, cast_type_column_rename;
|
||||
String cast_type_column_restore, cast_type_column_restore_name;
|
||||
String row_count_str;
|
||||
String extra_columns;
|
||||
String input = "dummy_input";
|
||||
for (auto column : kql_mv_expand.column_array_exprs)
|
||||
{
|
||||
if (column.alias == column.column_array_expr)
|
||||
expand_str = expand_str.empty() ? String("ARRAY JOIN ") + column.alias : expand_str + "," + column.alias;
|
||||
else
|
||||
{
|
||||
expand_str = expand_str.empty() ? std::format("ARRAY JOIN {} AS {} ", column.column_array_expr, column.alias)
|
||||
: expand_str + std::format(", {} AS {}", column.column_array_expr, column.alias);
|
||||
extra_columns = extra_columns + ", " + column.alias;
|
||||
}
|
||||
|
||||
if (!column.to_type.empty())
|
||||
{
|
||||
cast_type_column_remove
|
||||
= cast_type_column_remove.empty() ? " Except " + column.alias : cast_type_column_remove + " Except " + column.alias;
|
||||
String rename_str;
|
||||
|
||||
if (type_cast[column.to_type] == "Boolean")
|
||||
rename_str = std::format(
|
||||
"accurateCastOrNull(toInt64OrNull(toString({0})),'{1}') as {0}_ali", column.alias, type_cast[column.to_type]);
|
||||
else
|
||||
rename_str = std::format("accurateCastOrNull({0},'{1}') as {0}_ali", column.alias, type_cast[column.to_type]);
|
||||
|
||||
cast_type_column_rename = cast_type_column_rename.empty() ? rename_str : cast_type_column_rename + "," + rename_str;
|
||||
cast_type_column_restore = cast_type_column_restore.empty()
|
||||
? std::format(" Except {}_ali ", column.alias)
|
||||
: cast_type_column_restore + std::format(" Except {}_ali ", column.alias);
|
||||
cast_type_column_restore_name = cast_type_column_restore_name.empty()
|
||||
? std::format("{0}_ali as {0}", column.alias)
|
||||
: cast_type_column_restore_name + std::format(", {0}_ali as {0}", column.alias);
|
||||
}
|
||||
|
||||
if (!kql_mv_expand.with_itemindex.empty())
|
||||
{
|
||||
row_count_str = row_count_str.empty() ? "length(" + column.alias + ")" : row_count_str + ", length(" + column.alias + ")";
|
||||
}
|
||||
}
|
||||
|
||||
String columns = "*";
|
||||
if (!row_count_str.empty())
|
||||
{
|
||||
expand_str += std::format(", range(0, arrayMax([{}])) AS {} ", row_count_str, kql_mv_expand.with_itemindex);
|
||||
columns = kql_mv_expand.with_itemindex + " , " + columns;
|
||||
}
|
||||
|
||||
if (!kql_mv_expand.limit.empty())
|
||||
expand_str += " LIMIT " + kql_mv_expand.limit;
|
||||
|
||||
auto query = std::format("(Select {} {} From {} {})", columns, extra_columns, input, expand_str);
|
||||
|
||||
ASTPtr sub_query_node;
|
||||
Expected expected;
|
||||
|
||||
if (cast_type_column_remove.empty())
|
||||
{
|
||||
query = std::format("Select {} {} From {} {}", columns, extra_columns, input, expand_str);
|
||||
if (!parseSQLQueryByString(std::make_unique<ParserSelectQuery>(), query, sub_query_node, max_depth))
|
||||
return false;
|
||||
if (!setSubQuerySource(sub_query_node, select_node, false, false))
|
||||
return false;
|
||||
select_node = std::move(sub_query_node);
|
||||
}
|
||||
else
|
||||
{
|
||||
query = std::format("(Select {} {} From {} {})", columns, extra_columns, input, expand_str);
|
||||
if (!parseSQLQueryByString(std::make_unique<ParserTablesInSelectQuery>(), query, sub_query_node, max_depth))
|
||||
return false;
|
||||
if (!setSubQuerySource(sub_query_node, select_node, true, false))
|
||||
return false;
|
||||
select_node = std::move(sub_query_node);
|
||||
|
||||
auto rename_query = std::format("(Select * {}, {} From {})", cast_type_column_remove, cast_type_column_rename, "query");
|
||||
if (!parseSQLQueryByString(std::make_unique<ParserTablesInSelectQuery>(), rename_query, sub_query_node, max_depth))
|
||||
return false;
|
||||
if (!setSubQuerySource(sub_query_node, select_node, true, true))
|
||||
return false;
|
||||
|
||||
select_node = std::move(sub_query_node);
|
||||
query = std::format("Select * {}, {} from {}", cast_type_column_restore, cast_type_column_restore_name, "rename_query");
|
||||
|
||||
if (!parseSQLQueryByString(std::make_unique<ParserSelectQuery>(), query, sub_query_node, max_depth))
|
||||
return false;
|
||||
sub_query_node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::TABLES, std::move(select_node));
|
||||
select_node = std::move(sub_query_node);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserKQLMVExpand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr setting;
|
||||
ASTPtr select_expression_list;
|
||||
auto begin = pos;
|
||||
|
||||
KQLMVExpand kql_mv_expand;
|
||||
if (!parserMVExpand(kql_mv_expand, pos, expected))
|
||||
return false;
|
||||
if (!genQuery(kql_mv_expand, node, pos.max_depth))
|
||||
return false;
|
||||
|
||||
const String setting_str = "enable_unaligned_array_join = 1";
|
||||
Tokens token_settings(setting_str.c_str(), setting_str.c_str() + setting_str.size());
|
||||
IParser::Pos pos_settings(token_settings, pos.max_depth);
|
||||
|
||||
if (!ParserSetQuery(true).parse(pos_settings, setting, expected))
|
||||
return false;
|
||||
node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(setting));
|
||||
|
||||
pos = begin;
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
41
src/Parsers/Kusto/ParserKQLMVExpand.h
Normal file
41
src/Parsers/Kusto/ParserKQLMVExpand.h
Normal file
@ -0,0 +1,41 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserKQLMVExpand : public ParserKQLBase
|
||||
{
|
||||
protected:
|
||||
static std::unordered_map<String, String> type_cast;
|
||||
|
||||
struct ColumnArrayExpr
|
||||
{
|
||||
String alias;
|
||||
String column_array_expr;
|
||||
String to_type;
|
||||
ColumnArrayExpr(String alias_, String column_array_expr_, String to_type_)
|
||||
: alias(alias_), column_array_expr(column_array_expr_), to_type(to_type_)
|
||||
{
|
||||
}
|
||||
};
|
||||
using ColumnArrayExprs = std::vector<ColumnArrayExpr>;
|
||||
|
||||
struct KQLMVExpand
|
||||
{
|
||||
ColumnArrayExprs column_array_exprs;
|
||||
String bagexpansion;
|
||||
String with_itemindex;
|
||||
String limit;
|
||||
};
|
||||
|
||||
static bool parseColumnArrayExprs(ColumnArrayExprs & column_array_exprs, Pos & pos, Expected & expected);
|
||||
static bool parserMVExpand(KQLMVExpand & kql_mv_expand, Pos & pos, Expected & expected);
|
||||
static bool genQuery(KQLMVExpand & kql_mv_expand, ASTPtr & select_node, int32_t max_depth);
|
||||
|
||||
const char * getName() const override { return "KQL mv-expand"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
}
|
426
src/Parsers/Kusto/ParserKQLMakeSeries.cpp
Normal file
426
src/Parsers/Kusto/ParserKQLMakeSeries.cpp
Normal file
@ -0,0 +1,426 @@
|
||||
#include <format>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLDateTypeTimespan.h>
|
||||
#include <Parsers/Kusto/ParserKQLMakeSeries.h>
|
||||
#include <Parsers/Kusto/ParserKQLOperators.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/ParserSelectQuery.h>
|
||||
#include <Parsers/ParserTablesInSelectQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLMakeSeries ::parseAggregationColumns(AggregationColumns & aggregation_columns, Pos & pos)
|
||||
{
|
||||
std::unordered_set<String> allowed_aggregation(
|
||||
{"avg",
|
||||
"avgif",
|
||||
"count",
|
||||
"countif",
|
||||
"dcount",
|
||||
"dcountif",
|
||||
"max",
|
||||
"maxif",
|
||||
"min",
|
||||
"minif",
|
||||
"percentile",
|
||||
"take_any",
|
||||
"stdev",
|
||||
"sum",
|
||||
"sumif",
|
||||
"variance"});
|
||||
|
||||
Expected expected;
|
||||
ParserKeyword s_default("default");
|
||||
ParserToken equals(TokenType::Equals);
|
||||
ParserToken open_bracket(TokenType::OpeningRoundBracket);
|
||||
ParserToken close_bracket(TokenType::ClosingRoundBracket);
|
||||
ParserToken comma(TokenType::Comma);
|
||||
|
||||
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
String alias;
|
||||
String aggregation_fun;
|
||||
String column;
|
||||
double default_value = 0;
|
||||
|
||||
String first_token(pos->begin, pos->end);
|
||||
|
||||
++pos;
|
||||
if (equals.ignore(pos, expected))
|
||||
{
|
||||
alias = std::move(first_token);
|
||||
aggregation_fun = String(pos->begin, pos->end);
|
||||
++pos;
|
||||
}
|
||||
else
|
||||
aggregation_fun = std::move(first_token);
|
||||
|
||||
if (allowed_aggregation.find(aggregation_fun) == allowed_aggregation.end())
|
||||
return false;
|
||||
|
||||
if (open_bracket.ignore(pos, expected))
|
||||
column = String(pos->begin, pos->end);
|
||||
else
|
||||
return false;
|
||||
|
||||
++pos;
|
||||
if (!close_bracket.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (s_default.ignore(pos, expected))
|
||||
{
|
||||
if (!equals.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
default_value = std::stod(String(pos->begin, pos->end));
|
||||
++pos;
|
||||
}
|
||||
if (alias.empty())
|
||||
alias = std::format("{}_{}", aggregation_fun, column);
|
||||
aggregation_columns.push_back(AggregationColumn(alias, aggregation_fun, column, default_value));
|
||||
|
||||
if (!comma.ignore(pos, expected))
|
||||
break;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserKQLMakeSeries ::parseFromToStepClause(FromToStepClause & from_to_step, Pos & pos)
|
||||
{
|
||||
auto begin = pos;
|
||||
auto from_pos = begin;
|
||||
auto to_pos = begin;
|
||||
auto step_pos = begin;
|
||||
auto end_pos = begin;
|
||||
|
||||
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
if (String(pos->begin, pos->end) == "from")
|
||||
from_pos = pos;
|
||||
if (String(pos->begin, pos->end) == "to")
|
||||
to_pos = pos;
|
||||
if (String(pos->begin, pos->end) == "step")
|
||||
step_pos = pos;
|
||||
if (String(pos->begin, pos->end) == "by")
|
||||
{
|
||||
end_pos = pos;
|
||||
break;
|
||||
}
|
||||
++pos;
|
||||
}
|
||||
|
||||
if (end_pos == begin)
|
||||
end_pos = pos;
|
||||
|
||||
if (String(step_pos->begin, step_pos->end) != "step")
|
||||
return false;
|
||||
|
||||
if (String(from_pos->begin, from_pos->end) == "from")
|
||||
{
|
||||
++from_pos;
|
||||
auto end_from_pos = (to_pos != begin) ? to_pos : step_pos;
|
||||
--end_from_pos;
|
||||
from_to_step.from_str = String(from_pos->begin, end_from_pos->end);
|
||||
}
|
||||
|
||||
if (String(to_pos->begin, to_pos->end) == "to")
|
||||
{
|
||||
++to_pos;
|
||||
--step_pos;
|
||||
from_to_step.to_str = String(to_pos->begin, step_pos->end);
|
||||
++step_pos;
|
||||
}
|
||||
--end_pos;
|
||||
++step_pos;
|
||||
from_to_step.step_str = String(step_pos->begin, end_pos->end);
|
||||
|
||||
if (String(step_pos->begin, step_pos->end) == "time" || String(step_pos->begin, step_pos->end) == "timespan"
|
||||
|| ParserKQLDateTypeTimespan().parseConstKQLTimespan(from_to_step.step_str))
|
||||
{
|
||||
from_to_step.is_timespan = true;
|
||||
from_to_step.step = std::stod(getExprFromToken(from_to_step.step_str, pos.max_depth));
|
||||
}
|
||||
else
|
||||
from_to_step.step = std::stod(from_to_step.step_str);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserKQLMakeSeries ::makeSeries(KQLMakeSeries & kql_make_series, ASTPtr & select_node, const uint32_t & max_depth)
|
||||
{
|
||||
const uint64_t era_diff
|
||||
= 62135596800; // this magic number is the differicen is second form 0001-01-01 (Azure start time ) and 1970-01-01 (CH start time)
|
||||
|
||||
String start_str, end_str;
|
||||
String sub_query, main_query;
|
||||
|
||||
auto & aggregation_columns = kql_make_series.aggregation_columns;
|
||||
auto & from_to_step = kql_make_series.from_to_step;
|
||||
auto & subquery_columns = kql_make_series.subquery_columns;
|
||||
auto & axis_column = kql_make_series.axis_column;
|
||||
auto & group_expression = kql_make_series.group_expression;
|
||||
auto step = from_to_step.step;
|
||||
|
||||
if (!kql_make_series.from_to_step.from_str.empty())
|
||||
start_str = getExprFromToken(kql_make_series.from_to_step.from_str, max_depth);
|
||||
|
||||
if (!kql_make_series.from_to_step.to_str.empty())
|
||||
end_str = getExprFromToken(from_to_step.to_str, max_depth);
|
||||
|
||||
auto date_type_cast = [&](String & src)
|
||||
{
|
||||
Tokens tokens(src.c_str(), src.c_str() + src.size());
|
||||
IParser::Pos pos(tokens, max_depth);
|
||||
String res;
|
||||
while (!pos->isEnd())
|
||||
{
|
||||
String tmp = String(pos->begin, pos->end);
|
||||
if (tmp == "parseDateTime64BestEffortOrNull")
|
||||
tmp = "toDateTime64";
|
||||
|
||||
res = res.empty() ? tmp : res + " " + tmp;
|
||||
++pos;
|
||||
}
|
||||
return res;
|
||||
};
|
||||
|
||||
start_str = date_type_cast(start_str);
|
||||
end_str = date_type_cast(end_str);
|
||||
|
||||
String bin_str, start, end;
|
||||
|
||||
uint64_t diff = 0;
|
||||
String axis_column_format;
|
||||
String axis_str;
|
||||
|
||||
auto get_group_expression_alias = [&]
|
||||
{
|
||||
std::vector<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())
|
||||
{
|
||||
if (String(pos->begin, pos->end) == "AS")
|
||||
{
|
||||
if (!group_expression_tokens.empty())
|
||||
group_expression_tokens.pop_back();
|
||||
++pos;
|
||||
group_expression_tokens.push_back(String(pos->begin, pos->end));
|
||||
}
|
||||
else
|
||||
group_expression_tokens.push_back(String(pos->begin, pos->end));
|
||||
++pos;
|
||||
}
|
||||
String res;
|
||||
for (auto const & token : group_expression_tokens)
|
||||
res = res + token + " ";
|
||||
return res;
|
||||
};
|
||||
|
||||
auto group_expression_alias = get_group_expression_alias();
|
||||
|
||||
if (from_to_step.is_timespan)
|
||||
{
|
||||
axis_column_format = std::format("toFloat64(toDateTime64({}, 9, 'UTC'))", axis_column);
|
||||
}
|
||||
else
|
||||
axis_column_format = std::format("toFloat64({})", axis_column);
|
||||
|
||||
if (!start_str.empty()) // has from
|
||||
{
|
||||
bin_str = std::format(
|
||||
"toFloat64({0}) + (toInt64((({1} - toFloat64({0})) / {2})) * {2}) AS {3}_ali",
|
||||
start_str,
|
||||
axis_column_format,
|
||||
step,
|
||||
axis_column);
|
||||
start = std::format("toUInt64({})", start_str);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (from_to_step.is_timespan)
|
||||
diff = era_diff;
|
||||
bin_str = std::format(" toFloat64(toInt64(({0} + {1}) / {2}) * {2}) AS {3}_ali ", axis_column_format, diff, step, axis_column);
|
||||
}
|
||||
|
||||
if (!end_str.empty())
|
||||
end = std::format("toUInt64({})", end_str);
|
||||
|
||||
String range, condition;
|
||||
|
||||
if (!start_str.empty() && !end_str.empty())
|
||||
{
|
||||
range = std::format("range({}, {}, toUInt64({}))", start, end, step);
|
||||
condition = std::format("where toInt64({0}) >= {1} and toInt64({0}) < {2}", axis_column_format, start, end);
|
||||
}
|
||||
else if (start_str.empty() && !end_str.empty())
|
||||
{
|
||||
range = std::format("range(low, {} + {}, toUInt64({}))", end, diff, step);
|
||||
condition = std::format("where toInt64({0}) - {1} < {2}", axis_column_format, diff, end);
|
||||
}
|
||||
else if (!start_str.empty() && end_str.empty())
|
||||
{
|
||||
range = std::format("range({}, high, toUInt64({}))", start, step);
|
||||
condition = std::format("where toInt64({}) >= {}", axis_column_format, start);
|
||||
}
|
||||
else
|
||||
{
|
||||
range = std::format("range(low, high, toUInt64({}))", step);
|
||||
condition = " ";
|
||||
}
|
||||
|
||||
auto range_len = std::format("length({})", range);
|
||||
|
||||
String sub_sub_query;
|
||||
if (group_expression.empty())
|
||||
sub_sub_query = std::format(
|
||||
" (Select {0}, {1} FROM {2} {4} GROUP BY {3}_ali ORDER BY {3}_ali) ",
|
||||
subquery_columns,
|
||||
bin_str,
|
||||
"table_name",
|
||||
axis_column,
|
||||
condition);
|
||||
else
|
||||
sub_sub_query = std::format(
|
||||
" (Select {0}, {1}, {2} FROM {3} {5} GROUP BY {0}, {4}_ali ORDER BY {4}_ali) ",
|
||||
group_expression,
|
||||
subquery_columns,
|
||||
bin_str,
|
||||
"table_name",
|
||||
axis_column,
|
||||
condition);
|
||||
|
||||
ASTPtr sub_query_node;
|
||||
|
||||
if (!ParserSimpleCHSubquery(select_node).parseByString(sub_sub_query, sub_query_node, max_depth))
|
||||
return false;
|
||||
select_node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::TABLES, std::move(sub_query_node));
|
||||
|
||||
if (!group_expression.empty())
|
||||
main_query = std::format("{} ", group_expression_alias);
|
||||
|
||||
auto axis_and_agg_alias_list = axis_column;
|
||||
auto final_axis_agg_alias_list = std::format("tupleElement(zipped,1) AS {}", axis_column);
|
||||
int idx = 2;
|
||||
for (auto agg_column : aggregation_columns)
|
||||
{
|
||||
String agg_group_column = std::format(
|
||||
"arrayConcat(groupArray({}_ali) as ga, arrayMap(x -> ({}),range(0, toUInt32({} - length(ga) < 0 ? 0 : {} - length(ga)),1)))"
|
||||
"as {}",
|
||||
agg_column.alias,
|
||||
agg_column.default_value,
|
||||
range_len,
|
||||
range_len,
|
||||
agg_column.alias);
|
||||
main_query = main_query.empty() ? agg_group_column : main_query + ", " + agg_group_column;
|
||||
|
||||
axis_and_agg_alias_list += ", " + agg_column.alias;
|
||||
final_axis_agg_alias_list += std::format(", tupleElement(zipped,{}) AS {}", idx, agg_column.alias);
|
||||
}
|
||||
|
||||
if (from_to_step.is_timespan)
|
||||
axis_str = std::format(
|
||||
"arrayDistinct(arrayConcat(groupArray(toDateTime64({0}_ali - {1},9,'UTC')), arrayMap(x->(toDateTime64(x - {1} ,9,'UTC')),"
|
||||
"{2}))) as {0}",
|
||||
axis_column,
|
||||
diff,
|
||||
range);
|
||||
else
|
||||
axis_str
|
||||
= std::format("arrayDistinct(arrayConcat(groupArray({0}_ali), arrayMap(x->(toFloat64(x)), {1}))) as {0}", axis_column, range);
|
||||
|
||||
main_query += ", " + axis_str;
|
||||
auto sub_group_by = group_expression.empty() ? "" : std::format("GROUP BY {}", group_expression_alias);
|
||||
|
||||
sub_query = std::format(
|
||||
"( SELECT toUInt64(min({}_ali)) AS low, toUInt64(max({}_ali))+ {} AS high, arraySort(arrayZip({})) as zipped, {} FROM {} {} )",
|
||||
axis_column,
|
||||
axis_column,
|
||||
step,
|
||||
axis_and_agg_alias_list,
|
||||
main_query,
|
||||
sub_sub_query,
|
||||
sub_group_by);
|
||||
|
||||
if (group_expression.empty())
|
||||
main_query = std::format("{}", final_axis_agg_alias_list);
|
||||
else
|
||||
main_query = std::format("{},{}", group_expression_alias, final_axis_agg_alias_list);
|
||||
|
||||
if (!ParserSimpleCHSubquery(select_node).parseByString(sub_query, sub_query_node, max_depth))
|
||||
return false;
|
||||
select_node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::TABLES, std::move(sub_query_node));
|
||||
|
||||
kql_make_series.sub_query = std::move(sub_query);
|
||||
kql_make_series.main_query = std::move(main_query);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserKQLMakeSeries ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
auto begin = pos;
|
||||
ParserKeyword s_on("on");
|
||||
ParserKeyword s_by("by");
|
||||
|
||||
ParserToken equals(TokenType::Equals);
|
||||
ParserToken comma(TokenType::Comma);
|
||||
|
||||
ASTPtr select_expression_list;
|
||||
|
||||
KQLMakeSeries kql_make_series;
|
||||
auto & aggregation_columns = kql_make_series.aggregation_columns;
|
||||
auto & from_to_step = kql_make_series.from_to_step;
|
||||
auto & subquery_columns = kql_make_series.subquery_columns;
|
||||
auto & axis_column = kql_make_series.axis_column;
|
||||
auto & group_expression = kql_make_series.group_expression;
|
||||
|
||||
ParserKQLDateTypeTimespan time_span;
|
||||
|
||||
//const auto make_series_parameters = getMakeSeriesParameters(pos);
|
||||
|
||||
if (!parseAggregationColumns(aggregation_columns, pos))
|
||||
return false;
|
||||
|
||||
if (!s_on.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
axis_column = String(pos->begin, pos->end);
|
||||
++pos;
|
||||
|
||||
if (!parseFromToStepClause(from_to_step, pos))
|
||||
return false;
|
||||
|
||||
if (s_by.ignore(pos, expected))
|
||||
{
|
||||
group_expression = getExprFromToken(pos);
|
||||
if (group_expression.empty())
|
||||
return false;
|
||||
}
|
||||
|
||||
for (auto agg_column : aggregation_columns)
|
||||
{
|
||||
String column_str = std::format("{}({}) AS {}_ali", agg_column.aggregation_fun, agg_column.column, agg_column.alias);
|
||||
if (subquery_columns.empty())
|
||||
subquery_columns = column_str;
|
||||
else
|
||||
subquery_columns += ", " + column_str;
|
||||
}
|
||||
|
||||
makeSeries(kql_make_series, node, pos.max_depth);
|
||||
|
||||
Tokens token_main_query(kql_make_series.main_query.c_str(), kql_make_series.main_query.c_str() + kql_make_series.main_query.size());
|
||||
IParser::Pos pos_main_query(token_main_query, pos.max_depth);
|
||||
|
||||
if (!ParserNotEmptyExpressionList(true).parse(pos_main_query, select_expression_list, expected))
|
||||
return false;
|
||||
node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_expression_list));
|
||||
|
||||
pos = begin;
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
53
src/Parsers/Kusto/ParserKQLMakeSeries.h
Normal file
53
src/Parsers/Kusto/ParserKQLMakeSeries.h
Normal file
@ -0,0 +1,53 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserKQLMakeSeries : public ParserKQLBase
|
||||
{
|
||||
protected:
|
||||
struct AggregationColumn
|
||||
{
|
||||
String alias;
|
||||
String aggregation_fun;
|
||||
String column;
|
||||
double default_value;
|
||||
AggregationColumn(String alias_, String aggregation_fun_, String column_, double default_value_)
|
||||
: alias(alias_), aggregation_fun(aggregation_fun_), column(column_), default_value(default_value_)
|
||||
{
|
||||
}
|
||||
};
|
||||
using AggregationColumns = std::vector<AggregationColumn>;
|
||||
|
||||
struct FromToStepClause
|
||||
{
|
||||
String from_str;
|
||||
String to_str;
|
||||
String step_str;
|
||||
bool is_timespan = false;
|
||||
double step;
|
||||
};
|
||||
|
||||
struct KQLMakeSeries
|
||||
{
|
||||
AggregationColumns aggregation_columns;
|
||||
FromToStepClause from_to_step;
|
||||
String axis_column;
|
||||
String group_expression;
|
||||
String subquery_columns;
|
||||
String sub_query;
|
||||
String main_query;
|
||||
};
|
||||
|
||||
static bool makeSeries(KQLMakeSeries & kql_make_series, ASTPtr & select_node, const uint32_t & max_depth);
|
||||
static bool parseAggregationColumns(AggregationColumns & aggregation_columns, Pos & pos);
|
||||
static bool parseFromToStepClause(FromToStepClause & from_to_step, Pos & pos);
|
||||
|
||||
const char * getName() const override { return "KQL make-series"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
@ -1,9 +1,131 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLOperators.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/IParserKQLFunction.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h>
|
||||
#include <Parsers/Kusto/ParserKQLOperators.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include "KustoFunctions/IParserKQLFunction.h"
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
enum class KQLOperatorValue : uint16_t
|
||||
{
|
||||
none,
|
||||
between,
|
||||
not_between,
|
||||
contains,
|
||||
not_contains,
|
||||
contains_cs,
|
||||
not_contains_cs,
|
||||
endswith,
|
||||
not_endswith,
|
||||
endswith_cs,
|
||||
not_endswith_cs,
|
||||
equal, //=~
|
||||
not_equal, //!~
|
||||
equal_cs, //=
|
||||
not_equal_cs, //!=
|
||||
has,
|
||||
not_has,
|
||||
has_all,
|
||||
has_any,
|
||||
has_cs,
|
||||
not_has_cs,
|
||||
hasprefix,
|
||||
not_hasprefix,
|
||||
hasprefix_cs,
|
||||
not_hasprefix_cs,
|
||||
hassuffix,
|
||||
not_hassuffix,
|
||||
hassuffix_cs,
|
||||
not_hassuffix_cs,
|
||||
in_cs, //in
|
||||
not_in_cs, //!in
|
||||
in, //in~
|
||||
not_in, //!in~
|
||||
matches_regex,
|
||||
startswith,
|
||||
not_startswith,
|
||||
startswith_cs,
|
||||
not_startswith_cs,
|
||||
};
|
||||
|
||||
const std::unordered_map<String, KQLOperatorValue> KQLOperator = {
|
||||
{"between", KQLOperatorValue::between},
|
||||
{"!between", KQLOperatorValue::not_between},
|
||||
{"contains", KQLOperatorValue::contains},
|
||||
{"!contains", KQLOperatorValue::not_contains},
|
||||
{"contains_cs", KQLOperatorValue::contains_cs},
|
||||
{"!contains_cs", KQLOperatorValue::not_contains_cs},
|
||||
{"endswith", KQLOperatorValue::endswith},
|
||||
{"!endswith", KQLOperatorValue::not_endswith},
|
||||
{"endswith_cs", KQLOperatorValue::endswith_cs},
|
||||
{"!endswith_cs", KQLOperatorValue::not_endswith_cs},
|
||||
{"=~", KQLOperatorValue::equal},
|
||||
{"!~", KQLOperatorValue::not_equal},
|
||||
{"==", KQLOperatorValue::equal_cs},
|
||||
{"!=", KQLOperatorValue::not_equal_cs},
|
||||
{"has", KQLOperatorValue::has},
|
||||
{"!has", KQLOperatorValue::not_has},
|
||||
{"has_all", KQLOperatorValue::has_all},
|
||||
{"has_any", KQLOperatorValue::has_any},
|
||||
{"has_cs", KQLOperatorValue::has_cs},
|
||||
{"!has_cs", KQLOperatorValue::not_has_cs},
|
||||
{"hasprefix", KQLOperatorValue::hasprefix},
|
||||
{"!hasprefix", KQLOperatorValue::not_hasprefix},
|
||||
{"hasprefix_cs", KQLOperatorValue::hasprefix_cs},
|
||||
{"!hasprefix_cs", KQLOperatorValue::not_hasprefix_cs},
|
||||
{"hassuffix", KQLOperatorValue::hassuffix},
|
||||
{"!hassuffix", KQLOperatorValue::not_hassuffix},
|
||||
{"hassuffix_cs", KQLOperatorValue::hassuffix_cs},
|
||||
{"!hassuffix_cs", KQLOperatorValue::not_hassuffix_cs},
|
||||
{"in", KQLOperatorValue::in_cs},
|
||||
{"!in", KQLOperatorValue::not_in_cs},
|
||||
{"in~", KQLOperatorValue::in},
|
||||
{"!in~", KQLOperatorValue::not_in},
|
||||
{"matches regex", KQLOperatorValue::matches_regex},
|
||||
{"startswith", KQLOperatorValue::startswith},
|
||||
{"!startswith", KQLOperatorValue::not_startswith},
|
||||
{"startswith_cs", KQLOperatorValue::startswith_cs},
|
||||
{"!startswith_cs", KQLOperatorValue::not_startswith_cs},
|
||||
};
|
||||
|
||||
void rebuildSubqueryForInOperator(DB::ASTPtr & node, bool useLowerCase)
|
||||
{
|
||||
//A sub-query for in operator in kql can have multiple columns, but only takes the first column.
|
||||
//A sub-query for in operator in ClickHouse can not have multiple columns
|
||||
//So only take the first column if there are multiple columns.
|
||||
//select * not working for subquery. (a tabular statement without project)
|
||||
|
||||
const auto selectColumns = node->children[0]->children[0]->as<DB::ASTSelectQuery>()->select();
|
||||
while (selectColumns->children.size() > 1)
|
||||
selectColumns->children.pop_back();
|
||||
|
||||
if (useLowerCase)
|
||||
{
|
||||
auto args = std::make_shared<DB::ASTExpressionList>();
|
||||
args->children.push_back(selectColumns->children[0]);
|
||||
auto func_lower = std::make_shared<DB::ASTFunction>();
|
||||
func_lower->name = "lower";
|
||||
func_lower->children.push_back(selectColumns->children[0]);
|
||||
func_lower->arguments = args;
|
||||
if (selectColumns->children[0]->as<DB::ASTIdentifier>())
|
||||
func_lower->alias = std::move(selectColumns->children[0]->as<DB::ASTIdentifier>()->alias);
|
||||
else if (selectColumns->children[0]->as<DB::ASTFunction>())
|
||||
func_lower->alias = std::move(selectColumns->children[0]->as<DB::ASTFunction>()->alias);
|
||||
|
||||
auto funcs = std::make_shared<DB::ASTExpressionList>();
|
||||
funcs->children.push_back(func_lower);
|
||||
selectColumns->children[0] = std::move(funcs);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -12,7 +134,7 @@ namespace ErrorCodes
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
String KQLOperators::genHasAnyAllOpExpr(std::vector<String> &tokens, IParser::Pos &token_pos,String kql_op, String ch_op)
|
||||
String KQLOperators::genHasAnyAllOpExpr(std::vector<String> & tokens, IParser::Pos & token_pos, String kql_op, String ch_op)
|
||||
{
|
||||
String new_expr;
|
||||
Expected expected;
|
||||
@ -28,7 +150,7 @@ String KQLOperators::genHasAnyAllOpExpr(std::vector<String> &tokens, IParser::Po
|
||||
|
||||
while (!token_pos->isEnd() && token_pos->type != TokenType::PipeMark && token_pos->type != TokenType::Semicolon)
|
||||
{
|
||||
auto tmp_arg = String(token_pos->begin, token_pos->end);
|
||||
auto tmp_arg = IParserKQLFunction::getExpression(token_pos);
|
||||
if (token_pos->type == TokenType::Comma)
|
||||
new_expr = new_expr + logic_op;
|
||||
else
|
||||
@ -37,33 +159,114 @@ String KQLOperators::genHasAnyAllOpExpr(std::vector<String> &tokens, IParser::Po
|
||||
++token_pos;
|
||||
if (token_pos->type == TokenType::ClosingRoundBracket)
|
||||
break;
|
||||
|
||||
}
|
||||
|
||||
tokens.pop_back();
|
||||
return new_expr;
|
||||
}
|
||||
|
||||
String KQLOperators::genInOpExpr(IParser::Pos &token_pos, String kql_op, String ch_op)
|
||||
String genEqOpExprCis(std::vector<String> & tokens, DB::IParser::Pos & token_pos, const DB::String & ch_op)
|
||||
{
|
||||
String new_expr;
|
||||
DB::String tmp_arg(token_pos->begin, token_pos->end);
|
||||
|
||||
ParserToken s_lparen(TokenType::OpeningRoundBracket);
|
||||
if (tokens.empty() || tmp_arg != "~")
|
||||
return tmp_arg;
|
||||
|
||||
ASTPtr select;
|
||||
Expected expected;
|
||||
DB::String new_expr;
|
||||
new_expr += "lower(" + tokens.back() + ")" + " ";
|
||||
new_expr += ch_op + " ";
|
||||
++token_pos;
|
||||
|
||||
if (token_pos->type == DB::TokenType::StringLiteral || token_pos->type == DB::TokenType::QuotedIdentifier)
|
||||
new_expr += "lower('" + DB::IParserKQLFunction::escapeSingleQuotes(String(token_pos->begin + 1, token_pos->end - 1)) + "')";
|
||||
else
|
||||
new_expr += "lower(" + DB::IParserKQLFunction::getExpression(token_pos) + ")";
|
||||
|
||||
tokens.pop_back();
|
||||
return new_expr;
|
||||
}
|
||||
|
||||
String genInOpExprCis(std::vector<String> & tokens, DB::IParser::Pos & token_pos, const DB::String & kql_op, const DB::String & ch_op)
|
||||
{
|
||||
DB::ParserKQLTableFunction kqlfun_p;
|
||||
DB::ParserToken s_lparen(DB::TokenType::OpeningRoundBracket);
|
||||
|
||||
DB::ASTPtr select;
|
||||
DB::Expected expected;
|
||||
DB::String new_expr;
|
||||
|
||||
++token_pos;
|
||||
if (!s_lparen.ignore(token_pos, expected))
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op);
|
||||
throw DB::Exception(DB::ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op);
|
||||
|
||||
if (tokens.empty())
|
||||
throw DB::Exception(DB::ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op);
|
||||
|
||||
new_expr = "lower(" + tokens.back() + ") ";
|
||||
tokens.pop_back();
|
||||
auto pos = token_pos;
|
||||
if (kqlfun_p.parse(pos, select, expected))
|
||||
{
|
||||
rebuildSubqueryForInOperator(select, true);
|
||||
new_expr += ch_op + " (" + serializeAST(*select) + ")";
|
||||
token_pos = pos;
|
||||
return new_expr;
|
||||
}
|
||||
|
||||
--token_pos;
|
||||
--token_pos;
|
||||
|
||||
new_expr += ch_op;
|
||||
while (!token_pos->isEnd() && token_pos->type != DB::TokenType::PipeMark && token_pos->type != DB::TokenType::Semicolon)
|
||||
{
|
||||
auto tmp_arg = DB::String(token_pos->begin, token_pos->end);
|
||||
if (token_pos->type != DB::TokenType::Comma && token_pos->type != DB::TokenType::ClosingRoundBracket
|
||||
&& token_pos->type != DB::TokenType::OpeningRoundBracket && token_pos->type != DB::TokenType::OpeningSquareBracket
|
||||
&& token_pos->type != DB::TokenType::ClosingSquareBracket && tmp_arg != "~" && tmp_arg != "dynamic")
|
||||
{
|
||||
if (token_pos->type == DB::TokenType::StringLiteral || token_pos->type == DB::TokenType::QuotedIdentifier)
|
||||
new_expr += "lower('" + DB::IParserKQLFunction::escapeSingleQuotes(String(token_pos->begin + 1, token_pos->end - 1)) + "')";
|
||||
else
|
||||
new_expr += "lower(" + tmp_arg + ")";
|
||||
}
|
||||
else if (tmp_arg != "~" && tmp_arg != "dynamic" && tmp_arg != "[" && tmp_arg != "]")
|
||||
new_expr += tmp_arg;
|
||||
|
||||
if (token_pos->type == DB::TokenType::ClosingRoundBracket)
|
||||
break;
|
||||
++token_pos;
|
||||
}
|
||||
return new_expr;
|
||||
}
|
||||
|
||||
std::string genInOpExpr(DB::IParser::Pos & token_pos, const std::string & kql_op, const std::string & ch_op)
|
||||
{
|
||||
DB::ParserKQLTableFunction kqlfun_p;
|
||||
DB::ParserToken s_lparen(DB::TokenType::OpeningRoundBracket);
|
||||
|
||||
DB::ASTPtr select;
|
||||
DB::Expected expected;
|
||||
|
||||
++token_pos;
|
||||
if (!s_lparen.ignore(token_pos, expected))
|
||||
throw DB::Exception(DB::ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op);
|
||||
|
||||
auto pos = token_pos;
|
||||
if (kqlfun_p.parse(pos, select, expected))
|
||||
{
|
||||
rebuildSubqueryForInOperator(select, false);
|
||||
auto new_expr = ch_op + " (" + serializeAST(*select) + ")";
|
||||
token_pos = pos;
|
||||
return new_expr;
|
||||
}
|
||||
|
||||
--token_pos;
|
||||
--token_pos;
|
||||
return ch_op;
|
||||
|
||||
}
|
||||
|
||||
String KQLOperators::genHaystackOpExpr(std::vector<String> &tokens,IParser::Pos &token_pos,String kql_op, String ch_op, WildcardsPos wildcards_pos, WildcardsPos space_pos)
|
||||
String KQLOperators::genHaystackOpExpr(
|
||||
std::vector<String> & tokens, IParser::Pos & token_pos, String kql_op, String ch_op, WildcardsPos wildcards_pos, WildcardsPos space_pos)
|
||||
{
|
||||
String new_expr, left_wildcards, right_wildcards, left_space, right_space;
|
||||
|
||||
@ -73,7 +276,7 @@ String KQLOperators::genHaystackOpExpr(std::vector<String> &tokens,IParser::Pos
|
||||
break;
|
||||
|
||||
case WildcardsPos::left:
|
||||
left_wildcards ="%";
|
||||
left_wildcards = "%";
|
||||
break;
|
||||
|
||||
case WildcardsPos::right:
|
||||
@ -81,7 +284,7 @@ String KQLOperators::genHaystackOpExpr(std::vector<String> &tokens,IParser::Pos
|
||||
break;
|
||||
|
||||
case WildcardsPos::both:
|
||||
left_wildcards ="%";
|
||||
left_wildcards = "%";
|
||||
right_wildcards = "%";
|
||||
break;
|
||||
}
|
||||
@ -92,7 +295,7 @@ String KQLOperators::genHaystackOpExpr(std::vector<String> &tokens,IParser::Pos
|
||||
break;
|
||||
|
||||
case WildcardsPos::left:
|
||||
left_space =" ";
|
||||
left_space = " ";
|
||||
break;
|
||||
|
||||
case WildcardsPos::right:
|
||||
@ -100,7 +303,7 @@ String KQLOperators::genHaystackOpExpr(std::vector<String> &tokens,IParser::Pos
|
||||
break;
|
||||
|
||||
case WildcardsPos::both:
|
||||
left_space =" ";
|
||||
left_space = " ";
|
||||
right_space = " ";
|
||||
break;
|
||||
}
|
||||
@ -108,11 +311,13 @@ String KQLOperators::genHaystackOpExpr(std::vector<String> &tokens,IParser::Pos
|
||||
++token_pos;
|
||||
|
||||
if (!tokens.empty() && ((token_pos)->type == TokenType::StringLiteral || token_pos->type == TokenType::QuotedIdentifier))
|
||||
new_expr = ch_op +"(" + tokens.back() +", '"+left_wildcards + left_space + String(token_pos->begin + 1,token_pos->end - 1) + right_space + right_wildcards + "')";
|
||||
new_expr = ch_op + "(" + tokens.back() + ", '" + left_wildcards + left_space + String(token_pos->begin + 1, token_pos->end - 1)
|
||||
+ right_space + right_wildcards + "')";
|
||||
else if (!tokens.empty() && ((token_pos)->type == TokenType::BareWord))
|
||||
{
|
||||
auto tmp_arg = String(token_pos->begin, token_pos->end);
|
||||
new_expr = ch_op +"(" + tokens.back() +", concat('" + left_wildcards + left_space + "', " + tmp_arg +", '"+ right_space + right_wildcards + "'))";
|
||||
auto tmp_arg = IParserKQLFunction::getExpression(token_pos);
|
||||
new_expr = ch_op + "(" + tokens.back() + ", concat('" + left_wildcards + left_space + "', " + tmp_arg + ", '" + right_space
|
||||
+ right_wildcards + "'))";
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op);
|
||||
@ -120,7 +325,7 @@ String KQLOperators::genHaystackOpExpr(std::vector<String> &tokens,IParser::Pos
|
||||
return new_expr;
|
||||
}
|
||||
|
||||
bool KQLOperators::convert(std::vector<String> &tokens,IParser::Pos &pos)
|
||||
bool KQLOperators::convert(std::vector<String> & tokens, IParser::Pos & pos)
|
||||
{
|
||||
auto begin = pos;
|
||||
|
||||
@ -128,7 +333,7 @@ bool KQLOperators::convert(std::vector<String> &tokens,IParser::Pos &pos)
|
||||
{
|
||||
KQLOperatorValue op_value = KQLOperatorValue::none;
|
||||
|
||||
auto token = String(pos->begin,pos->end);
|
||||
auto token = String(pos->begin, pos->end);
|
||||
|
||||
String op = token;
|
||||
if (token == "!")
|
||||
@ -136,15 +341,15 @@ bool KQLOperators::convert(std::vector<String> &tokens,IParser::Pos &pos)
|
||||
++pos;
|
||||
if (pos->isEnd() || pos->type == TokenType::PipeMark || pos->type == TokenType::Semicolon)
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Invalid negative operator");
|
||||
op ="!"+String(pos->begin,pos->end);
|
||||
op = "!" + String(pos->begin, pos->end);
|
||||
}
|
||||
else if (token == "matches")
|
||||
{
|
||||
++pos;
|
||||
if (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
if (String(pos->begin,pos->end) == "regex")
|
||||
op +=" regex";
|
||||
if (String(pos->begin, pos->end) == "regex")
|
||||
op += " regex";
|
||||
else
|
||||
--pos;
|
||||
}
|
||||
@ -157,8 +362,8 @@ bool KQLOperators::convert(std::vector<String> &tokens,IParser::Pos &pos)
|
||||
++pos;
|
||||
if (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
if (String(pos->begin,pos->end) == "~")
|
||||
op +="~";
|
||||
if (String(pos->begin, pos->end) == "~")
|
||||
op += "~";
|
||||
else
|
||||
--pos;
|
||||
}
|
||||
@ -171,7 +376,7 @@ bool KQLOperators::convert(std::vector<String> &tokens,IParser::Pos &pos)
|
||||
return false;
|
||||
}
|
||||
|
||||
op_value = KQLOperator[op];
|
||||
op_value = KQLOperator.at(op);
|
||||
|
||||
String new_expr;
|
||||
|
||||
@ -179,172 +384,179 @@ bool KQLOperators::convert(std::vector<String> &tokens,IParser::Pos &pos)
|
||||
tokens.push_back(op);
|
||||
else
|
||||
{
|
||||
if (tokens.empty())
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", op);
|
||||
|
||||
auto last_op = tokens.back();
|
||||
auto last_pos = pos;
|
||||
|
||||
switch (op_value)
|
||||
{
|
||||
case KQLOperatorValue::contains:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::both);
|
||||
break;
|
||||
case KQLOperatorValue::contains:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::both);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_contains:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::both);
|
||||
break;
|
||||
case KQLOperatorValue::not_contains:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::both);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::contains_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "like", WildcardsPos::both);
|
||||
break;
|
||||
case KQLOperatorValue::contains_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "like", WildcardsPos::both);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_contains_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not like", WildcardsPos::both);
|
||||
break;
|
||||
case KQLOperatorValue::not_contains_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not like", WildcardsPos::both);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::endswith:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::left);
|
||||
break;
|
||||
case KQLOperatorValue::endswith:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::left);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_endswith:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::left);
|
||||
break;
|
||||
case KQLOperatorValue::not_endswith:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::left);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::endswith_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "endsWith", WildcardsPos::none);
|
||||
break;
|
||||
case KQLOperatorValue::endswith_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "endsWith", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_endswith_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not endsWith", WildcardsPos::none);
|
||||
break;
|
||||
case KQLOperatorValue::not_endswith_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not endsWith", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::equal:
|
||||
break;
|
||||
case KQLOperatorValue::equal:
|
||||
new_expr = genEqOpExprCis(tokens, pos, "==");
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_equal:
|
||||
break;
|
||||
case KQLOperatorValue::not_equal:
|
||||
new_expr = genEqOpExprCis(tokens, pos, "!=");
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::equal_cs:
|
||||
new_expr = "==";
|
||||
break;
|
||||
case KQLOperatorValue::equal_cs:
|
||||
new_expr = "==";
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_equal_cs:
|
||||
new_expr = "!=";
|
||||
break;
|
||||
case KQLOperatorValue::has:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "hasTokenCaseInsensitive", WildcardsPos::none);
|
||||
break;
|
||||
case KQLOperatorValue::not_equal_cs:
|
||||
new_expr = "!=";
|
||||
break;
|
||||
case KQLOperatorValue::has:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "hasTokenCaseInsensitive", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_has:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not hasTokenCaseInsensitive", WildcardsPos::none);
|
||||
break;
|
||||
case KQLOperatorValue::not_has:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not hasTokenCaseInsensitive", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::has_all:
|
||||
new_expr = genHasAnyAllOpExpr(tokens, pos, "has_all", "hasTokenCaseInsensitive");
|
||||
break;
|
||||
case KQLOperatorValue::has_all:
|
||||
new_expr = genHasAnyAllOpExpr(tokens, pos, "has_all", "hasTokenCaseInsensitive");
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::has_any:
|
||||
new_expr = genHasAnyAllOpExpr(tokens, pos, "has_any", "hasTokenCaseInsensitive");
|
||||
break;
|
||||
case KQLOperatorValue::has_any:
|
||||
new_expr = genHasAnyAllOpExpr(tokens, pos, "has_any", "hasTokenCaseInsensitive");
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::has_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "hasToken", WildcardsPos::none);
|
||||
break;
|
||||
case KQLOperatorValue::has_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "hasToken", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_has_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not hasToken", WildcardsPos::none);
|
||||
break;
|
||||
case KQLOperatorValue::not_has_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not hasToken", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::hasprefix:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::right);
|
||||
new_expr += " or ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "ilike", WildcardsPos::both, WildcardsPos::left);
|
||||
break;
|
||||
case KQLOperatorValue::hasprefix:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::right);
|
||||
new_expr += " or ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "ilike", WildcardsPos::both, WildcardsPos::left);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_hasprefix:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::right);
|
||||
new_expr += " and ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "not ilike", WildcardsPos::both, WildcardsPos::left);
|
||||
break;
|
||||
case KQLOperatorValue::not_hasprefix:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::right);
|
||||
new_expr += " and ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "not ilike", WildcardsPos::both, WildcardsPos::left);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::hasprefix_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "startsWith", WildcardsPos::none);
|
||||
new_expr += " or ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "like", WildcardsPos::both, WildcardsPos::left);
|
||||
break;
|
||||
case KQLOperatorValue::hasprefix_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "startsWith", WildcardsPos::none);
|
||||
new_expr += " or ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "like", WildcardsPos::both, WildcardsPos::left);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_hasprefix_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not startsWith", WildcardsPos::none);
|
||||
new_expr += " and ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "not like", WildcardsPos::both, WildcardsPos::left);
|
||||
break;
|
||||
case KQLOperatorValue::not_hasprefix_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not startsWith", WildcardsPos::none);
|
||||
new_expr += " and ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "not like", WildcardsPos::both, WildcardsPos::left);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::hassuffix:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::left);
|
||||
new_expr += " or ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "ilike", WildcardsPos::both, WildcardsPos::right);
|
||||
break;
|
||||
case KQLOperatorValue::hassuffix:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::left);
|
||||
new_expr += " or ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "ilike", WildcardsPos::both, WildcardsPos::right);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_hassuffix:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::left);
|
||||
new_expr += " and ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "not ilike", WildcardsPos::both, WildcardsPos::right);
|
||||
break;
|
||||
case KQLOperatorValue::not_hassuffix:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::left);
|
||||
new_expr += " and ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "not ilike", WildcardsPos::both, WildcardsPos::right);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::hassuffix_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "endsWith", WildcardsPos::none);
|
||||
new_expr += " or ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "like", WildcardsPos::both, WildcardsPos::right);
|
||||
break;
|
||||
case KQLOperatorValue::hassuffix_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "endsWith", WildcardsPos::none);
|
||||
new_expr += " or ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "like", WildcardsPos::both, WildcardsPos::right);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_hassuffix_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not endsWith", WildcardsPos::none);
|
||||
new_expr += " and ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "not like", WildcardsPos::both, WildcardsPos::right);
|
||||
break;
|
||||
case KQLOperatorValue::not_hassuffix_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not endsWith", WildcardsPos::none);
|
||||
new_expr += " and ";
|
||||
tokens.push_back(last_op);
|
||||
new_expr += genHaystackOpExpr(tokens, last_pos, op, "not like", WildcardsPos::both, WildcardsPos::right);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::in_cs:
|
||||
new_expr = genInOpExpr(pos,op,"in");
|
||||
break;
|
||||
case KQLOperatorValue::in_cs:
|
||||
new_expr = genInOpExpr(pos, op, "in");
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_in_cs:
|
||||
new_expr = genInOpExpr(pos,op,"not in");
|
||||
break;
|
||||
case KQLOperatorValue::not_in_cs:
|
||||
new_expr = genInOpExpr(pos, op, "not in");
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::in:
|
||||
break;
|
||||
case KQLOperatorValue::in:
|
||||
new_expr = genInOpExprCis(tokens, pos, op, "in");
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_in:
|
||||
break;
|
||||
case KQLOperatorValue::not_in:
|
||||
new_expr = genInOpExprCis(tokens, pos, op, "not in");
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::matches_regex:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "match", WildcardsPos::none);
|
||||
break;
|
||||
case KQLOperatorValue::matches_regex:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "match", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::startswith:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::right);
|
||||
break;
|
||||
case KQLOperatorValue::startswith:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "ilike", WildcardsPos::right);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_startswith:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::right);
|
||||
break;
|
||||
case KQLOperatorValue::not_startswith:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not ilike", WildcardsPos::right);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::startswith_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "startsWith", WildcardsPos::none);
|
||||
break;
|
||||
case KQLOperatorValue::startswith_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "startsWith", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
case KQLOperatorValue::not_startswith_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not startsWith", WildcardsPos::none);
|
||||
break;
|
||||
case KQLOperatorValue::not_startswith_cs:
|
||||
new_expr = genHaystackOpExpr(tokens, pos, op, "not startsWith", WildcardsPos::none);
|
||||
break;
|
||||
|
||||
default:
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
tokens.push_back(new_expr);
|
||||
@ -356,4 +568,3 @@ bool KQLOperators::convert(std::vector<String> &tokens,IParser::Pos &pos)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
@ -1,18 +1,18 @@
|
||||
#pragma once
|
||||
|
||||
#include <unordered_map>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <unordered_map>
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class KQLOperators
|
||||
{
|
||||
public:
|
||||
bool convert(std::vector<String> &tokens,IParser::Pos &pos);
|
||||
protected:
|
||||
static bool convert(std::vector<String> & tokens, IParser::Pos & pos);
|
||||
|
||||
enum class WildcardsPos:uint8_t
|
||||
protected:
|
||||
enum class WildcardsPos : uint8_t
|
||||
{
|
||||
none,
|
||||
left,
|
||||
@ -20,87 +20,14 @@ protected:
|
||||
both
|
||||
};
|
||||
|
||||
enum class KQLOperatorValue : uint16_t
|
||||
{
|
||||
none,
|
||||
contains,
|
||||
not_contains,
|
||||
contains_cs,
|
||||
not_contains_cs,
|
||||
endswith,
|
||||
not_endswith,
|
||||
endswith_cs,
|
||||
not_endswith_cs,
|
||||
equal, /// =~
|
||||
not_equal, /// !~
|
||||
equal_cs, /// =
|
||||
not_equal_cs, /// !=
|
||||
has,
|
||||
not_has,
|
||||
has_all,
|
||||
has_any,
|
||||
has_cs,
|
||||
not_has_cs,
|
||||
hasprefix,
|
||||
not_hasprefix,
|
||||
hasprefix_cs,
|
||||
not_hasprefix_cs,
|
||||
hassuffix,
|
||||
not_hassuffix,
|
||||
hassuffix_cs,
|
||||
not_hassuffix_cs,
|
||||
in_cs, /// in
|
||||
not_in_cs, /// !in
|
||||
in, /// in~
|
||||
not_in, /// !in~
|
||||
matches_regex,
|
||||
startswith,
|
||||
not_startswith,
|
||||
startswith_cs,
|
||||
not_startswith_cs,
|
||||
};
|
||||
|
||||
std::unordered_map <String,KQLOperatorValue> KQLOperator =
|
||||
{
|
||||
{"contains" , KQLOperatorValue::contains},
|
||||
{"!contains" , KQLOperatorValue::not_contains},
|
||||
{"contains_cs" , KQLOperatorValue::contains_cs},
|
||||
{"!contains_cs" , KQLOperatorValue::not_contains_cs},
|
||||
{"endswith" , KQLOperatorValue::endswith},
|
||||
{"!endswith" , KQLOperatorValue::not_endswith},
|
||||
{"endswith_cs" , KQLOperatorValue::endswith_cs},
|
||||
{"!endswith_cs" , KQLOperatorValue::not_endswith_cs},
|
||||
{"=~" , KQLOperatorValue::equal},
|
||||
{"!~" , KQLOperatorValue::not_equal},
|
||||
{"==" , KQLOperatorValue::equal_cs},
|
||||
{"!=" , KQLOperatorValue::not_equal_cs},
|
||||
{"has" , KQLOperatorValue::has},
|
||||
{"!has" , KQLOperatorValue::not_has},
|
||||
{"has_all" , KQLOperatorValue::has_all},
|
||||
{"has_any" , KQLOperatorValue::has_any},
|
||||
{"has_cs" , KQLOperatorValue::has_cs},
|
||||
{"!has_cs" , KQLOperatorValue::not_has_cs},
|
||||
{"hasprefix" , KQLOperatorValue::hasprefix},
|
||||
{"!hasprefix" , KQLOperatorValue::not_hasprefix},
|
||||
{"hasprefix_cs" , KQLOperatorValue::hasprefix_cs},
|
||||
{"!hasprefix_cs" , KQLOperatorValue::not_hasprefix_cs},
|
||||
{"hassuffix" , KQLOperatorValue::hassuffix},
|
||||
{"!hassuffix" , KQLOperatorValue::not_hassuffix},
|
||||
{"hassuffix_cs" , KQLOperatorValue::hassuffix_cs},
|
||||
{"!hassuffix_cs" , KQLOperatorValue::not_hassuffix_cs},
|
||||
{"in" , KQLOperatorValue::in_cs},
|
||||
{"!in" , KQLOperatorValue::not_in_cs},
|
||||
{"in~" , KQLOperatorValue::in},
|
||||
{"!in~" , KQLOperatorValue::not_in},
|
||||
{"matches regex" , KQLOperatorValue::matches_regex},
|
||||
{"startswith" , KQLOperatorValue::startswith},
|
||||
{"!startswith" , KQLOperatorValue::not_startswith},
|
||||
{"startswith_cs" , KQLOperatorValue::startswith_cs},
|
||||
{"!startswith_cs" , KQLOperatorValue::not_startswith_cs},
|
||||
};
|
||||
static String genHaystackOpExpr(std::vector<String> &tokens,IParser::Pos &token_pos,String kql_op, String ch_op, WildcardsPos wildcards_pos, WildcardsPos space_pos = WildcardsPos::none);
|
||||
static String genInOpExpr(IParser::Pos &token_pos,String kql_op, String ch_op);
|
||||
static String genHasAnyAllOpExpr(std::vector<String> &tokens,IParser::Pos &token_pos,String kql_op, String ch_op);
|
||||
static String genHaystackOpExpr(
|
||||
std::vector<String> & tokens,
|
||||
IParser::Pos & token_pos,
|
||||
String kql_op,
|
||||
String ch_op,
|
||||
WildcardsPos wildcards_pos,
|
||||
WildcardsPos space_pos = WildcardsPos::none);
|
||||
static String genHasAnyAllOpExpr(std::vector<String> & tokens, IParser::Pos & token_pos, String kql_op, String ch_op);
|
||||
};
|
||||
|
||||
}
|
||||
|
22
src/Parsers/Kusto/ParserKQLPrint.cpp
Normal file
22
src/Parsers/Kusto/ParserKQLPrint.cpp
Normal file
@ -0,0 +1,22 @@
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/Kusto/ParserKQLPrint.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLPrint::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr select_expression_list;
|
||||
const String expr = getExprFromToken(pos);
|
||||
|
||||
Tokens tokens(expr.c_str(), expr.c_str() + expr.size());
|
||||
IParser::Pos new_pos(tokens, pos.max_depth);
|
||||
|
||||
if (!ParserNotEmptyExpressionList(true).parse(new_pos, select_expression_list, expected))
|
||||
return false;
|
||||
node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_expression_list));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
16
src/Parsers/Kusto/ParserKQLPrint.h
Normal file
16
src/Parsers/Kusto/ParserKQLPrint.h
Normal file
@ -0,0 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserKQLPrint : public ParserKQLBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "KQL project"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
@ -1,20 +1,20 @@
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLProject.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLProject :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
bool ParserKQLProject ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr select_expression_list;
|
||||
String expr;
|
||||
|
||||
expr = getExprFromToken(pos);
|
||||
|
||||
Tokens tokens(expr.c_str(), expr.c_str()+expr.size());
|
||||
Tokens tokens(expr.c_str(), expr.c_str() + expr.size());
|
||||
IParser::Pos new_pos(tokens, pos.max_depth);
|
||||
|
||||
if (!ParserNotEmptyExpressionList(true).parse(new_pos, select_expression_list, expected))
|
||||
if (!ParserNotEmptyExpressionList(false).parse(new_pos, select_expression_list, expected))
|
||||
return false;
|
||||
|
||||
node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_expression_list));
|
||||
|
@ -1,24 +1,124 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLTable.h>
|
||||
#include <Parsers/Kusto/ParserKQLProject.h>
|
||||
#include <Parsers/Kusto/ParserKQLFilter.h>
|
||||
#include <Parsers/Kusto/ParserKQLSort.h>
|
||||
#include <Parsers/Kusto/ParserKQLSummarize.h>
|
||||
#include <Parsers/Kusto/ParserKQLLimit.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <format>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h>
|
||||
#include <Parsers/Kusto/ParserKQLDistinct.h>
|
||||
#include <Parsers/Kusto/ParserKQLExtend.h>
|
||||
#include <Parsers/Kusto/ParserKQLFilter.h>
|
||||
#include <Parsers/Kusto/ParserKQLLimit.h>
|
||||
#include <Parsers/Kusto/ParserKQLMVExpand.h>
|
||||
#include <Parsers/Kusto/ParserKQLMakeSeries.h>
|
||||
#include <Parsers/Kusto/ParserKQLOperators.h>
|
||||
#include <Parsers/Kusto/ParserKQLPrint.h>
|
||||
#include <Parsers/Kusto/ParserKQLProject.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLSort.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/Kusto/ParserKQLSummarize.h>
|
||||
#include <Parsers/Kusto/ParserKQLTable.h>
|
||||
#include <Parsers/ParserSelectWithUnionQuery.h>
|
||||
#include <Parsers/ParserTablesInSelectQuery.h>
|
||||
|
||||
#include <format>
|
||||
namespace DB
|
||||
{
|
||||
|
||||
String ParserKQLBase :: getExprFromToken(const String & text, const uint32_t & max_depth)
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
bool ParserKQLBase::parseByString(const String expr, ASTPtr & node, const uint32_t max_depth)
|
||||
{
|
||||
Expected expected;
|
||||
|
||||
Tokens tokens(expr.c_str(), expr.c_str() + expr.size());
|
||||
IParser::Pos pos(tokens, max_depth);
|
||||
return parse(pos, node, expected);
|
||||
}
|
||||
|
||||
bool ParserKQLBase::parseSQLQueryByString(ParserPtr && parser, String & query, ASTPtr & select_node, int32_t max_depth)
|
||||
{
|
||||
Expected expected;
|
||||
Tokens token_subquery(query.c_str(), query.c_str() + query.size());
|
||||
IParser::Pos pos_subquery(token_subquery, max_depth);
|
||||
if (!parser->parse(pos_subquery, select_node, expected))
|
||||
return false;
|
||||
return true;
|
||||
};
|
||||
|
||||
bool ParserKQLBase::setSubQuerySource(ASTPtr & select_query, ASTPtr & source, bool dest_is_subquery, bool src_is_subquery)
|
||||
{
|
||||
ASTPtr table_expr;
|
||||
if (!dest_is_subquery)
|
||||
{
|
||||
if (!select_query || !select_query->as<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->as<ASTTablesInSelectQueryElement>()->table_expression
|
||||
= source->as<ASTSelectQuery>()->tables()->children[0]->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[0]
|
||||
->as<ASTTablesInSelectQueryElement>()
|
||||
->table_expression->as<ASTTableExpression>()
|
||||
->subquery->children.empty()
|
||||
|| select_query->as<ASTTablesInSelectQuery>()
|
||||
->children[0]
|
||||
->as<ASTTablesInSelectQueryElement>()
|
||||
->table_expression->as<ASTTableExpression>()
|
||||
->subquery->children[0]
|
||||
->as<ASTSelectWithUnionQuery>()
|
||||
->list_of_selects->children.empty()
|
||||
|| select_query->as<ASTTablesInSelectQuery>()
|
||||
->children[0]
|
||||
->as<ASTTablesInSelectQueryElement>()
|
||||
->table_expression->as<ASTTableExpression>()
|
||||
->subquery->children[0]
|
||||
->as<ASTSelectWithUnionQuery>()
|
||||
->list_of_selects->children[0]
|
||||
->as<ASTSelectQuery>()
|
||||
->tables()
|
||||
->as<ASTTablesInSelectQuery>()
|
||||
->children.empty())
|
||||
return false;
|
||||
|
||||
table_expr = select_query->as<ASTTablesInSelectQuery>()
|
||||
->children[0]
|
||||
->as<ASTTablesInSelectQueryElement>()
|
||||
->table_expression->as<ASTTableExpression>()
|
||||
->subquery->children[0]
|
||||
->as<ASTSelectWithUnionQuery>()
|
||||
->list_of_selects->children[0]
|
||||
->as<ASTSelectQuery>()
|
||||
->tables()
|
||||
->as<ASTTablesInSelectQuery>()
|
||||
->children[0];
|
||||
|
||||
if (!src_is_subquery)
|
||||
{
|
||||
table_expr->as<ASTTablesInSelectQueryElement>()->table_expression
|
||||
= source->as<ASTSelectQuery>()->tables()->children[0]->as<ASTTablesInSelectQueryElement>()->table_expression;
|
||||
}
|
||||
else
|
||||
{
|
||||
table_expr->as<ASTTablesInSelectQueryElement>()->table_expression
|
||||
= source->children[0]->as<ASTTablesInSelectQueryElement>()->table_expression;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
String ParserKQLBase::getExprFromToken(const String & text, const uint32_t max_depth)
|
||||
{
|
||||
Tokens tokens(text.c_str(), text.c_str() + text.size());
|
||||
IParser::Pos pos(tokens, max_depth);
|
||||
@ -26,72 +126,179 @@ String ParserKQLBase :: getExprFromToken(const String & text, const uint32_t & m
|
||||
return getExprFromToken(pos);
|
||||
}
|
||||
|
||||
String ParserKQLBase :: getExprFromPipe(Pos & pos)
|
||||
String ParserKQLBase::getExprFromPipe(Pos & pos)
|
||||
{
|
||||
uint16_t bracket_count = 0;
|
||||
auto begin = pos;
|
||||
BracketCount bracket_count;
|
||||
auto end = pos;
|
||||
while (!end->isEnd() && end->type != TokenType::Semicolon)
|
||||
{
|
||||
if (end->type == TokenType::OpeningRoundBracket)
|
||||
++bracket_count;
|
||||
|
||||
if (end->type == TokenType::OpeningRoundBracket)
|
||||
--bracket_count;
|
||||
|
||||
if (end->type == TokenType::PipeMark && bracket_count == 0)
|
||||
bracket_count.count(end);
|
||||
if (end->type == TokenType::PipeMark && bracket_count.isZero())
|
||||
break;
|
||||
|
||||
++end;
|
||||
}
|
||||
--end;
|
||||
return String(begin->begin, end->end);
|
||||
if (end != pos)
|
||||
--end;
|
||||
return (pos <= end) ? String(pos->begin, end->end) : "";
|
||||
}
|
||||
|
||||
String ParserKQLBase :: getExprFromToken(Pos & pos)
|
||||
String ParserKQLBase::getExprFromToken(Pos & pos)
|
||||
{
|
||||
String res;
|
||||
std::vector<String> tokens;
|
||||
String alias;
|
||||
std::vector<Pos> comma_pos;
|
||||
comma_pos.push_back(pos);
|
||||
|
||||
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
size_t paren_count = 0;
|
||||
while (!pos->isEnd() && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
String token = String(pos->begin,pos->end);
|
||||
if (pos->type == TokenType::PipeMark && paren_count == 0)
|
||||
break;
|
||||
|
||||
if (token == "=")
|
||||
if (pos->type == TokenType::OpeningRoundBracket)
|
||||
++paren_count;
|
||||
if (pos->type == TokenType::ClosingRoundBracket)
|
||||
--paren_count;
|
||||
|
||||
if (pos->type == TokenType::Comma && paren_count == 0)
|
||||
{
|
||||
++pos;
|
||||
if (String(pos->begin,pos->end) != "~")
|
||||
{
|
||||
alias = tokens.back();
|
||||
tokens.pop_back();
|
||||
}
|
||||
comma_pos.push_back(pos);
|
||||
--pos;
|
||||
}
|
||||
else if (!KQLOperators().convert(tokens,pos))
|
||||
{
|
||||
tokens.push_back(token);
|
||||
}
|
||||
|
||||
if (pos->type == TokenType::Comma && !alias.empty())
|
||||
{
|
||||
tokens.pop_back();
|
||||
tokens.push_back("AS");
|
||||
tokens.push_back(alias);
|
||||
tokens.push_back(",");
|
||||
alias.clear();
|
||||
}
|
||||
++pos;
|
||||
}
|
||||
|
||||
if (!alias.empty())
|
||||
std::vector<String> columns;
|
||||
auto set_columns = [&](Pos & start_pos, Pos & end_pos)
|
||||
{
|
||||
tokens.push_back("AS");
|
||||
tokens.push_back(alias);
|
||||
bool has_alias = false;
|
||||
auto equal_pos = start_pos;
|
||||
auto columms_start_pos = start_pos;
|
||||
auto it_pos = start_pos;
|
||||
if (String(it_pos->begin, it_pos->end) == "=")
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Invalid equal symbol (=)");
|
||||
|
||||
BracketCount bracket_count;
|
||||
while (it_pos < end_pos)
|
||||
{
|
||||
bracket_count.count(it_pos);
|
||||
if (String(it_pos->begin, it_pos->end) == "=")
|
||||
{
|
||||
++it_pos;
|
||||
if (String(it_pos->begin, it_pos->end) != "~" && bracket_count.isZero())
|
||||
{
|
||||
if (has_alias)
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Invalid equal symbol (=)");
|
||||
has_alias = true;
|
||||
}
|
||||
|
||||
--it_pos;
|
||||
if (equal_pos == start_pos)
|
||||
equal_pos = it_pos;
|
||||
}
|
||||
++it_pos;
|
||||
}
|
||||
|
||||
if (has_alias)
|
||||
{
|
||||
columms_start_pos = equal_pos;
|
||||
++columms_start_pos;
|
||||
}
|
||||
String column_str;
|
||||
String function_name;
|
||||
std::vector<String> tokens;
|
||||
|
||||
while (columms_start_pos < end_pos)
|
||||
{
|
||||
if (!KQLOperators::convert(tokens, columms_start_pos))
|
||||
{
|
||||
if (columms_start_pos->type == TokenType::BareWord && function_name.empty())
|
||||
function_name = String(columms_start_pos->begin, columms_start_pos->end);
|
||||
|
||||
auto expr = IParserKQLFunction::getExpression(columms_start_pos);
|
||||
tokens.push_back(expr);
|
||||
}
|
||||
++columms_start_pos;
|
||||
}
|
||||
|
||||
for (const auto & token : tokens)
|
||||
column_str = column_str.empty() ? token : column_str + " " + token;
|
||||
|
||||
if (has_alias)
|
||||
{
|
||||
--equal_pos;
|
||||
if (start_pos == equal_pos)
|
||||
{
|
||||
String new_column_str;
|
||||
if (start_pos->type != TokenType::BareWord)
|
||||
throw Exception(
|
||||
ErrorCodes::SYNTAX_ERROR, "{} is not a valid alias", std::string_view(start_pos->begin, start_pos->end));
|
||||
|
||||
if (function_name == "array_sort_asc" || function_name == "array_sort_desc")
|
||||
new_column_str = std::format("{0}[1] AS {1}", column_str, String(start_pos->begin, start_pos->end));
|
||||
else
|
||||
new_column_str = std::format("{0} AS {1}", column_str, String(start_pos->begin, start_pos->end));
|
||||
|
||||
columns.push_back(new_column_str);
|
||||
}
|
||||
else
|
||||
{
|
||||
String whole_alias(start_pos->begin, equal_pos->end);
|
||||
|
||||
if (function_name != "array_sort_asc" && function_name != "array_sort_desc")
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "{} is not a valid alias", whole_alias);
|
||||
|
||||
if (start_pos->type != TokenType::OpeningRoundBracket && equal_pos->type != TokenType::ClosingRoundBracket)
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "{} is not a valid alias for {}", whole_alias, function_name);
|
||||
|
||||
String alias_inside;
|
||||
bool comma_meet = false;
|
||||
size_t index = 1;
|
||||
++start_pos;
|
||||
while (start_pos < equal_pos)
|
||||
{
|
||||
if (start_pos->type == TokenType::Comma)
|
||||
{
|
||||
alias_inside.clear();
|
||||
if (comma_meet)
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "{} has invalid alias for {}", whole_alias, function_name);
|
||||
comma_meet = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!alias_inside.empty() || start_pos->type != TokenType::BareWord)
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "{} has invalid alias for {}", whole_alias, function_name);
|
||||
|
||||
alias_inside = String(start_pos->begin, start_pos->end);
|
||||
auto new_column_str = std::format("{0}[{1}] AS {2}", column_str, index, alias_inside);
|
||||
columns.push_back(new_column_str);
|
||||
comma_meet = false;
|
||||
++index;
|
||||
}
|
||||
++start_pos;
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
columns.push_back(column_str);
|
||||
};
|
||||
|
||||
size_t cloumn_size = comma_pos.size();
|
||||
for (size_t i = 0; i < cloumn_size; ++i)
|
||||
{
|
||||
if (i == cloumn_size - 1)
|
||||
set_columns(comma_pos[i], pos);
|
||||
else
|
||||
{
|
||||
auto end_pos = comma_pos[i + 1];
|
||||
--end_pos;
|
||||
set_columns(comma_pos[i], end_pos);
|
||||
}
|
||||
}
|
||||
|
||||
for (auto const &token : tokens)
|
||||
res = res.empty()? token : res +" " + token;
|
||||
String res;
|
||||
for (const auto & token : columns)
|
||||
res = res.empty() ? token : res + "," + token;
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -103,12 +310,22 @@ std::unique_ptr<IParserBase> ParserKQLQuery::getOperator(String & op_name)
|
||||
return std::make_unique<ParserKQLLimit>();
|
||||
else if (op_name == "project")
|
||||
return std::make_unique<ParserKQLProject>();
|
||||
else if (op_name == "distinct")
|
||||
return std::make_unique<ParserKQLDistinct>();
|
||||
else if (op_name == "extend")
|
||||
return std::make_unique<ParserKQLExtend>();
|
||||
else if (op_name == "sort by" || op_name == "order by")
|
||||
return std::make_unique<ParserKQLSort>();
|
||||
else if (op_name == "summarize")
|
||||
return std::make_unique<ParserKQLSummarize>();
|
||||
else if (op_name == "table")
|
||||
return std::make_unique<ParserKQLTable>();
|
||||
else if (op_name == "make-series")
|
||||
return std::make_unique<ParserKQLMakeSeries>();
|
||||
else if (op_name == "mv-expand")
|
||||
return std::make_unique<ParserKQLMVExpand>();
|
||||
else if (op_name == "print")
|
||||
return std::make_unique<ParserKQLPrint>();
|
||||
else
|
||||
return nullptr;
|
||||
}
|
||||
@ -127,49 +344,78 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
node = select_query;
|
||||
ASTPtr tables;
|
||||
|
||||
std::unordered_map<std::string, KQLOperatorDataFlowState> kql_parser =
|
||||
{
|
||||
{ "filter", {"filter", false, false, 3}},
|
||||
{ "where", {"filter", false, false, 3}},
|
||||
{ "limit", {"limit", false, true, 3}},
|
||||
{ "take", {"limit", false, true, 3}},
|
||||
{ "project", {"project", false, false, 3}},
|
||||
{ "sort by", {"order by", false, false, 4}},
|
||||
{ "order by", {"order by", false, false, 4}},
|
||||
{ "table", {"table", false, false, 3}},
|
||||
{ "summarize", {"summarize", true, true, 3}}
|
||||
};
|
||||
std::unordered_map<std::string, KQLOperatorDataFlowState> kql_parser
|
||||
= {{"filter", {"filter", false, false, 3}},
|
||||
{"where", {"filter", false, false, 3}},
|
||||
{"limit", {"limit", false, true, 3}},
|
||||
{"take", {"limit", false, true, 3}},
|
||||
{"project", {"project", false, false, 3}},
|
||||
{"distinct", {"distinct", false, true, 3}},
|
||||
{"extend", {"extend", true, true, 3}},
|
||||
{"sort by", {"order by", false, false, 4}},
|
||||
{"order by", {"order by", false, false, 4}},
|
||||
{"table", {"table", false, false, 3}},
|
||||
{"print", {"print", false, true, 3}},
|
||||
{"summarize", {"summarize", true, true, 3}},
|
||||
{"make-series", {"make-series", true, true, 5}},
|
||||
{"mv-expand", {"mv-expand", true, true, 5}}};
|
||||
|
||||
std::vector<std::pair<String, Pos>> operation_pos;
|
||||
|
||||
String table_name(pos->begin, pos->end);
|
||||
|
||||
operation_pos.push_back(std::make_pair("table", pos));
|
||||
if (table_name == "print")
|
||||
operation_pos.emplace_back(table_name, pos);
|
||||
else
|
||||
operation_pos.emplace_back("table", pos);
|
||||
|
||||
++pos;
|
||||
|
||||
uint16_t bracket_count = 0;
|
||||
|
||||
while (!pos->isEnd() && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
if (pos->type == TokenType::OpeningRoundBracket)
|
||||
++bracket_count;
|
||||
if (pos->type == TokenType::OpeningRoundBracket)
|
||||
if (pos->type == TokenType::ClosingRoundBracket)
|
||||
--bracket_count;
|
||||
|
||||
if (pos->type == TokenType::PipeMark && bracket_count == 0)
|
||||
{
|
||||
++pos;
|
||||
String kql_operator(pos->begin, pos->end);
|
||||
if (kql_operator == "order" || kql_operator == "sort")
|
||||
|
||||
auto validate_kql_operator = [&]
|
||||
{
|
||||
++pos;
|
||||
ParserKeyword s_by("by");
|
||||
if (s_by.ignore(pos,expected))
|
||||
if (kql_operator == "order" || kql_operator == "sort")
|
||||
{
|
||||
kql_operator = "order by";
|
||||
--pos;
|
||||
++pos;
|
||||
ParserKeyword s_by("by");
|
||||
if (s_by.ignore(pos, expected))
|
||||
{
|
||||
kql_operator = "order by";
|
||||
--pos;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (pos->type != TokenType::BareWord || kql_parser.find(kql_operator) == kql_parser.end())
|
||||
else
|
||||
{
|
||||
auto op_pos_begin = pos;
|
||||
++pos;
|
||||
ParserToken s_dash(TokenType::Minus);
|
||||
if (s_dash.ignore(pos, expected))
|
||||
{
|
||||
String tmp_op(op_pos_begin->begin, pos->end);
|
||||
kql_operator = tmp_op;
|
||||
}
|
||||
else
|
||||
--pos;
|
||||
}
|
||||
if (kql_parser.find(kql_operator) == kql_parser.end())
|
||||
return false;
|
||||
return true;
|
||||
};
|
||||
|
||||
if (!validate_kql_operator())
|
||||
return false;
|
||||
++pos;
|
||||
operation_pos.push_back(std::make_pair(kql_operator, pos));
|
||||
@ -190,16 +436,27 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|
||||
if (operation_pos.size() == 1)
|
||||
{
|
||||
if (!kql_operator_p->parse(npos, node, expected))
|
||||
return false;
|
||||
if (kql_operator_str == "print")
|
||||
{
|
||||
++npos;
|
||||
if (!ParserKQLPrint().parse(npos, node, expected))
|
||||
return false;
|
||||
}
|
||||
else if (kql_operator_str == "table")
|
||||
{
|
||||
if (!kql_operator_p->parse(npos, node, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else if (operation_pos.size() == 2 && operation_pos.front().first == "table")
|
||||
{
|
||||
if (!kql_operator_p->parse(npos, node, expected))
|
||||
return false;
|
||||
npos = operation_pos.front().second;
|
||||
if (!ParserKQLTable().parse(npos, node, expected))
|
||||
return false;
|
||||
|
||||
npos = operation_pos.back().second;
|
||||
if (!kql_operator_p->parse(npos, node, expected))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -207,7 +464,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
auto last_pos = operation_pos.back().second;
|
||||
auto last_op = operation_pos.back().first;
|
||||
|
||||
auto set_main_query_clause =[&](String & op, Pos & op_pos)
|
||||
auto set_main_query_clause = [&](const String & op, Pos & op_pos)
|
||||
{
|
||||
auto op_str = ParserKQLBase::getExprFromPipe(op_pos);
|
||||
if (op == "project")
|
||||
@ -224,12 +481,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|
||||
operation_pos.pop_back();
|
||||
|
||||
if (kql_parser[last_op].need_input)
|
||||
{
|
||||
if (!kql_operator_p->parse(npos, node, expected))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
if (!kql_parser[last_op].need_input)
|
||||
{
|
||||
while (!operation_pos.empty())
|
||||
{
|
||||
@ -240,6 +492,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
break;
|
||||
if (!project_clause.empty() && prev_op == "project")
|
||||
break;
|
||||
|
||||
set_main_query_clause(prev_op, prev_pos);
|
||||
operation_pos.pop_back();
|
||||
last_op = prev_op;
|
||||
@ -249,7 +502,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|
||||
if (!operation_pos.empty())
|
||||
{
|
||||
for (auto i = 0; i< kql_parser[last_op].backspace_steps; ++i)
|
||||
for (auto i = 0; i < kql_parser[last_op].backspace_steps; ++i)
|
||||
--last_pos;
|
||||
|
||||
String sub_query = std::format("({})", String(operation_pos.front().second->begin, last_pos->end));
|
||||
@ -266,7 +519,10 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
}
|
||||
|
||||
auto set_query_clasue =[&](String op_str, String op_calsue)
|
||||
if (!kql_operator_p->parse(npos, node, expected))
|
||||
return false;
|
||||
|
||||
auto set_query_clasue = [&](String op_str, String op_calsue)
|
||||
{
|
||||
auto oprator = getOperator(op_str);
|
||||
if (oprator)
|
||||
@ -279,7 +535,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return true;
|
||||
};
|
||||
|
||||
if (!select_query->select())
|
||||
if (!node->as<ASTSelectQuery>()->select())
|
||||
{
|
||||
if (project_clause.empty())
|
||||
project_clause = "*";
|
||||
@ -301,23 +557,23 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return true;
|
||||
}
|
||||
|
||||
if (!select_query->select())
|
||||
if (!node->as<ASTSelectQuery>()->select())
|
||||
{
|
||||
auto expr = String("*");
|
||||
Tokens tokens(expr.c_str(), expr.c_str()+expr.size());
|
||||
Tokens tokens(expr.c_str(), expr.c_str() + expr.size());
|
||||
IParser::Pos new_pos(tokens, pos.max_depth);
|
||||
if (!std::make_unique<ParserKQLProject>()->parse(new_pos, node, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserKQLSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr select_node;
|
||||
|
||||
if (!ParserKQLTaleFunction().parse(pos, select_node, expected))
|
||||
if (!ParserKQLTableFunction().parse(pos, select_node, expected))
|
||||
return false;
|
||||
|
||||
ASTPtr node_subquery = std::make_shared<ASTSubquery>();
|
||||
@ -339,4 +595,46 @@ bool ParserKQLSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserSimpleCHSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr sub_select_node;
|
||||
ParserSelectWithUnionQuery select;
|
||||
|
||||
if (pos->type != TokenType::OpeningRoundBracket)
|
||||
return false;
|
||||
++pos;
|
||||
|
||||
if (!select.parse(pos, sub_select_node, expected))
|
||||
return false;
|
||||
|
||||
if (pos->type != TokenType::ClosingRoundBracket)
|
||||
return false;
|
||||
++pos;
|
||||
|
||||
if (parent_select_node && parent_select_node->as<ASTSelectQuery>()->tables())
|
||||
{
|
||||
auto select_query = sub_select_node->as<ASTSelectWithUnionQuery>()->list_of_selects->children[0];
|
||||
select_query->as<ASTSelectQuery>()->setExpression(
|
||||
ASTSelectQuery::Expression::TABLES, parent_select_node->as<ASTSelectQuery>()->tables());
|
||||
}
|
||||
|
||||
ASTPtr node_subquery = std::make_shared<ASTSubquery>();
|
||||
node_subquery->children.push_back(sub_select_node);
|
||||
|
||||
ASTPtr node_table_expr = std::make_shared<ASTTableExpression>();
|
||||
node_table_expr->as<ASTTableExpression>()->subquery = node_subquery;
|
||||
|
||||
node_table_expr->children.emplace_back(node_subquery);
|
||||
|
||||
ASTPtr node_table_in_select_query_emlement = std::make_shared<ASTTablesInSelectQueryElement>();
|
||||
node_table_in_select_query_emlement->as<ASTTablesInSelectQueryElement>()->table_expression = node_table_expr;
|
||||
|
||||
ASTPtr res = std::make_shared<ASTTablesInSelectQuery>();
|
||||
|
||||
res->children.emplace_back(node_table_in_select_query_emlement);
|
||||
|
||||
node = res;
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -9,24 +9,57 @@ class ParserKQLBase : public IParserBase
|
||||
{
|
||||
public:
|
||||
static String getExprFromToken(Pos & pos);
|
||||
static String getExprFromToken(const String & text, const uint32_t max_depth);
|
||||
static String getExprFromPipe(Pos & pos);
|
||||
static String getExprFromToken(const String & text, const uint32_t & max_depth);
|
||||
static bool setSubQuerySource(ASTPtr & select_query, ASTPtr & source, bool dest_is_subquery, bool src_is_subquery);
|
||||
static bool parseSQLQueryByString(ParserPtr && parser, String & query, ASTPtr & select_node, int32_t max_depth);
|
||||
bool parseByString(const String expr, ASTPtr & node, const uint32_t max_depth);
|
||||
};
|
||||
|
||||
class ParserKQLQuery : public IParserBase
|
||||
{
|
||||
|
||||
protected:
|
||||
static std::unique_ptr<IParserBase> getOperator(String &op_name);
|
||||
static std::unique_ptr<IParserBase> getOperator(String & op_name);
|
||||
const char * getName() const override { return "KQL query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
class ParserKQLSubquery : public IParserBase
|
||||
class ParserKQLSubquery : public ParserKQLBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "KQL subquery"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
class ParserSimpleCHSubquery : public ParserKQLBase
|
||||
{
|
||||
public:
|
||||
ParserSimpleCHSubquery(ASTPtr parent_select_node_ = nullptr) { parent_select_node = parent_select_node_; }
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "Simple ClickHouse subquery"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
ASTPtr parent_select_node;
|
||||
};
|
||||
|
||||
class BracketCount
|
||||
{
|
||||
public:
|
||||
void count(IParser::Pos & pos)
|
||||
{
|
||||
if (pos->type == TokenType::OpeningRoundBracket)
|
||||
++round_bracket_count;
|
||||
if (pos->type == TokenType::ClosingRoundBracket)
|
||||
--round_bracket_count;
|
||||
if (pos->type == TokenType::OpeningSquareBracket)
|
||||
++square_bracket_count;
|
||||
if (pos->type == TokenType::ClosingSquareBracket)
|
||||
--square_bracket_count;
|
||||
}
|
||||
bool isZero() const { return round_bracket_count == 0 && square_bracket_count == 0; }
|
||||
|
||||
private:
|
||||
int16_t round_bracket_count = 0;
|
||||
int16_t square_bracket_count = 0;
|
||||
};
|
||||
}
|
||||
|
@ -1,17 +1,17 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/ASTOrderByElement.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLSort.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLSort :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
bool ParserKQLSort::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
bool has_dir = false;
|
||||
std::vector <bool> has_directions;
|
||||
std::vector<bool> has_directions;
|
||||
ParserOrderByExpressionList order_list;
|
||||
ASTPtr order_expression_list;
|
||||
|
||||
@ -35,6 +35,7 @@ bool ParserKQLSort :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
has_directions.push_back(has_dir);
|
||||
has_dir = false;
|
||||
}
|
||||
|
||||
++new_pos;
|
||||
}
|
||||
has_directions.push_back(has_dir);
|
||||
@ -43,7 +44,7 @@ bool ParserKQLSort :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
if (!has_directions[i])
|
||||
{
|
||||
auto *order_expr = order_expression_list->children[i]->as<ASTOrderByElement>();
|
||||
auto * order_expr = order_expression_list->children[i]->as<ASTOrderByElement>();
|
||||
order_expr->direction = -1; // default desc
|
||||
if (!order_expr->nulls_direction_was_explicitly_specified)
|
||||
order_expr->nulls_direction = -1;
|
||||
@ -53,7 +54,6 @@ bool ParserKQLSort :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
}
|
||||
|
||||
node->as<ASTSelectQuery>()->setExpression(ASTSelectQuery::Expression::ORDER_BY, std::move(order_expression_list));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -1,10 +1,11 @@
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -14,8 +15,7 @@ bool ParserKQLStatement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKQLWithOutput query_with_output_p(end, allow_settings_after_format_in_insert);
|
||||
ParserSetQuery set_p;
|
||||
|
||||
bool res = query_with_output_p.parse(pos, node, expected)
|
||||
|| set_p.parse(pos, node, expected);
|
||||
bool res = query_with_output_p.parse(pos, node, expected) || set_p.parse(pos, node, expected);
|
||||
|
||||
return res;
|
||||
}
|
||||
@ -36,6 +36,7 @@ bool ParserKQLWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
|
||||
|
||||
bool ParserKQLWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
// will support union next phase
|
||||
ASTPtr kql_query;
|
||||
|
||||
if (!ParserKQLQuery().parse(pos, kql_query, expected))
|
||||
@ -58,33 +59,40 @@ bool ParserKQLWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserKQLTaleFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
bool ParserKQLTableFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKQLWithUnionQuery kql_p;
|
||||
ASTPtr select;
|
||||
ParserToken s_lparen(TokenType::OpeningRoundBracket);
|
||||
|
||||
auto begin = pos;
|
||||
auto paren_count = 0 ;
|
||||
auto paren_count = 0;
|
||||
String kql_statement;
|
||||
|
||||
if (s_lparen.ignore(pos, expected))
|
||||
{
|
||||
++paren_count;
|
||||
while (!pos->isEnd())
|
||||
if (pos->type == TokenType::HereDoc)
|
||||
{
|
||||
if (pos->type == TokenType::ClosingRoundBracket)
|
||||
--paren_count;
|
||||
if (pos->type == TokenType::OpeningRoundBracket)
|
||||
++paren_count;
|
||||
|
||||
if (paren_count == 0)
|
||||
break;
|
||||
|
||||
kql_statement = kql_statement + " " + String(pos->begin,pos->end);
|
||||
++pos;
|
||||
kql_statement = String(pos->begin + 2, pos->end - 2);
|
||||
}
|
||||
else
|
||||
{
|
||||
++paren_count;
|
||||
auto pos_start = pos;
|
||||
while (!pos->isEnd())
|
||||
{
|
||||
if (pos->type == TokenType::ClosingRoundBracket)
|
||||
--paren_count;
|
||||
if (pos->type == TokenType::OpeningRoundBracket)
|
||||
++paren_count;
|
||||
|
||||
if (paren_count == 0)
|
||||
break;
|
||||
++pos;
|
||||
}
|
||||
kql_statement = String(pos_start->begin, (--pos)->end);
|
||||
}
|
||||
++pos;
|
||||
Tokens token_kql(kql_statement.c_str(), kql_statement.c_str() + kql_statement.size());
|
||||
IParser::Pos pos_kql(token_kql, pos.max_depth);
|
||||
|
||||
@ -95,8 +103,7 @@ bool ParserKQLTaleFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
return true;
|
||||
}
|
||||
}
|
||||
pos = begin;
|
||||
pos = begin;
|
||||
return false;
|
||||
};
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -12,14 +12,14 @@ private:
|
||||
bool allow_settings_after_format_in_insert;
|
||||
const char * getName() const override { return "KQL Statement"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
public:
|
||||
explicit ParserKQLStatement(const char * end_, bool allow_settings_after_format_in_insert_ = false)
|
||||
: end(end_)
|
||||
, allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_)
|
||||
{}
|
||||
: end(end_), allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_)
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class ParserKQLWithOutput : public IParserBase
|
||||
{
|
||||
protected:
|
||||
@ -27,11 +27,12 @@ protected:
|
||||
bool allow_settings_after_format_in_insert;
|
||||
const char * getName() const override { return "KQL with output"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
public:
|
||||
explicit ParserKQLWithOutput(const char * end_, bool allow_settings_after_format_in_insert_ = false)
|
||||
: end(end_)
|
||||
, allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_)
|
||||
{}
|
||||
: end(end_), allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_)
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
class ParserKQLWithUnionQuery : public IParserBase
|
||||
@ -41,7 +42,7 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
class ParserKQLTaleFunction : public IParserBase
|
||||
class ParserKQLTableFunction : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "KQL() function"; }
|
||||
@ -49,4 +50,3 @@ protected:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -1,3 +1,8 @@
|
||||
#include <memory>
|
||||
#include <queue>
|
||||
#include <vector>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTInterpolateElement.h>
|
||||
@ -10,15 +15,17 @@
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLSummarize.h>
|
||||
#include <Parsers/ParserSampleRatio.h>
|
||||
#include <Parsers/ParserSelectQuery.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
#include <Parsers/ParserTablesInSelectQuery.h>
|
||||
#include <Parsers/ParserWithElement.h>
|
||||
#include <format>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLSummarize ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
bool ParserKQLSummarize::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr select_expression_list;
|
||||
ASTPtr group_expression_list;
|
||||
@ -27,33 +34,164 @@ bool ParserKQLSummarize ::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
|
||||
String expr_groupby;
|
||||
String expr_columns;
|
||||
bool groupby = false;
|
||||
auto column_begin_pos = pos;
|
||||
|
||||
auto begin = pos;
|
||||
auto pos_groupby = pos;
|
||||
uint16_t bracket_count = 0;
|
||||
int32_t new_column_index = 1;
|
||||
|
||||
std::vector<String> expr_aggregations;
|
||||
std::vector<String> expr_groupbys;
|
||||
|
||||
std::unordered_set<String> aggregate_functions(
|
||||
{"arg_max",
|
||||
"arg_min",
|
||||
"avg",
|
||||
"avgif",
|
||||
"binary_all_and",
|
||||
"binary_all_or",
|
||||
"binary_all_xor",
|
||||
"buildschema",
|
||||
"count",
|
||||
"countif",
|
||||
"dcount",
|
||||
"dcountif",
|
||||
"make_bag",
|
||||
"make_bag_if",
|
||||
"make_list",
|
||||
"make_list_if",
|
||||
"make_list_with_nulls",
|
||||
"make_set",
|
||||
"make_set_if",
|
||||
"max",
|
||||
"maxif",
|
||||
"min",
|
||||
"minif",
|
||||
"percentile",
|
||||
"percentilew",
|
||||
"percentiles",
|
||||
"percentiles_array",
|
||||
"percentilesw",
|
||||
"percentilesw_array",
|
||||
"stdev",
|
||||
"stdevif",
|
||||
"sum",
|
||||
"sumif",
|
||||
"take_any",
|
||||
"take_anyif",
|
||||
"variance",
|
||||
"varianceif"});
|
||||
|
||||
auto apply_aliais = [&](Pos & begin_pos, Pos & end_pos, bool is_groupby)
|
||||
{
|
||||
auto expr = String(begin_pos->begin, end_pos->end);
|
||||
auto equal_pos = begin_pos;
|
||||
++equal_pos;
|
||||
if (!is_groupby)
|
||||
{
|
||||
if (String(equal_pos->begin, equal_pos->end) != "=")
|
||||
{
|
||||
String alias;
|
||||
String aggregate_fun = String(begin_pos->begin, begin_pos->end);
|
||||
if (aggregate_functions.find(aggregate_fun) == aggregate_functions.end())
|
||||
{
|
||||
alias = std::format("Columns{}", new_column_index);
|
||||
++new_column_index;
|
||||
}
|
||||
else
|
||||
{
|
||||
alias = std::format("{}_", aggregate_fun);
|
||||
auto agg_colum_pos = begin_pos;
|
||||
++agg_colum_pos;
|
||||
++agg_colum_pos;
|
||||
++agg_colum_pos;
|
||||
if (agg_colum_pos->type == TokenType::Comma || agg_colum_pos->type == TokenType::ClosingRoundBracket)
|
||||
{
|
||||
--agg_colum_pos;
|
||||
if (agg_colum_pos->type != TokenType::ClosingRoundBracket)
|
||||
alias = alias + String(agg_colum_pos->begin, agg_colum_pos->end);
|
||||
}
|
||||
}
|
||||
expr = std::format("{} = {}", alias, expr);
|
||||
}
|
||||
expr_aggregations.push_back(expr);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (String(equal_pos->begin, equal_pos->end) != "=")
|
||||
{
|
||||
String groupby_fun = String(begin_pos->begin, begin_pos->end);
|
||||
if (equal_pos->isEnd() || equal_pos->type == TokenType::Comma || equal_pos->type == TokenType::Semicolon
|
||||
|| equal_pos->type == TokenType::PipeMark)
|
||||
{
|
||||
expr = groupby_fun;
|
||||
}
|
||||
else
|
||||
{
|
||||
String alias;
|
||||
if (groupby_fun == "bin" || groupby_fun == "bin_at")
|
||||
{
|
||||
auto bin_colum_pos = begin_pos;
|
||||
++bin_colum_pos;
|
||||
++bin_colum_pos;
|
||||
alias = String(bin_colum_pos->begin, bin_colum_pos->end);
|
||||
++bin_colum_pos;
|
||||
if (bin_colum_pos->type != TokenType::Comma)
|
||||
alias.clear();
|
||||
}
|
||||
if (alias.empty())
|
||||
{
|
||||
alias = std::format("Columns{}", new_column_index);
|
||||
++new_column_index;
|
||||
}
|
||||
|
||||
expr = std::format("{} = {}", alias, expr);
|
||||
}
|
||||
}
|
||||
expr_groupbys.push_back(expr);
|
||||
}
|
||||
};
|
||||
|
||||
while (!pos->isEnd() && pos->type != TokenType::PipeMark && pos->type != TokenType::Semicolon)
|
||||
{
|
||||
if (String(pos->begin, pos->end) == "by")
|
||||
if (pos->type == TokenType::OpeningRoundBracket)
|
||||
++bracket_count;
|
||||
|
||||
if (pos->type == TokenType::ClosingRoundBracket)
|
||||
--bracket_count;
|
||||
|
||||
if ((bracket_count == 0 and pos->type == TokenType::Comma) || String(pos->begin, pos->end) == "by")
|
||||
{
|
||||
groupby = true;
|
||||
auto end = pos;
|
||||
--end;
|
||||
expr_aggregation = begin <= end ? String(begin->begin, end->end) : "";
|
||||
pos_groupby = pos;
|
||||
++pos_groupby;
|
||||
auto end_pos = pos;
|
||||
--end_pos;
|
||||
apply_aliais(column_begin_pos, end_pos, groupby);
|
||||
if (String(pos->begin, pos->end) == "by")
|
||||
groupby = true;
|
||||
column_begin_pos = pos;
|
||||
++column_begin_pos;
|
||||
}
|
||||
++pos;
|
||||
}
|
||||
--pos;
|
||||
if (groupby)
|
||||
expr_groupby = String(pos_groupby->begin, pos->end);
|
||||
else
|
||||
expr_aggregation = begin <= pos ? String(begin->begin, pos->end) : "";
|
||||
apply_aliais(column_begin_pos, pos, groupby);
|
||||
|
||||
auto expr_aggregation_str = expr_aggregation.empty() ? "" : expr_aggregation +",";
|
||||
expr_columns = groupby ? expr_aggregation_str + expr_groupby : expr_aggregation_str;
|
||||
for (auto const & expr : expr_aggregations)
|
||||
expr_aggregation = expr_aggregation.empty() ? expr : expr_aggregation + "," + expr;
|
||||
|
||||
String converted_columns = getExprFromToken(expr_columns, pos.max_depth);
|
||||
for (auto const & expr : expr_groupbys)
|
||||
expr_groupby = expr_groupby.empty() ? expr : expr_groupby + "," + expr;
|
||||
|
||||
if (!expr_groupby.empty())
|
||||
expr_columns = expr_groupby;
|
||||
|
||||
if (!expr_aggregation.empty())
|
||||
{
|
||||
if (expr_columns.empty())
|
||||
expr_columns = expr_aggregation;
|
||||
else
|
||||
expr_columns = expr_columns + "," + expr_aggregation;
|
||||
}
|
||||
|
||||
String converted_columns = getExprFromToken(expr_columns, pos.max_depth);
|
||||
|
||||
Tokens token_converted_columns(converted_columns.c_str(), converted_columns.c_str() + converted_columns.size());
|
||||
IParser::Pos pos_converted_columns(token_converted_columns, pos.max_depth);
|
||||
@ -65,7 +203,7 @@ bool ParserKQLSummarize ::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
|
||||
|
||||
if (groupby)
|
||||
{
|
||||
String converted_groupby = getExprFromToken(expr_groupby, pos.max_depth);
|
||||
String converted_groupby = getExprFromToken(expr_groupby, pos.max_depth);
|
||||
|
||||
Tokens token_converted_groupby(converted_groupby.c_str(), converted_groupby.c_str() + converted_groupby.size());
|
||||
IParser::Pos postoken_converted_groupby(token_converted_groupby, pos.max_depth);
|
||||
@ -77,5 +215,4 @@ bool ParserKQLSummarize ::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -8,7 +8,6 @@ namespace DB
|
||||
|
||||
class ParserKQLSummarize : public ParserKQLBase
|
||||
{
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "KQL summarize"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
@ -1,45 +1,23 @@
|
||||
#include <unordered_set>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/ParserTablesInSelectQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLQuery.h>
|
||||
#include <Parsers/Kusto/ParserKQLTable.h>
|
||||
#include <unordered_set>
|
||||
#include <Parsers/ParserTablesInSelectQuery.h>
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserKQLTable :: parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
bool ParserKQLTable ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
std::unordered_set<String> sql_keywords
|
||||
({
|
||||
"SELECT",
|
||||
"INSERT",
|
||||
"CREATE",
|
||||
"ALTER",
|
||||
"SYSTEM",
|
||||
"SHOW",
|
||||
"GRANT",
|
||||
"REVOKE",
|
||||
"ATTACH",
|
||||
"CHECK",
|
||||
"DESCRIBE",
|
||||
"DESC",
|
||||
"DETACH",
|
||||
"DROP",
|
||||
"EXISTS",
|
||||
"KILL",
|
||||
"OPTIMIZE",
|
||||
"RENAME",
|
||||
"SET",
|
||||
"TRUNCATE",
|
||||
"USE",
|
||||
"EXPLAIN"
|
||||
});
|
||||
std::unordered_set<String> sql_keywords({"SELECT", "INSERT", "CREATE", "ALTER", "SYSTEM", "SHOW", "GRANT", "REVOKE",
|
||||
"ATTACH", "CHECK", "DESCRIBE", "DESC", "DETACH", "DROP", "EXISTS", "KILL",
|
||||
"OPTIMIZE", "RENAME", "SET", "TRUNCATE", "USE", "EXPLAIN"});
|
||||
|
||||
ASTPtr tables;
|
||||
String table_name(pos->begin,pos->end);
|
||||
String table_name(pos->begin, pos->end);
|
||||
String table_name_upcase(table_name);
|
||||
|
||||
std::transform(table_name_upcase.begin(), table_name_upcase.end(),table_name_upcase.begin(), toupper);
|
||||
std::transform(table_name_upcase.begin(), table_name_upcase.end(), table_name_upcase.begin(), toupper);
|
||||
|
||||
if (sql_keywords.find(table_name_upcase) != sql_keywords.end())
|
||||
return false;
|
||||
|
@ -8,7 +8,6 @@ namespace DB
|
||||
|
||||
class ParserKQLTable : public ParserKQLBase
|
||||
{
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "KQL Table"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
74
src/Parsers/Kusto/Utilities.cpp
Normal file
74
src/Parsers/Kusto/Utilities.cpp
Normal file
@ -0,0 +1,74 @@
|
||||
#include "Utilities.h"
|
||||
|
||||
#include "KustoFunctions/IParserKQLFunction.h"
|
||||
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
String extractLiteralArgumentWithoutQuotes(const std::string & function_name, IParser::Pos & pos)
|
||||
{
|
||||
++pos;
|
||||
if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral)
|
||||
{
|
||||
auto result = extractTokenWithoutQuotes(pos);
|
||||
++pos;
|
||||
return result;
|
||||
}
|
||||
|
||||
--pos;
|
||||
return IParserKQLFunction::getArgument(function_name, pos, IParserKQLFunction::ArgumentState::Raw);
|
||||
}
|
||||
|
||||
String extractTokenWithoutQuotes(IParser::Pos & pos)
|
||||
{
|
||||
const auto offset = static_cast<int>(pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral);
|
||||
return {pos->begin + offset, pos->end - offset};
|
||||
}
|
||||
|
||||
void setSelectAll(ASTSelectQuery & select_query)
|
||||
{
|
||||
auto expression_list = std::make_shared<ASTExpressionList>();
|
||||
expression_list->children.push_back(std::make_shared<ASTAsterisk>());
|
||||
select_query.setExpression(ASTSelectQuery::Expression::SELECT, std::move(expression_list));
|
||||
}
|
||||
|
||||
String wildcardToRegex(const String & wildcard)
|
||||
{
|
||||
String regex;
|
||||
for (char c : wildcard)
|
||||
{
|
||||
if (c == '*')
|
||||
{
|
||||
regex += ".*";
|
||||
}
|
||||
else if (c == '?')
|
||||
{
|
||||
regex += ".";
|
||||
}
|
||||
else if (c == '.' || c == '+' || c == '(' || c == ')' || c == '[' || c == ']' || c == '\\' || c == '^' || c == '$')
|
||||
{
|
||||
regex += "\\";
|
||||
regex += c;
|
||||
}
|
||||
else
|
||||
{
|
||||
regex += c;
|
||||
}
|
||||
}
|
||||
return regex;
|
||||
}
|
||||
|
||||
ASTPtr wrapInSelectWithUnion(const ASTPtr & select_query)
|
||||
{
|
||||
auto select_with_union_query = std::make_shared<ASTSelectWithUnionQuery>();
|
||||
auto & list_of_selects = select_with_union_query->list_of_selects;
|
||||
list_of_selects = std::make_shared<ASTExpressionList>();
|
||||
list_of_selects->children.push_back(select_query);
|
||||
select_with_union_query->children.push_back(list_of_selects);
|
||||
|
||||
return select_with_union_query;
|
||||
}
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user