Skip to content

Commit

Permalink
Merge pull request #272 from ClibMouse/KQL_X3_URL_Parse
Browse files Browse the repository at this point in the history
KQL URL parse using x3
  • Loading branch information
ltrk2 authored Apr 3, 2023
2 parents 764876c + c3ef4c1 commit 258dabb
Show file tree
Hide file tree
Showing 7 changed files with 240 additions and 60 deletions.
154 changes: 154 additions & 0 deletions src/Functions/Kusto/kqlParseURL.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,154 @@
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <boost/fusion/adapted/std_pair.hpp>
#include <boost/spirit/home/x3.hpp>

#include <format>

namespace x3 = boost::spirit::x3;

namespace
{
using x3::char_;
using x3::lexeme;
using x3::lit;

struct KQLURLstate
{
std::string schema;
std::string user;
std::string pass;
std::string host;
std::string port;
std::string path;
std::string frag;
std::vector<std::pair<std::string, std::string>> args;
};

const auto endschema = lit("://");
const auto colon = lit(":");
const auto at = lit("@");
const auto slash = lit("/");
const auto equals = lit("=");
const auto fragmark = lit("#");
const auto openbracket = lit("[");
const auto closebracket = lit("]");
const auto question = lit("?");
const auto ampersand = lit("&");

const auto endhost = char_("/:?#");
const auto endport = char_("/?#");
const auto endauth = char_("@:/?#");
const auto endpath = char_("?#");
const auto endarg = char_("#&");

const auto set_schema = [](auto & ctx) { _val(ctx).schema = _attr(ctx); };
const auto set_auth = [](auto & ctx)
{
const auto & auth = _attr(ctx);
_val(ctx).user = at_c<0>(auth);
_val(ctx).pass = at_c<1>(auth);
};
const auto set_host = [](auto & ctx) { _val(ctx).host = _attr(ctx); };
const auto set_port = [](auto & ctx) { _val(ctx).port = _attr(ctx); };
const auto set_path = [](auto & ctx) { _val(ctx).path = _attr(ctx); };
const auto set_arg = [](auto & ctx)
{
const auto & arg = _attr(ctx);
_val(ctx).args.emplace_back(at_c<0>(arg), at_c<1>(arg));
};
const auto set_frag = [](auto & ctx) { _val(ctx).frag = _attr(ctx); };

template <typename T>
auto as = [](auto p) { return x3::rule<struct _, T>{} = as_parser(p); };

const auto KQL_URL_SCHEMA_def = lexeme[+(char_ - endschema) >> endschema][set_schema];
const auto KQL_URL_AUTH_def = lexeme[+(char_ - endauth) >> colon >> +(char_ - endauth) >> at][set_auth];
const auto KQL_URL_HOST_def
= lexeme[as<std::string>((openbracket >> +(char_ - closebracket) >> closebracket) | (+(char_ - endhost)))][set_host];
const auto KQL_URL_PORT_def = lexeme[colon >> +(char_ - endport)][set_port];
const auto KQL_URL_PATH_def = lexeme[&slash >> +(char_ - endpath)][set_path];
const auto KQL_URL_ARG_def = lexeme[(question | ampersand) >> +(char_ - equals) >> equals >> +(char_ - endarg)][set_arg];
const auto KQL_URL_FRAG_def = lexeme[fragmark >> +char_][set_frag];

const x3::rule<class KQLURL, KQLURLstate> KQL_URL = "KQL URL";
const auto KQL_URL_def = KQL_URL_SCHEMA_def >> -KQL_URL_AUTH_def >> -KQL_URL_HOST_def >> -KQL_URL_PORT_def >> -KQL_URL_PATH_def
>> *KQL_URL_ARG_def >> -KQL_URL_FRAG_def;

BOOST_SPIRIT_DEFINE(KQL_URL);
}

namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}

class FunctionKqlParseURL : public IFunction
{
public:
static constexpr auto name = "kql_parseurl";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionKqlParseURL>(std::move(context)); }

explicit FunctionKqlParseURL(ContextPtr context_) : context(std::move(context_)) { }
~FunctionKqlParseURL() override = default;

ColumnPtr
executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override;
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
DataTypePtr getReturnTypeImpl(const DataTypes &) const override { return std::make_shared<DataTypeString>(); }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }

private:
ContextPtr context;
};

ColumnPtr
FunctionKqlParseURL::executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, const size_t input_rows_count) const
{
auto result = ColumnString::create();

if (!isStringOrFixedString(arguments.at(0).type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "parse_url(): argument #1 - invalid data type: string");

for (size_t i = 0; i < input_rows_count; ++i)
{
const auto in_str = arguments[0].column->getDataAt(i).toView();
KQLURLstate url;
parse(in_str.begin(), in_str.end(), KQL_URL, url);
bool first = false;
std::string args = "{";
for (auto q_iter = url.args.begin(); q_iter < url.args.end(); ++q_iter)
{
args.append((first ? ",\"" : "\"") + q_iter->first + "\":\"" + q_iter->second + "\"");
first = true;
}
args.append("}");
const auto out_str = std::format(
"{}\"Scheme\":\"{}\",\"Host\":\"{}\",\"Port\":\"{}\",\"Path\":\"{}\",\"Username\":\"{}\",\"Password\":\"{}\",\"Query "
"Parameters\":{},\"Fragment\":\"{}\"{}",
"{",
url.schema,
url.host,
url.port,
url.path,
url.user,
url.pass,
args,
url.frag,
"}");
result->insertData(out_str.c_str(), out_str.size());
}
return result;
}

REGISTER_FUNCTION(KqlParseURL)
{
factory.registerFunction<FunctionKqlParseURL>();
}
}
38 changes: 36 additions & 2 deletions src/Parsers/Kusto/KQL_ReleaseNote.md
Original file line number Diff line number Diff line change
@@ -1,6 +1,41 @@
## KQL implemented features
## KQL implemented features
# April XX, 2023
## Bugfixes
- Corrected an issue with parse_url in which hostnames and port numbers were not correctly parsed.
```
parse_url follows the folowing structure.
Scheme://Username:Password@Host:Port/Path?QueryParameters#Fragment
'://' is required for further parsing.
All other fields are optional and are parsed from left to right.
Username and Password are parsed together, require ':' and '@', and will not match if either contains '/', '?', or '#'.
IPv6 addresses are required to be encapsulated in brackets.
Host ends with '/', ':', '?' or '#'.
Port starts with ':' and ends with '/', '?' or '#'.
Path requires to start with '/' and ends with '?' or '#'.
Query Parameters is recursive, starts with '?', ends with '#', expected to be in the form of argument=value, and separated by '&'.
Fragment must start with '#'.
Notes on differences between ADX and ClickHouse:
ClickHouse will return a formated string. 'extract_json' can be used to convert the string.
print x = parse_url("http://[2001:0db8:0000:0000:0000:ff00:0042:8329]?asd=qwe&qwe=asd") | project extract_json("$.Scheme", x);
ClickHouse includes Path as '/' where ADX requires anything after '/' to populate Path.
print parse_url("http://host:1234/");
ClickHouse includes Port where ADX requires '/' for Port and without '/' will treat Port as part of Host.
print parse_url("http://host:1234?arg=value")
ClickHouse includes arg value in Query parameters where ADX treats this as host.
print parse_url("http://?arg=value");
ClickHouse will not parse IPv6 addresses not encapsulated in brackets [RFC 3986](https://datatracker.ietf.org/doc/html/rfc3986#section-3.2.2)
Correct IPv6
print parse_url("http://[2001:db8:3333:4444:5555:6666:7777:8888]:1234/filepath/index.htm")
Incorrect IPv6
print parse_url("http://2001:db8:3333:4444:5555:6666:7777:8888:1234/filepath/index.htm");
print parse_url("http://2001:db8:3333:4444:5555:6666:7777:8888/filepath/index.htm");
ADX will incorrectly consume part of encapsulated IPv6 Host as Port from last colon to '/'.
print parse_url("http://[2001:db8:3333:4444:5555:6666:7777:8888]/filepath/index.htm")
```
- [arg_max()](https://learn.microsoft.com/en-us/azure/data-explorer/kusto/query/arg-max-aggfunction) and [arg_min()](https://learn.microsoft.com/en-us/azure/data-explorer/kusto/query/arg-min-aggfunction)
support multiple arguments now.
`Customers | arg_max(Age, FirstName, LastName)`
Expand Down Expand Up @@ -37,7 +72,6 @@
select * from kql(TableWithVariousDataTypes | project JoinDate | where JoinDate between (datetime('2020-06-30') .. datetime('2025-06-30')));
select * from kql(TableWithVariousDataTypes | project JoinDate | where JoinDate !between (datetime('2020-06-30') .. datetime('2025-06-30')));
```
# March 15, 2023
## Feature
- KQL - improve timespan textual representation in the CLI
Expand Down
50 changes: 1 addition & 49 deletions src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -419,55 +419,7 @@ bool ParseJson::convertImpl(String & out, IParser::Pos & pos)

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);
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);
String query_parameters
= std::format(R"(concat('"Query Parameters":', concat('{{"', replace(replace({}, '=', '":"'),'&','","') ,'"}}')))", query_string);

bool all_space = true;
for (char ch : url)
{
if (ch == '\'' || ch == '\"')
continue;
if (ch != ' ')
{
all_space = false;
break;
}
}

if (all_space)
{
port = R"('"Port":""')";
query_parameters = "'\"Query Parameters\":{}'";
}
out = std::format(
"concat('{{',{},',',{},',',{},',',{},',',{},',',{},',',{},',',{},'}}')",
scheme,
host,
port,
path,
username,
password,
query_parameters,
fragment);
return true;
return directMapping(out, pos, "kql_parseurl");
}

bool ParseURLQuery::convertImpl(String & out, IParser::Pos & pos)
Expand Down
4 changes: 0 additions & 4 deletions src/Parsers/tests/KQL/gtest_KQL_Operators.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -250,10 +250,6 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_Operators, ParserTest,
"print strcmp('a','b')",
"SELECT multiIf('a' = 'b', 0, 'a' < 'b', -1, 1) AS print_0"
},
{
"print parse_url('https://john:[email protected]:1234/this/is/a/path?k1=v1&k2=v2#fragment')",
"SELECT concat('{', concat('\"Scheme\":\"', protocol('https://john:[email protected]:1234/this/is/a/path?k1=v1&k2=v2#fragment'), '\"'), ',', concat('\"Host\":\"', domain('https://john:[email protected]:1234/this/is/a/path?k1=v1&k2=v2#fragment'), '\"'), ',', concat('\"Port\":\"', toString(port('https://john:[email protected]:1234/this/is/a/path?k1=v1&k2=v2#fragment')), '\"'), ',', concat('\"Path\":\"', path('https://john:[email protected]:1234/this/is/a/path?k1=v1&k2=v2#fragment'), '\"'), ',', concat('\"Username\":\"', splitByChar(':', splitByChar('@', netloc('https://john:[email protected]:1234/this/is/a/path?k1=v1&k2=v2#fragment'))[1])[1], '\"'), ',', concat('\"Password\":\"', splitByChar(':', splitByChar('@', netloc('https://john:[email protected]:1234/this/is/a/path?k1=v1&k2=v2#fragment'))[1])[2], '\"'), ',', concat('\"Query Parameters\":', concat('{\"', replace(replace(queryString('https://john:[email protected]:1234/this/is/a/path?k1=v1&k2=v2#fragment'), '=', '\":\"'), '&', '\",\"'), '\"}')), ',', concat('\"Fragment\":\"', fragment('https://john:[email protected]:1234/this/is/a/path?k1=v1&k2=v2#fragment'), '\"'), '}') AS print_0"
},
{
"Customers | summarize t = make_list(FirstName) by FirstName",
"SELECT\n FirstName,\n groupArrayIf(FirstName, FirstName IS NOT NULL) AS t\nFROM Customers\nGROUP BY FirstName"
Expand Down
4 changes: 4 additions & 0 deletions src/Parsers/tests/KQL/gtest_KQL_StringFunctions.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -241,6 +241,10 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_String, ParserTest,
{
"print new_guid()",
"SELECT generateUUIDv4() AS print_0"
},
{
"print parse_url('https://john:[email protected]:1234/this/is/a/path?k1=v1&k2=v2#fragment')",
"SELECT kql_parseurl('https://john:[email protected]:1234/this/is/a/path?k1=v1&k2=v2#fragment') AS print_0",
},
{
"print str = make_string(dynamic([75, 117, 115, 116, 111]))",
Expand Down
25 changes: 22 additions & 3 deletions tests/queries/0_stateless/02366_kql_func_string.reference
Original file line number Diff line number Diff line change
Expand Up @@ -302,10 +302,29 @@ S3VzdG8x

Kusto1
\N
-- parse_url()
{"Scheme":"scheme","Host":"","Port":"0","Path":"/this/is/a/path","Username":"username","Password":"password","Query Parameters":{"k1":"v1","k2":"v2"},"Fragment":"fragment"}
-- parse_url()
-- parse_url() same as ADX
{"Scheme":"scheme","Host":"host","Port":"1234","Path":"/this/is/a/path","Username":"username","Password":"password","Query Parameters":{"k1":"v1","k2":"v2"},"Fragment":"fragment"}
{"Scheme":"","Host":"","Port":"","Path":"","Username":"","Password":"","Query Parameters":{},"Fragment":""}
{"Scheme":"http","Host":"2001:db8:3333:4444:5555:6666:7777:8888","Port":"1234","Path":"/filepath/index.htm","Username":"","Password":"","Query Parameters":{},"Fragment":""}
{"Scheme":"http","Host":"host","Port":"1234","Path":"","Username":"","Password":"","Query Parameters":{},"Fragment":""}
{"Scheme":"http","Host":"","Port":"","Path":"/this/is/a/path/index.htm","Username":"","Password":"","Query Parameters":{},"Fragment":""}
{"Scheme":"http","Host":"","Port":"","Path":"","Username":"","Password":"","Query Parameters":{},"Fragment":"fragment"}
{"Scheme":"http","Host":"host","Port":"abcd","Path":"","Username":"","Password":"","Query Parameters":{},"Fragment":""}
{"Scheme":"http","Host":"host","Port":"","Path":"/filepath","Username":"","Password":"","Query Parameters":{"arg":":bogus@some"},"Fragment":""}
{"Scheme":"http","Host":"","Port":"","Path":"","Username":"username","Password":"password","Query Parameters":{},"Fragment":""}
-- parse_url() differs from ADX
{"Scheme":"http","Host":"host","Port":"1234","Path":"/","Username":"","Password":"","Query Parameters":{},"Fragment":""}
{"Scheme":"http","Host":"","Port":"1234","Path":"/","Username":"","Password":"","Query Parameters":{},"Fragment":""}
{"Scheme":"http","Host":"","Port":"","Path":"","Username":"","Password":"","Query Parameters":{"arg":"value"},"Fragment":""}
{"Scheme":"http","Host":"host","Port":"1234","Path":"","Username":"","Password":"","Query Parameters":{"arg":"value"},"Fragment":""}
{"Scheme":"http","Host":"","Port":"","Path":"/","Username":"","Password":"","Query Parameters":{},"Fragment":""}
{"Scheme":"http","Host":"","Port":"","Path":"/filepath","Username":"","Password":"","Query Parameters":{},"Fragment":""}
{"Scheme":"http","Host":"","Port":"port:","Path":"/anything","Username":"","Password":"","Query Parameters":{"arg":"value"},"Fragment":""}
{"Scheme":"http","Host":"","Port":"port:port","Path":"/anything","Username":"","Password":"","Query Parameters":{"arg":"value"},"Fragment":""}
{"Scheme":"http","Host":"host","Port":"","Path":"/","Username":"","Password":"","Query Parameters":{},"Fragment":""}
-- parse_url() invalid IPV6
{"Scheme":"http","Host":"2001","Port":"db8:3333:4444:5555:6666:7777:8888:1234","Path":"/filepath/index.htm","Username":"","Password":"","Query Parameters":{},"Fragment":""}
{"Scheme":"http","Host":"2001","Port":"db8:3333:4444:5555:6666:7777:8888","Path":"/filepath/index.htm","Username":"","Password":"","Query Parameters":{},"Fragment":""}
-- parse_urlquery()
{"Query Parameters":{"k1":"v1","k2":"v2","k3":"v3"}}
-- strcat --
Expand Down
25 changes: 23 additions & 2 deletions tests/queries/0_stateless/02366_kql_func_string.sql
Original file line number Diff line number Diff line change
Expand Up @@ -284,10 +284,31 @@ print '-- base64_decode_tostring';
print base64_decode_tostring('');
print base64_decode_tostring('S3VzdG8x');
print base64_decode_tostring('S3VzdG8====');
print '-- parse_url()';
print '-- parse_url() same as ADX';
print parse_url('scheme://username:password@host:1234/this/is/a/path?k1=v1&k2=v2#fragment');
print '-- parse_url()';
print parse_url('');
print parse_url("http://[2001:db8:3333:4444:5555:6666:7777:8888]:1234/filepath/index.htm")
print parse_url("http://host");
print parse_url("http://host:1234");
print parse_url("http:///this/is/a/path/index.htm");
print parse_url("http://#fragment");
print parse_url("http://host:abcd");
print parse_url('http://host/filepath?arg=:bogus@some');
print parse_url("http://username:password@");
print parse_url(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
print '-- parse_url() differs from ADX';
print parse_url("http://host:1234/");
print parse_url("http://:1234/");
print parse_url("http://?arg=value");
print parse_url("http://host:1234?arg=value");
print parse_url("http:///");
print parse_url("http:///filepath");
print parse_url("http://:port:/anything?arg=value");
print parse_url("http://:port:port/anything?arg=value");
print parse_url("http://host/");
print '-- parse_url() invalid IPV6';
print parse_url("http://2001:db8:3333:4444:5555:6666:7777:8888:1234/filepath/index.htm");
print parse_url("http://2001:db8:3333:4444:5555:6666:7777:8888/filepath/index.htm");
print '-- parse_urlquery()';
print parse_urlquery('k1=v1&k2=v2&k3=v3');
print '-- strcat --';
Expand Down

0 comments on commit 258dabb

Please sign in to comment.