Skip to content

Commit

Permalink
Kusto-phase3: rebase v24.2.1.2248-stable
Browse files Browse the repository at this point in the history
Kusto-phase3: rebase v24.2.1.2248-stable fixing tests.

Kusto-phase3: rebase v24.2.1.2248-stable fix has any ip functions.

Kusto-phase3: rebase v24.2.1.2248-stable provide additional protection on ip functions.
  • Loading branch information
kothiga committed Mar 12, 2024
1 parent 84dc585 commit 1c44848
Show file tree
Hide file tree
Showing 22 changed files with 84 additions and 80 deletions.
8 changes: 4 additions & 4 deletions src/Common/IntervalKind.h
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ struct IntervalKind
Float64 toSeconds() const;

/// Chooses an interval kind based on number of seconds.
/// For example, `IntervalKind::fromAvgSeconds(3600)` returns `IntervalKind::Hour`.
/// For example, `IntervalKind::fromAvgSeconds(3600)` returns `IntervalKind::Kind::Hour`.
static IntervalKind fromAvgSeconds(Int64 num_seconds);

/// Returns whether IntervalKind has a fixed number of seconds (e.g. Day) or non-fixed(e.g. Month)
Expand All @@ -54,16 +54,16 @@ struct IntervalKind
const char * toLowercasedKeyword() const;

/// Returns the string which can be passed to the `unit` parameter of the dateDiff() function.
/// For example, `IntervalKind{IntervalKind::Day}.getDateDiffParameter()` returns "day".
/// For example, `IntervalKind{IntervalKind::Kind::Day}.getDateDiffParameter()` returns "day".
const char * toDateDiffUnit() const;

/// Returns the name of the function converting a number to the interval data type.
/// For example, `IntervalKind{IntervalKind::Day}.getToIntervalDataTypeFunctionName()`
/// For example, `IntervalKind{IntervalKind::Kind::Day}.getToIntervalDataTypeFunctionName()`
/// returns "toIntervalDay".
const char * toNameOfFunctionToIntervalDataType() const;

/// Returns the name of the function extracting time part from a date or a time.
/// For example, `IntervalKind{IntervalKind::Day}.getExtractTimePartFunctionName()`
/// For example, `IntervalKind{IntervalKind::Kind::Day}.getExtractTimePartFunctionName()`
/// returns "toDayOfMonth".
const char * toNameOfFunctionExtractTimePart() const;

Expand Down
4 changes: 3 additions & 1 deletion src/Core/SettingsChangesHistory.h
Original file line number Diff line number Diff line change
Expand Up @@ -141,7 +141,9 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."},
{"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"},
{"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"},
{"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}},
{"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"},
{"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"},
{"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}},
{"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"},
{"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"},
{"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"},
Expand Down
46 changes: 23 additions & 23 deletions src/Functions/FunctionBinaryArithmetic.h
Original file line number Diff line number Diff line change
Expand Up @@ -1723,29 +1723,7 @@ class FunctionBinaryArithmetic : public IFunction

if constexpr (!std::is_same_v<ResultDataType, InvalidType>)
{
if constexpr (is_div_int || is_div_int_or_zero)
type_res = std::make_shared<ResultDataType>();
else if constexpr (IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType>)
{
if constexpr (is_division)
{
if (context->getSettingsRef().decimal_check_overflow)
{
/// Check overflow by using operands scale (based on big decimal division implementation details):
/// big decimal arithmetic is based on big integers, decimal operands are converted to big integers
/// i.e. int_operand = decimal_operand*10^scale
/// For division, left operand will be scaled by right operand scale also to do big integer division,
/// BigInt result = left*10^(left_scale + right_scale) / right * 10^right_scale
/// So, we can check upfront possible overflow just by checking max scale used for left operand
/// Note: it doesn't detect all possible overflow during big decimal division
if (left.getScale() + right.getScale() > ResultDataType::maxPrecision())
throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Overflow during decimal division");
}
}
ResultDataType result_type = decimalResultType<is_multiply, is_division>(left, right);
type_res = std::make_shared<ResultDataType>(result_type.getPrecision(), result_type.getScale());
}
else if constexpr (((IsDataTypeDecimal<LeftDataType> && IsFloatingPoint<RightDataType>) ||
if constexpr (((IsDataTypeDecimal<LeftDataType> && IsFloatingPoint<RightDataType>) ||
(IsDataTypeDecimal<RightDataType> && IsFloatingPoint<LeftDataType>)))
{
type_res = std::make_shared<DataTypeFloat64>();
Expand Down Expand Up @@ -1807,6 +1785,28 @@ class FunctionBinaryArithmetic : public IFunction

return static_cast<bool>(nested_type);
}
else if constexpr (IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType>)
{
if constexpr (is_division)
{
if (context->getSettingsRef().decimal_check_overflow)
{
/// Check overflow by using operands scale (based on big decimal division implementation details):
/// big decimal arithmetic is based on big integers, decimal operands are converted to big integers
/// i.e. int_operand = decimal_operand*10^scale
/// For division, left operand will be scaled by right operand scale also to do big integer division,
/// BigInt result = left*10^(left_scale + right_scale) / right * 10^right_scale
/// So, we can check upfront possible overflow just by checking max scale used for left operand
/// Note: it doesn't detect all possible overflow during big decimal division
if (left.getScale() + right.getScale() > ResultDataType::maxPrecision())
throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Overflow during decimal division");
}
}
ResultDataType result_type = decimalResultType<is_multiply, is_division>(left, right);
type_res = std::make_shared<ResultDataType>(result_type.getPrecision(), result_type.getScale());
}
else if constexpr (is_div_int || is_div_int_or_zero)
type_res = std::make_shared<ResultDataType>();
else
type_res = std::make_shared<ResultDataType>();
return true;
Expand Down
2 changes: 1 addition & 1 deletion src/Functions/FunctionsConversion.h
Original file line number Diff line number Diff line change
Expand Up @@ -3316,7 +3316,7 @@ class FunctionCast final : public FunctionCastBase
switch (interval_kind)
{
#define DECLARE_CASE(NAME) \
case IntervalKind::NAME: \
case IntervalKind::Kind::NAME: \
return FunctionToInterval##NAME::create();
FOR_EACH_INTERVAL_KIND(DECLARE_CASE)
#undef DECLARE_CASE
Expand Down
2 changes: 1 addition & 1 deletion src/Functions/Kusto/KqlRange.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -809,7 +809,7 @@ class FunctionKqlRange : public IFunction
total_values = total_elements;
}

auto out = ColumnArray::create(std::make_shared<DataTypeInterval>(IntervalKind::Nanosecond)->createColumn());
auto out = ColumnArray::create(std::make_shared<DataTypeInterval>(IntervalKind::Kind::Nanosecond)->createColumn());
IColumn & out_data = out->getData();
IColumn::Offsets & out_offsets = out->getOffsets();

Expand Down
4 changes: 2 additions & 2 deletions src/Functions/Kusto/kqlDateTime.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -95,11 +95,11 @@ ColumnPtr FunctionKqlDateTime<input_policy>::executeImpl(

const ColumnsWithTypeAndName addition_args{
asArgument(converted, "converted"),
createConstColumnWithTypeAndName<DataTypeInterval>(50, "interval_50", IntervalKind::Nanosecond)};
createConstColumnWithTypeAndName<DataTypeInterval>(50, "interval_50", IntervalKind::Kind::Nanosecond)};
const auto sum = executeFunctionCall(context, "plus", addition_args, input_rows_count);

const ColumnsWithTypeAndName to_start_of_interval_args{
asArgument(sum, "sum"), createConstColumnWithTypeAndName<DataTypeInterval>(100, "interval_100", IntervalKind::Nanosecond)};
asArgument(sum, "sum"), createConstColumnWithTypeAndName<DataTypeInterval>(100, "interval_100", IntervalKind::Kind::Nanosecond)};
const auto [rounded_column, _] = executeFunctionCall(context, "toStartOfInterval", to_start_of_interval_args, input_rows_count);

return wrapInNullable(rounded_column, conversion_args, result_type, input_rows_count);
Expand Down
21 changes: 15 additions & 6 deletions src/Functions/Kusto/kqlHasAnyIp.cpp
Original file line number Diff line number Diff line change
@@ -1,9 +1,9 @@
#include <optional>
#include <ranges>
#include <regex>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/OptimizedRegularExpression.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
Expand Down Expand Up @@ -397,17 +397,26 @@ class FunctionKqlHasIpGeneric : public IFunction
const auto ips = extractIpsFromArguments(arguments, result_type, context, i);

std::string source = arguments[0].column->getDataAt(i).toString();
const std::regex ip_finder(Func::regex);
std::smatch matches;

while (!res && std::regex_search(source, matches, ip_finder))
OptimizedRegularExpression ip_finder(Func::regex);
OptimizedRegularExpression::MatchVec matches;

unsigned num_matches;
while (!res && (num_matches = ip_finder.match(source, matches)))
{
res = Func::checkRegexMatch(matches[2].str(), result_type, context, ips);
if (num_matches <= 3)
continue;

This comment has been minimized.

Copy link
@kashwy

kashwy Mar 13, 2024

is it possible that num_matches always less than 3?

This comment has been minimized.

Copy link
@kothiga

kothiga Mar 13, 2024

Author Member

I think that it is based off of the regular expression used. More specifically, it is based on the number of sub patterns found by re2 when the finder is constructed.

For this case, it should always be equal to 4 based off of the regexs defined for ip4 and ipv6. I did, however, opt to check the range prior to use for protection, so that in the event of something changing we don't accidentally segfault.

This comment has been minimized.

Copy link
@kashwy

kashwy Mar 13, 2024

if an invalid input, will it still get 4 ?

This comment has been minimized.

Copy link
@kothiga

kothiga Mar 13, 2024

Author Member

No. If an unexpected input for Func::regex (which is just defined as a std::string containing some regex pattern) is provided to the constructor, the number of sub patterns found could be less than 4.

source = matches.suffix().str();
res = Func::checkRegexMatch(source.substr(matches[2].offset, matches[2].length), result_type, context, ips);

if (matches[3].length == 0)
source = "";
else
source = source.substr(matches[3].offset, source.length());
}
result_column.push_back(static_cast<UInt8>(res));
}

return result;
}

Expand Down
10 changes: 1 addition & 9 deletions src/Functions/Kusto/kqlIndexOfRegex.cpp
Original file line number Diff line number Diff line change
@@ -1,21 +1,13 @@
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Common/re2.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>

#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif

namespace DB::ErrorCodes
{
extern const int CANNOT_COMPILE_REGEXP;
Expand Down
2 changes: 1 addition & 1 deletion src/Functions/Kusto/kqlToTimespan.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ ColumnPtr FunctionKqlToTimespan::executeImpl(

DataTypePtr FunctionKqlToTimespan::getReturnTypeImpl(const DataTypes &) const
{
return makeNullable(std::make_shared<DataTypeInterval>(IntervalKind::Nanosecond));
return makeNullable(std::make_shared<DataTypeInterval>(IntervalKind::Kind::Nanosecond));
}

REGISTER_FUNCTION(KqlToTimespan)
Expand Down
2 changes: 1 addition & 1 deletion src/Functions/dateTime64Diff.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ DataTypePtr FunctionDateTime64Diff::getReturnTypeImpl(const DataTypes & argument
rhs->getName(),
getName());

return std::make_shared<DataTypeInterval>(IntervalKind::Nanosecond);
return std::make_shared<DataTypeInterval>(IntervalKind::Kind::Nanosecond);
}

REGISTER_FUNCTION(DateTime64Diff)
Expand Down
1 change: 1 addition & 0 deletions src/Interpreters/Context.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
#include <Common/thread_local_rng.h>
#include <Common/FieldVisitorToString.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/callOnce.h>
#include <Common/SharedLockGuard.h>
#include <Coordination/KeeperDispatcher.h>
Expand Down
4 changes: 2 additions & 2 deletions src/Parsers/ExpressionElementParsers.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -124,7 +124,7 @@ bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "EXPLAIN in a subquery cannot have a table function or table override");

/// Replace subquery `(EXPLAIN <kind> <explain_settings> SELECT ...)`
/// with `(SELECT * FROM viewExplain("<kind>", "<explain_settings>", SELECT ...))`
/// with `(SELECT * FROM viewExplain('<kind>', '<explain_settings>', (SELECT ...)))`

String kind_str = ASTExplainQuery::toString(explain_query.getKind());

Expand All @@ -142,7 +142,7 @@ bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
auto view_explain = makeASTFunction("viewExplain",
std::make_shared<ASTLiteral>(kind_str),
std::make_shared<ASTLiteral>(settings_str),
explained_ast);
std::make_shared<ASTSubquery>(explained_ast));
result_node = buildSelectFromTableFunction(view_explain);
}
else
Expand Down
2 changes: 1 addition & 1 deletion src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -364,7 +364,7 @@ String IParserKQLFunction::getExpression(IParser::Pos & pos)
arg = kqlTicksToInterval(ticks);
}
}
else if ((pos->type == TokenType::QuotedIdentifier))
else if (pos->type == TokenType::QuotedIdentifier)
arg = "'" + escapeSingleQuotes(String(pos->begin + 1, pos->end - 1)) + "'";
else if (pos->type == TokenType::At)
{
Expand Down
3 changes: 1 addition & 2 deletions src/Parsers/Kusto/KustoFunctions/KQLDataTypeFunctions.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -5,11 +5,10 @@

#include <boost/lexical_cast/try_lexical_convert.hpp>
#include <Poco/String.h>
#include <Common/re2.h>

#include <format>
#include <regex>
#include <unordered_set>
#include <iostream>

namespace DB::ErrorCodes
{
Expand Down
2 changes: 1 addition & 1 deletion src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.cpp
Original file line number Diff line number Diff line change
@@ -1,12 +1,12 @@
#include "KQLDateTimeFunctions.h"

#include <Parsers/Kusto/Utilities.h>
#include <Common/StringUtils/StringUtils.h>

#include <Poco/String.h>

#include <format>
#include <optional>
#include <regex>
#include <unordered_set>

namespace
Expand Down
16 changes: 8 additions & 8 deletions src/Parsers/Kusto/ParserKQLOperators.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -200,7 +200,7 @@ String genHasAnyAllOpExpr(
auto haystack = tokens.back();

const auto * const logic_op = (kql_op == "has_all") ? " and " : " or ";
while (isValidKQLPos(token_pos) && token_pos->type != TokenType::PipeMark && token_pos->type != TokenType::Semicolon)
while (isValidKQLPos(token_pos) && token_pos->type != DB::TokenType::PipeMark && token_pos->type != DB::TokenType::Semicolon)
{
auto tmp_arg = DB::IParserKQLFunction::getExpression(token_pos);
if (token_pos->type == DB::TokenType::Comma)
Expand All @@ -217,14 +217,14 @@ String genHasAnyAllOpExpr(
return new_expr;
}

String genEqOpExprCis(std::vector<String> & tokens, DB::IParser::Pos & token_pos, const DB::String & ch_op)
String genEqOpExprCis(std::vector<String> & tokens, DB::IParser::Pos & token_pos, const String & ch_op)
{
DB::String tmp_arg(token_pos->begin, token_pos->end);
String tmp_arg(token_pos->begin, token_pos->end);

if (tokens.empty() || tmp_arg != "~")
return tmp_arg;

DB::String new_expr;
String new_expr;
new_expr += "lower(" + tokens.back() + ")" + " ";
new_expr += ch_op + " ";
++token_pos;
Expand All @@ -240,7 +240,7 @@ String genEqOpExprCis(std::vector<String> & tokens, DB::IParser::Pos & token_pos

String genBetweenOpExpr(std::vector<std::string> & tokens, DB::IParser::Pos & token_pos, const String & ch_op)
{
DB::String new_expr;
String new_expr;
new_expr += ch_op + "(";
new_expr += tokens.back() + ",";
tokens.pop_back();
Expand Down Expand Up @@ -289,15 +289,15 @@ String genBetweenOpExpr(std::vector<std::string> & tokens, DB::IParser::Pos & to
return new_expr;
}

String genInOpExprCis(std::vector<String> & tokens, DB::IParser::Pos & token_pos, const DB::String & kql_op, const DB::String & ch_op)
String genInOpExprCis(std::vector<String> & tokens, DB::IParser::Pos & token_pos, const String & kql_op, const String & ch_op)
{
DB::KQLContext kql_context;
DB::ParserKQLTableFunction kqlfun_p(kql_context);
DB::ParserToken s_lparen(DB::TokenType::OpeningRoundBracket);

DB::ASTPtr select;
DB::Expected expected;
DB::String new_expr;
String new_expr;

++token_pos;
if (!s_lparen.ignore(token_pos, expected))
Expand All @@ -323,7 +323,7 @@ String genInOpExprCis(std::vector<String> & tokens, DB::IParser::Pos & token_pos
new_expr += ch_op;
while (isValidKQLPos(token_pos) && token_pos->type != DB::TokenType::PipeMark && token_pos->type != DB::TokenType::Semicolon)
{
auto tmp_arg = DB::String(token_pos->begin, token_pos->end);
auto tmp_arg = 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")
Expand Down
4 changes: 4 additions & 0 deletions src/Parsers/Kusto/ParserKQLQuery.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -503,14 +503,18 @@ bool ParserKQLQuery::getOperations(Pos & pos, Expected & expected, OperationsPos

if ((kql_operator == "print" || kql_operator == "range") && !operation_pos.empty())
throw Exception(ErrorCodes::SYNTAX_ERROR, "{} must be the first operator in the query", kql_operator);

operation_pos.emplace_back(kql_operator, pos);
}
else
++pos;
}
return true;
}

bool ParserKQLQuery::pre_process(String & source, Pos & pos)
{
bool need_preprocess = false;
auto begin = pos;
while (isValidKQLPos(pos) && pos->type != TokenType::Semicolon)
{
Expand Down
20 changes: 11 additions & 9 deletions src/Storages/System/StorageSystemSchema.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -24,15 +24,6 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}

NamesAndTypesList StorageSystemSchema::getNamesAndTypes()
{
return {
{"ColumnName", std::make_shared<DataTypeString>()},
{"ColumnOrdinal", std::make_shared<DataTypeInt32>()},
{"DataType", std::make_shared<DataTypeString>()},
{"ColumnType", std::make_shared<DataTypeString>()}};
}

void StorageSystemSchema::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
if (res_columns.size() != 4)
Expand Down Expand Up @@ -60,4 +51,15 @@ void StorageSystemSchema::fillData(MutableColumns & res_columns, ContextPtr cont
column_types->insert(toField(is_kql ? toKQLDataTypeName(type) : type_name));
}
}

ColumnsDescription StorageSystemSchema::getColumnsDescription()
{
return ColumnsDescription
{
{"ColumnName", std::make_shared<DataTypeString>(), "Name of the column."},
{"ColumnOrdinal", std::make_shared<DataTypeInt32>(), "Ordinal of the column."},
{"DataType", std::make_shared<DataTypeString>(), "Data type of the column."},
{"ColumnType", std::make_shared<DataTypeString>(), "Type of the column."}
};
}
}
Loading

0 comments on commit 1c44848

Please sign in to comment.