Skip to content

Commit

Permalink
Kusto-phase3 : fix extract bug
Browse files Browse the repository at this point in the history
  • Loading branch information
kashwy authored and ltrk2 committed Nov 25, 2022
1 parent d3e922d commit 8cf6010
Show file tree
Hide file tree
Showing 8 changed files with 137 additions and 55 deletions.
60 changes: 60 additions & 0 deletions src/Functions/FunctionsStringSearchToString.h
Original file line number Diff line number Diff line change
Expand Up @@ -86,4 +86,64 @@ class FunctionsStringSearchToString : public IFunction
}
};


template <typename Impl, typename Name>
class KqlStringSearchToString : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(ContextPtr) { return std::make_shared<KqlStringSearchToString>(); }

String getName() const override { return name; }

size_t getNumberOfArguments() const override { return 3; }

bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }

bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }

DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

if (!isString(arguments[1]))
throw Exception(
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

if (!isUnsignedInteger(arguments[2]))
throw Exception(
"Illegal type " + arguments[2]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

return std::make_shared<DataTypeString>();
}

ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
{
const ColumnPtr column = arguments[0].column;
const ColumnPtr column_needle = arguments[1].column;
const auto capture = arguments[2].column->get64(0);

const ColumnConst * col_needle = typeid_cast<const ColumnConst *>(&*column_needle);
if (!col_needle)
throw Exception("Second argument of function " + getName() + " must be constant string", ErrorCodes::ILLEGAL_COLUMN);

if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
{
auto col_res = ColumnString::create();

ColumnString::Chars & vec_res = col_res->getChars();
ColumnString::Offsets & offsets_res = col_res->getOffsets();
Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue<String>(), capture, vec_res, offsets_res);

return col_res;
}
else
throw Exception(
"Illegal column " + arguments[0].column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};
}
59 changes: 59 additions & 0 deletions src/Functions/extract.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -70,4 +70,63 @@ REGISTER_FUNCTION(Extract)
factory.registerFunction<FunctionExtract>();
}

struct KqlExtractImpl
{
static void vector(
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
const std::string & pattern,
unsigned capture,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
res_data.reserve(data.size() / 5);
res_offsets.resize(offsets.size());

const Regexps::Regexp regexp = Regexps::createRegexp<false, false, false>(pattern);

OptimizedRegularExpression::MatchVec matches;
matches.reserve(capture + 1);
size_t prev_offset = 0;
size_t res_offset = 0;

for (size_t i = 0; i < offsets.size(); ++i)
{
size_t cur_offset = offsets[i];

unsigned count
= regexp.match(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1, matches, capture + 1);
if (count > capture && matches[capture].offset != std::string::npos)
{
const auto & match = matches[capture];
res_data.resize(res_offset + match.length + 1);
memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &data[prev_offset + match.offset], match.length);
res_offset += match.length;
}
else
{
res_data.resize(res_offset + 1);
}

res_data[res_offset] = 0;
++res_offset;
res_offsets[i] = res_offset;

prev_offset = cur_offset;
}
}
};

struct NameKqlExtract
{
static constexpr auto name = "kql_extract";
};

using FunctionKqlExtract = KqlStringSearchToString<KqlExtractImpl, NameKqlExtract>;

REGISTER_FUNCTION(KqlExtract)
{
factory.registerFunction<FunctionKqlExtract>();
}

}
5 changes: 3 additions & 2 deletions src/Parsers/Kusto/KQL_ReleaseNote.md
Original file line number Diff line number Diff line change
Expand Up @@ -67,9 +67,9 @@
- semi join flavor
ADX : only returns left side or right side columns
KQL-CH : returns columns from both side
- Join hints : not supported yet
- [lookup](https://learn.microsoft.com/en-us/azure/data-explorer/kusto/query/lookupoperator)
lookup is a sunset of join, only support : kind=leftouter and kind=inner. if kind unspecified, kind=leftouter
lookup is a subset of join, only support : kind=leftouter and kind=inner. if kind unspecified, kind=leftouter
```
DROP TABLE IF EXISTS FactTable;
CREATE TABLE FactTable (Row String, Personal String, Family String) ENGINE = Memory;
Expand All @@ -92,6 +92,7 @@
```
## Bugs fixed
- [Incorrect Regx convertion](https://github.ibm.com/ClickHouse/issue-repo/issues/1817)
- [KQL phase 2 - timespan calculation results in exception](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1129)
- [KQL phase 2 - format_timespan returns incorrect results](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1133)
- [Bin function should support time intervals less than 1 second](https://zenhub.ibm.com/workspaces/clickhouse-project-61250df53aaf060db4e08052/issues/clickhouse/issue-repo/1207)
Expand Down
44 changes: 2 additions & 42 deletions src/Parsers/Kusto/KustoFunctions/KQLStringFunctions.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,7 @@ bool Extract::convertImpl(String & out, IParser::Pos & pos)
String regex = getConvertedArgument(fn_name, pos);

++pos;
size_t capture_group = stoi(getConvertedArgument(fn_name, pos));
String capture_group = getConvertedArgument(fn_name, pos);

++pos;
String source = getConvertedArgument(fn_name, pos);
Expand Down Expand Up @@ -162,47 +162,7 @@ bool Extract::convertImpl(String & out, IParser::Pos & pos)
}
}

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 + ")";

out = std::format("kql_extract({}, {}, {})", source, regex, capture_group);
if (type_literal == "Decimal")
{
out = std::format("countSubstrings({0}, '.') > 1 ? NULL: {0}, length(substr({0}, position({0},'.') + 1)))", out);
Expand Down
14 changes: 7 additions & 7 deletions src/Parsers/tests/KQL/gtest_KQL_StringFunctions.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -132,31 +132,31 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery_String, ParserTest,
},
{
"print extract('x=([0-9.]+)', 1, 'hello x=456|wo' , typeof(bool));",
"SELECT accurateCastOrNull(toInt64OrNull(extract('hello x=456|wo', '[0-9.]+')), 'Boolean')"
"SELECT accurateCastOrNull(toInt64OrNull(kql_extract('hello x=456|wo', 'x=([0-9.]+)', 1)), 'Boolean')"
},
{
"print extract('x=([0-9.]+)', 1, 'hello x=456|wo' , typeof(date));",
"SELECT accurateCastOrNull(extract('hello x=456|wo', '[0-9.]+'), 'DateTime')"
"SELECT accurateCastOrNull(kql_extract('hello x=456|wo', 'x=([0-9.]+)', 1), 'DateTime')"
},
{
"print extract('x=([0-9.]+)', 1, 'hello x=456|wo' , typeof(guid));",
"SELECT accurateCastOrNull(extract('hello x=456|wo', '[0-9.]+'), 'UUID')"
"SELECT accurateCastOrNull(kql_extract('hello x=456|wo', 'x=([0-9.]+)', 1), 'UUID')"
},
{
"print extract('x=([0-9.]+)', 1, 'hello x=456|wo' , typeof(int));",
"SELECT accurateCastOrNull(extract('hello x=456|wo', '[0-9.]+'), 'Int32')"
"SELECT accurateCastOrNull(kql_extract('hello x=456|wo', 'x=([0-9.]+)', 1), 'Int32')"
},
{
"print extract('x=([0-9.]+)', 1, 'hello x=456|wo' , typeof(long));",
"SELECT accurateCastOrNull(extract('hello x=456|wo', '[0-9.]+'), 'Int64')"
"SELECT accurateCastOrNull(kql_extract('hello x=456|wo', 'x=([0-9.]+)', 1), 'Int64')"
},
{
"print extract('x=([0-9.]+)', 1, 'hello x=456|wo' , typeof(real));",
"SELECT accurateCastOrNull(extract('hello x=456|wo', '[0-9.]+'), 'Float64')"
"SELECT accurateCastOrNull(kql_extract('hello x=456|wo', 'x=([0-9.]+)', 1), 'Float64')"
},
{
"print extract('x=([0-9.]+)', 1, 'hello x=456|wo' , typeof(decimal));",
"SELECT toDecimal128OrNull(if(countSubstrings(extract('hello x=456|wo', '[0-9.]+'), '.') > 1, NULL, extract('hello x=456|wo', '[0-9.]+')), length(substr(extract('hello x=456|wo', '[0-9.]+'), position(extract('hello x=456|wo', '[0-9.]+'), '.') + 1)))"
"SELECT toDecimal128OrNull(if(countSubstrings(kql_extract('hello x=456|wo', 'x=([0-9.]+)', 1), '.') > 1, NULL, kql_extract('hello x=456|wo', 'x=([0-9.]+)', 1)), length(substr(kql_extract('hello x=456|wo', 'x=([0-9.]+)', 1), position(kql_extract('hello x=456|wo', 'x=([0-9.]+)', 1), '.') + 1)))"
},
{
"print parse_version('1.2.3.40')",
Expand Down
8 changes: 4 additions & 4 deletions src/Parsers/tests/gtest_Parser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -449,19 +449,19 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserTest,
},
{
"Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 0, 'The price of PINEAPPLE ice cream is 10')",
"SELECT extract('The price of PINEAPPLE ice cream is 10', '\\b[A-Z]+\\b.+\\b\\\\d+')\nFROM Customers"
"SELECT kql_extract('The price of PINEAPPLE ice cream is 10', '(\\b[A-Z]+\\b).+(\\b\\\\d+)', 0)\nFROM Customers"
},
{
"Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 1, 'The price of PINEAPPLE ice cream is 20')",
"SELECT extract('The price of PINEAPPLE ice cream is 20', '\\b[A-Z]+\\b')\nFROM Customers"
"SELECT kql_extract('The price of PINEAPPLE ice cream is 20', '(\\b[A-Z]+\\b).+(\\b\\\\d+)', 1)\nFROM Customers"
},
{
"Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 30')",
"SELECT extract('The price of PINEAPPLE ice cream is 30', '\\b\\\\d+')\nFROM Customers"
"SELECT kql_extract('The price of PINEAPPLE ice cream is 30', '(\\b[A-Z]+\\b).+(\\b\\\\d+)', 2)\nFROM Customers"
},
{
"Customers | project extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 40', typeof(int))",
"SELECT accurateCastOrNull(extract('The price of PINEAPPLE ice cream is 40', '\\b\\\\d+'), 'Int32')\nFROM Customers"
"SELECT accurateCastOrNull(kql_extract('The price of PINEAPPLE ice cream is 40', '(\\b[A-Z]+\\b).+(\\b\\\\d+)', 2), 'Int32')\nFROM Customers"
},
{
"Customers | project extract_all('(\\w)(\\w+)(\\w)','The price of PINEAPPLE ice cream is 50')",
Expand Down
1 change: 1 addition & 0 deletions tests/queries/0_stateless/02366_kql_func_string.reference
Original file line number Diff line number Diff line change
Expand Up @@ -244,6 +244,7 @@ PINEAPPLE
\N
45.6
45.6
alert

-- extract_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractallfunction); TODO: captureGroups not supported yet
[['T','h','e'],['p','ric','e'],['P','INEAPPL','E'],['i','c','e'],['c','rea','m']]
Expand Down
1 change: 1 addition & 0 deletions tests/queries/0_stateless/02366_kql_func_string.sql
Original file line number Diff line number Diff line change
Expand Up @@ -183,6 +183,7 @@ print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(int));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(long));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(real));
print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(decimal));
print extract(".*Action=(\\w+)",1, "dstPostNATPort=80 proto=tcp Action=alert");
print '';
print '-- extract_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractallfunction); TODO: captureGroups not supported yet';
Customers | project extract_all('(\\w)(\\w+)(\\w)','The price of PINEAPPLE ice cream is 20') | take 1;
Expand Down

0 comments on commit 8cf6010

Please sign in to comment.