Merge pull request #3826 from yandex/missing-sql-compatibility-functions

Additional functions for SQL compatibility
This commit is contained in:
alexey-milovidov 2018-12-20 20:51:05 +03:00 committed by GitHub
commit f85857d8ff
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
30 changed files with 1115 additions and 39 deletions

View File

@ -19,6 +19,7 @@ void registerDataTypeInterval(DataTypeFactory & factory)
factory.registerSimpleDataType("IntervalDay", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(DataTypeInterval::Day)); });
factory.registerSimpleDataType("IntervalWeek", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(DataTypeInterval::Week)); });
factory.registerSimpleDataType("IntervalMonth", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(DataTypeInterval::Month)); });
factory.registerSimpleDataType("IntervalQuarter", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(DataTypeInterval::Quarter)); });
factory.registerSimpleDataType("IntervalYear", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(DataTypeInterval::Year)); });
}

View File

@ -25,6 +25,7 @@ public:
Day,
Week,
Month,
Quarter,
Year
};
@ -46,6 +47,7 @@ public:
case Day: return "Day";
case Week: return "Week";
case Month: return "Month";
case Quarter: return "Quarter";
case Year: return "Year";
default: __builtin_unreachable();
}

View File

@ -113,6 +113,21 @@ struct AddMonthsImpl
}
};
struct AddQuartersImpl
{
static constexpr auto name = "addQuarters";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.addQuarters(t, delta);
}
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.addQuarters(DayNum(d), delta);
}
};
struct AddYearsImpl
{
static constexpr auto name = "addYears";
@ -149,6 +164,7 @@ struct SubtractHoursImpl : SubtractIntervalImpl<AddHoursImpl> { static constexpr
struct SubtractDaysImpl : SubtractIntervalImpl<AddDaysImpl> { static constexpr auto name = "subtractDays"; };
struct SubtractWeeksImpl : SubtractIntervalImpl<AddWeeksImpl> { static constexpr auto name = "subtractWeeks"; };
struct SubtractMonthsImpl : SubtractIntervalImpl<AddMonthsImpl> { static constexpr auto name = "subtractMonths"; };
struct SubtractQuartersImpl : SubtractIntervalImpl<AddQuartersImpl> { static constexpr auto name = "subtractQuarters"; };
struct SubtractYearsImpl : SubtractIntervalImpl<AddYearsImpl> { static constexpr auto name = "subtractYears"; };

View File

@ -89,6 +89,7 @@ void registerFunctionsConversion(FunctionFactory & factory)
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalDay, PositiveMonotonicity>>();
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalWeek, PositiveMonotonicity>>();
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalMonth, PositiveMonotonicity>>();
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalQuarter, PositiveMonotonicity>>();
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalYear, PositiveMonotonicity>>();
}

View File

@ -738,6 +738,7 @@ DEFINE_NAME_TO_INTERVAL(Hour)
DEFINE_NAME_TO_INTERVAL(Day)
DEFINE_NAME_TO_INTERVAL(Week)
DEFINE_NAME_TO_INTERVAL(Month)
DEFINE_NAME_TO_INTERVAL(Quarter)
DEFINE_NAME_TO_INTERVAL(Year)
#undef DEFINE_NAME_TO_INTERVAL

View File

@ -1080,7 +1080,7 @@ void registerFunctionsStringSearch(FunctionFactory & factory)
factory.registerFunction<FunctionReplaceAll>();
factory.registerFunction<FunctionReplaceRegexpOne>();
factory.registerFunction<FunctionReplaceRegexpAll>();
factory.registerFunction<FunctionPosition>();
factory.registerFunction<FunctionPosition>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionPositionUTF8>();
factory.registerFunction<FunctionPositionCaseInsensitive>();
factory.registerFunction<FunctionPositionCaseInsensitiveUTF8>();

View File

@ -48,7 +48,7 @@ template <> struct FunctionUnaryArithmeticMonotonicity<NameAbs>
void registerFunctionAbs(FunctionFactory & factory)
{
factory.registerFunction<FunctionAbs>();
factory.registerFunction<FunctionAbs>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,18 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionDateOrDateTimeAddInterval.h>
namespace DB
{
using FunctionAddQuarters = FunctionDateOrDateTimeAddInterval<AddQuartersImpl>;
void registerFunctionAddQuarters(FunctionFactory & factory)
{
factory.registerFunction<FunctionAddQuarters>();
}
}

View File

@ -9,7 +9,7 @@ using FunctionRand = FunctionRandom<UInt32, NameRand>;
void registerFunctionRand(FunctionFactory & factory)
{
factory.registerFunction<FunctionRand>();
factory.registerFunction<FunctionRand>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,119 @@
#include <Common/config.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/GatherUtils/Algorithms.h>
#include <IO/WriteHelpers.h>
#include <re2/re2.h>
#include <re2/stringpiece.h>
#if USE_RE2_ST
#include <re2_st/re2.h> // Y_IGNORE
#else
#define re2_st re2
#endif
namespace DB
{
using namespace GatherUtils;
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
class FunctionRegexpQuoteMeta : public IFunction
{
public:
static constexpr auto name = "regexpQuoteMeta";
static FunctionPtr create(const Context &)
{
return std::make_shared<FunctionRegexpQuoteMeta>();
}
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override
{
return 1;
}
bool useDefaultImplementationForConstants() const override
{
return true;
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (!WhichDataType(arguments[0].type).isString())
throw Exception(
"Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + ". Must be String.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const ColumnPtr column_string = block.getByPosition(arguments[0]).column;
const ColumnString * input = checkAndGetColumn<ColumnString>(column_string.get());
if (!input)
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
auto dst_column = ColumnString::create();
auto & dst_data = dst_column->getChars();
auto & dst_offsets = dst_column->getOffsets();
dst_data.resize(input->getChars().size() * input->size());
dst_offsets.resize(input_rows_count);
const ColumnString::Offsets & src_offsets = input->getOffsets();
auto source = reinterpret_cast<const char *>(input->getChars().data());
auto dst = reinterpret_cast<char *>(dst_data.data());
auto dst_pos = dst;
size_t src_offset_prev = 0;
for (size_t row = 0; row < input_rows_count; ++row)
{
size_t srclen = src_offsets[row] - src_offset_prev - 1;
/// suboptimal, but uses original implementation from re2
re2_st::StringPiece unquoted(source, srclen);
const auto & quoted = re2_st::RE2::QuoteMeta(unquoted);
const auto size = quoted.size();
std::memcpy(dst_pos, quoted.data(), size);
source += srclen + 1;
dst_pos[size] = '\0';
dst_pos += size + 1;
dst_offsets[row] = dst_pos - dst;
src_offset_prev = src_offsets[row];
}
dst_data.resize(dst_pos - dst);
block.getByPosition(result).column = std::move(dst_column);
}
};
void registerFunctionRegexpQuoteMeta(FunctionFactory & factory)
{
factory.registerFunction<FunctionRegexpQuoteMeta>();
}
}

View File

@ -47,6 +47,7 @@ void registerFunctionAddHours(FunctionFactory &);
void registerFunctionAddDays(FunctionFactory &);
void registerFunctionAddWeeks(FunctionFactory &);
void registerFunctionAddMonths(FunctionFactory &);
void registerFunctionAddQuarters(FunctionFactory &);
void registerFunctionAddYears(FunctionFactory &);
void registerFunctionSubtractSeconds(FunctionFactory &);
void registerFunctionSubtractMinutes(FunctionFactory &);
@ -54,6 +55,7 @@ void registerFunctionSubtractHours(FunctionFactory &);
void registerFunctionSubtractDays(FunctionFactory &);
void registerFunctionSubtractWeeks(FunctionFactory &);
void registerFunctionSubtractMonths(FunctionFactory &);
void registerFunctionSubtractQuarters(FunctionFactory &);
void registerFunctionSubtractYears(FunctionFactory &);
void registerFunctionDateDiff(FunctionFactory &);
void registerFunctionToTimeZone(FunctionFactory &);
@ -106,6 +108,7 @@ void registerFunctionsDateTime(FunctionFactory & factory)
registerFunctionAddDays(factory);
registerFunctionAddWeeks(factory);
registerFunctionAddMonths(factory);
registerFunctionAddQuarters(factory);
registerFunctionAddYears(factory);
registerFunctionSubtractSeconds(factory);
registerFunctionSubtractMinutes(factory);
@ -113,6 +116,7 @@ void registerFunctionsDateTime(FunctionFactory & factory)
registerFunctionSubtractDays(factory);
registerFunctionSubtractWeeks(factory);
registerFunctionSubtractMonths(factory);
registerFunctionSubtractQuarters(factory);
registerFunctionSubtractYears(factory);
registerFunctionDateDiff(factory);
registerFunctionToTimeZone(factory);

View File

@ -21,6 +21,8 @@ void registerFunctionSubstringUTF8(FunctionFactory &);
void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory &);
void registerFunctionStartsWith(FunctionFactory &);
void registerFunctionEndsWith(FunctionFactory &);
void registerFunctionTrim(FunctionFactory &);
void registerFunctionRegexpQuoteMeta(FunctionFactory &);
#if USE_BASE64
void registerFunctionBase64Encode(FunctionFactory &);
@ -46,6 +48,8 @@ void registerFunctionsString(FunctionFactory & factory)
registerFunctionAppendTrailingCharIfAbsent(factory);
registerFunctionStartsWith(factory);
registerFunctionEndsWith(factory);
registerFunctionTrim(factory);
registerFunctionRegexpQuoteMeta(factory);
#if USE_BASE64
registerFunctionBase64Encode(factory);
registerFunctionBase64Decode(factory);

View File

@ -147,7 +147,7 @@ private:
void registerFunctionReverse(FunctionFactory & factory)
{
factory.registerFunction<FunctionBuilderReverse>();
factory.registerFunction<FunctionBuilderReverse>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,18 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionDateOrDateTimeAddInterval.h>
namespace DB
{
using FunctionSubtractQuarters = FunctionDateOrDateTimeAddInterval<SubtractQuartersImpl>;
void registerFunctionSubtractQuarters(FunctionFactory & factory)
{
factory.registerFunction<FunctionSubtractQuarters>();
}
}

142
dbms/src/Functions/trim.cpp Normal file
View File

@ -0,0 +1,142 @@
#include <Columns/ColumnString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionStringToString.h>
#if __SSE4_2__
#include <nmmintrin.h>
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
struct TrimModeLeft
{
static constexpr auto name = "trimLeft";
static constexpr bool trim_left = true;
static constexpr bool trim_right = false;
};
struct TrimModeRight
{
static constexpr auto name = "trimRight";
static constexpr bool trim_left = false;
static constexpr bool trim_right = true;
};
struct TrimModeBoth
{
static constexpr auto name = "trimBoth";
static constexpr bool trim_left = true;
static constexpr bool trim_right = true;
};
template <typename mode>
class FunctionTrimImpl
{
public:
static void vector(
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
size_t size = offsets.size();
res_offsets.resize(size);
res_data.reserve(data.size());
size_t prev_offset = 0;
size_t res_offset = 0;
const UInt8 * start;
size_t length;
for (size_t i = 0; i < size; ++i)
{
execute(reinterpret_cast<const UInt8 *>(&data[prev_offset]), offsets[i] - prev_offset - 1, start, length);
res_data.resize(res_data.size() + length + 1);
std::memcpy(&res_data[res_offset], start, length);
res_offset += length + 1;
res_data[res_offset - 1] = '\0';
res_offsets[i] = res_offset;
prev_offset = offsets[i];
}
}
static void vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
{
throw Exception("Functions trimLeft, trimRight and trimBoth cannot work with FixedString argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
private:
static void execute(const UInt8 * data, size_t size, const UInt8 *& res_data, size_t & res_size)
{
size_t chars_to_trim_left = 0;
size_t chars_to_trim_right = 0;
char whitespace = ' ';
#if __SSE4_2__
const auto bytes_sse = sizeof(__m128i);
const auto size_sse = size - (size % bytes_sse);
const auto whitespace_mask = _mm_set1_epi8(whitespace);
constexpr auto base_sse_mode = _SIDD_UBYTE_OPS | _SIDD_CMP_EQUAL_EACH | _SIDD_NEGATIVE_POLARITY;
auto mask = bytes_sse;
#endif
if constexpr (mode::trim_left)
{
#if __SSE4_2__
/// skip whitespace from left in blocks of up to 16 characters
constexpr auto left_sse_mode = base_sse_mode | _SIDD_LEAST_SIGNIFICANT;
while (mask == bytes_sse && chars_to_trim_left < size_sse)
{
const auto chars = _mm_loadu_si128(reinterpret_cast<const __m128i *>(data + chars_to_trim_left));
mask = _mm_cmpistri(whitespace_mask, chars, left_sse_mode);
chars_to_trim_left += mask;
}
#endif
/// skip remaining whitespace from left, character by character
while (chars_to_trim_left < size && data[chars_to_trim_left] == whitespace)
++chars_to_trim_left;
}
if constexpr (mode::trim_right)
{
constexpr auto right_sse_mode = base_sse_mode | _SIDD_MOST_SIGNIFICANT;
const auto trim_right_size = size - chars_to_trim_left;
#if __SSE4_2__
/// try to skip whitespace from right in blocks of up to 16 characters
const auto trim_right_size_sse = trim_right_size - (trim_right_size % bytes_sse);
while (mask == bytes_sse && chars_to_trim_right < trim_right_size_sse)
{
const auto chars = _mm_loadu_si128(reinterpret_cast<const __m128i *>(data + size - chars_to_trim_right - bytes_sse));
mask = _mm_cmpistri(whitespace_mask, chars, right_sse_mode);
chars_to_trim_right += mask;
}
#endif
/// skip remaining whitespace from right, character by character
while (chars_to_trim_right < trim_right_size && data[size - chars_to_trim_right - 1] == whitespace)
++chars_to_trim_right;
}
res_data = data + chars_to_trim_left;
res_size = size - chars_to_trim_left - chars_to_trim_right;
}
};
using FunctionTrimLeft = FunctionStringToString<FunctionTrimImpl<TrimModeLeft>, TrimModeLeft>;
using FunctionTrimRight = FunctionStringToString<FunctionTrimImpl<TrimModeRight>, TrimModeRight>;
using FunctionTrimBoth = FunctionStringToString<FunctionTrimImpl<TrimModeBoth>, TrimModeBoth>;
void registerFunctionTrim(FunctionFactory & factory)
{
factory.registerFunction<FunctionTrimLeft>();
factory.registerFunction<FunctionTrimRight>();
factory.registerFunction<FunctionTrimBoth>();
}
}

View File

@ -46,4 +46,98 @@ protected:
}
};
class ParserInterval: public IParserBase
{
public:
enum class IntervalKind
{
Incorrect,
Second,
Minute,
Hour,
Day,
Week,
Month,
Quarter,
Year
};
IntervalKind interval_kind;
ParserInterval() : interval_kind(IntervalKind::Incorrect) {}
const char * getToIntervalKindFunctionName()
{
switch (interval_kind)
{
case ParserInterval::IntervalKind::Second:
return "toIntervalSecond";
case ParserInterval::IntervalKind::Minute:
return "toIntervalMinute";
case ParserInterval::IntervalKind::Hour:
return "toIntervalHour";
case ParserInterval::IntervalKind::Day:
return "toIntervalDay";
case ParserInterval::IntervalKind::Week:
return "toIntervalWeek";
case ParserInterval::IntervalKind::Month:
return "toIntervalMonth";
case ParserInterval::IntervalKind::Quarter:
return "toIntervalQuarter";
case ParserInterval::IntervalKind::Year:
return "toIntervalYear";
default:
return nullptr;
}
}
protected:
const char * getName() const override { return "interval"; }
bool parseImpl(Pos & pos, ASTPtr & /*node*/, Expected & expected) override
{
if (ParserKeyword("SECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_SECOND").ignore(pos, expected)
|| ParserKeyword("SS").ignore(pos, expected) || ParserKeyword("S").ignore(pos, expected))
interval_kind = IntervalKind::Second;
else if (
ParserKeyword("MINUTE").ignore(pos, expected) || ParserKeyword("SQL_TSI_MINUTE").ignore(pos, expected)
|| ParserKeyword("MI").ignore(pos, expected) || ParserKeyword("N").ignore(pos, expected))
interval_kind = IntervalKind::Minute;
else if (
ParserKeyword("HOUR").ignore(pos, expected) || ParserKeyword("SQL_TSI_HOUR").ignore(pos, expected)
|| ParserKeyword("HH").ignore(pos, expected))
interval_kind = IntervalKind::Hour;
else if (
ParserKeyword("DAY").ignore(pos, expected) || ParserKeyword("SQL_TSI_DAY").ignore(pos, expected)
|| ParserKeyword("DD").ignore(pos, expected) || ParserKeyword("D").ignore(pos, expected))
interval_kind = IntervalKind::Day;
else if (
ParserKeyword("WEEK").ignore(pos, expected) || ParserKeyword("SQL_TSI_WEEK").ignore(pos, expected)
|| ParserKeyword("WK").ignore(pos, expected) || ParserKeyword("WW").ignore(pos, expected))
interval_kind = IntervalKind::Week;
else if (
ParserKeyword("MONTH").ignore(pos, expected) || ParserKeyword("SQL_TSI_MONTH").ignore(pos, expected)
|| ParserKeyword("MM").ignore(pos, expected) || ParserKeyword("M").ignore(pos, expected))
interval_kind = IntervalKind::Month;
else if (
ParserKeyword("QUARTER").ignore(pos, expected) || ParserKeyword("SQL_TSI_QUARTER").ignore(pos, expected)
|| ParserKeyword("QQ").ignore(pos, expected) || ParserKeyword("Q").ignore(pos, expected))
interval_kind = IntervalKind::Quarter;
else if (
ParserKeyword("YEAR").ignore(pos, expected) || ParserKeyword("SQL_TSI_YEAR").ignore(pos, expected)
|| ParserKeyword("YYYY").ignore(pos, expected) || ParserKeyword("YY").ignore(pos, expected))
interval_kind = IntervalKind::Year;
else
interval_kind = IntervalKind::Incorrect;
if (interval_kind == IntervalKind::Incorrect)
{
expected.add(pos, "YEAR, QUARTER, MONTH, WEEK, DAY, HOUR, MINUTE or SECOND");
return false;
}
/// one of ParserKeyword already made ++pos
return true;
}
};
}

View File

@ -388,6 +388,255 @@ bool ParserSubstringExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & e
return true;
}
bool ParserTrimExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
/// Handles all possible TRIM/LTRIM/RTRIM call variants
std::string func_name;
bool trim_left = false;
bool trim_right = false;
bool char_override = false;
ASTPtr expr_node;
ASTPtr pattern_node;
ASTPtr to_remove;
if (ParserKeyword("LTRIM").ignore(pos, expected))
{
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
trim_left = true;
}
else if (ParserKeyword("RTRIM").ignore(pos, expected))
{
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
trim_right = true;
}
else if (ParserKeyword("TRIM").ignore(pos, expected))
{
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
if (ParserKeyword("BOTH").ignore(pos, expected))
{
trim_left = true;
trim_right = true;
char_override = true;
}
else if (ParserKeyword("LEADING").ignore(pos, expected))
{
trim_left = true;
char_override = true;
}
else if (ParserKeyword("TRAILING").ignore(pos, expected))
{
trim_right = true;
char_override = true;
}
else
{
trim_left = true;
trim_right = true;
}
if (char_override)
{
if (!ParserExpression().parse(pos, to_remove, expected))
return false;
if (!ParserKeyword("FROM").ignore(pos, expected))
return false;
auto quote_meta_func_node = std::make_shared<ASTFunction>();
auto quote_meta_list_args = std::make_shared<ASTExpressionList>();
quote_meta_list_args->children = {to_remove};
quote_meta_func_node->name = "regexpQuoteMeta";
quote_meta_func_node->arguments = std::move(quote_meta_list_args);
quote_meta_func_node->children.push_back(quote_meta_func_node->arguments);
to_remove = std::move(quote_meta_func_node);
}
}
if (!(trim_left || trim_right))
return false;
if (!ParserExpression().parse(pos, expr_node, expected))
return false;
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
/// Convert to regexp replace function call
if (char_override)
{
auto pattern_func_node = std::make_shared<ASTFunction>();
auto pattern_list_args = std::make_shared<ASTExpressionList>();
if (trim_left && trim_right)
{
pattern_list_args->children = {
std::make_shared<ASTLiteral>("^["),
to_remove,
std::make_shared<ASTLiteral>("]*|["),
to_remove,
std::make_shared<ASTLiteral>("]*$")
};
func_name = "replaceRegexpAll";
}
else
{
if (trim_left)
{
pattern_list_args->children = {
std::make_shared<ASTLiteral>("^["),
to_remove,
std::make_shared<ASTLiteral>("]*")
};
}
else
{
/// trim_right == false not possible
pattern_list_args->children = {
std::make_shared<ASTLiteral>("["),
to_remove,
std::make_shared<ASTLiteral>("]*$")
};
}
func_name = "replaceRegexpOne";
}
pattern_func_node->name = "concat";
pattern_func_node->arguments = std::move(pattern_list_args);
pattern_func_node->children.push_back(pattern_func_node->arguments);
pattern_node = std::move(pattern_func_node);
}
else
{
if (trim_left && trim_right)
{
func_name = "trimBoth";
}
else
{
if (trim_left)
{
func_name = "trimLeft";
}
else
{
/// trim_right == false not possible
func_name = "trimRight";
}
}
}
auto expr_list_args = std::make_shared<ASTExpressionList>();
if (char_override)
expr_list_args->children = {expr_node, pattern_node, std::make_shared<ASTLiteral>("")};
else
expr_list_args->children = {expr_node};
auto func_node = std::make_shared<ASTFunction>();
func_node->name = func_name;
func_node->arguments = std::move(expr_list_args);
func_node->children.push_back(func_node->arguments);
node = std::move(func_node);
return true;
}
bool ParserLeftExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
/// Rewrites left(expr, length) to SUBSTRING(expr, 1, length)
ASTPtr expr_node;
ASTPtr start_node;
ASTPtr length_node;
if (!ParserKeyword("LEFT").ignore(pos, expected))
return false;
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
if (!ParserExpression().parse(pos, expr_node, expected))
return false;
ParserToken(TokenType::Comma).ignore(pos, expected);
if (!ParserExpression().parse(pos, length_node, expected))
return false;
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
auto expr_list_args = std::make_shared<ASTExpressionList>();
start_node = std::make_shared<ASTLiteral>(1);
expr_list_args->children = {expr_node, start_node, length_node};
auto func_node = std::make_shared<ASTFunction>();
func_node->name = "substring";
func_node->arguments = std::move(expr_list_args);
func_node->children.push_back(func_node->arguments);
node = std::move(func_node);
return true;
}
bool ParserRightExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
/// Rewrites RIGHT(expr, length) to substring(expr, -length)
ASTPtr expr_node;
ASTPtr length_node;
if (!ParserKeyword("RIGHT").ignore(pos, expected))
return false;
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
if (!ParserExpression().parse(pos, expr_node, expected))
return false;
ParserToken(TokenType::Comma).ignore(pos, expected);
if (!ParserExpression().parse(pos, length_node, expected))
return false;
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
auto start_expr_list_args = std::make_shared<ASTExpressionList>();
start_expr_list_args->children = {length_node};
auto start_node = std::make_shared<ASTFunction>();
start_node->name = "negate";
start_node->arguments = std::move(start_expr_list_args);
start_node->children.push_back(start_node->arguments);
auto expr_list_args = std::make_shared<ASTExpressionList>();
expr_list_args->children = {expr_node, start_node};
auto func_node = std::make_shared<ASTFunction>();
func_node->name = "substring";
func_node->arguments = std::move(expr_list_args);
func_node->children.push_back(func_node->arguments);
node = std::move(func_node);
return true;
}
bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
auto begin = pos;
@ -402,26 +651,42 @@ bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
ASTPtr expr;
const char * function_name = nullptr;
if (ParserKeyword("SECOND").ignore(pos, expected))
function_name = "toSecond";
else if (ParserKeyword("MINUTE").ignore(pos, expected))
function_name = "toMinute";
else if (ParserKeyword("HOUR").ignore(pos, expected))
function_name = "toHour";
else if (ParserKeyword("DAY").ignore(pos, expected))
function_name = "toDayOfMonth";
// TODO: SELECT toRelativeWeekNum(toDate('2017-06-15')) - toRelativeWeekNum(toStartOfYear(toDate('2017-06-15')))
// else if (ParserKeyword("WEEK").ignore(pos, expected))
// function_name = "toRelativeWeekNum";
else if (ParserKeyword("MONTH").ignore(pos, expected))
function_name = "toMonth";
else if (ParserKeyword("YEAR").ignore(pos, expected))
function_name = "toYear";
else
ParserInterval interval_parser;
if (!interval_parser.ignore(pos, expected))
return false;
switch (interval_parser.interval_kind)
{
case ParserInterval::IntervalKind::Second:
function_name = "toSecond";
break;
case ParserInterval::IntervalKind::Minute:
function_name = "toMinute";
break;
case ParserInterval::IntervalKind::Hour:
function_name = "toHour";
break;
case ParserInterval::IntervalKind::Day:
function_name = "toDayOfMonth";
break;
case ParserInterval::IntervalKind::Week:
// TODO: SELECT toRelativeWeekNum(toDate('2017-06-15')) - toRelativeWeekNum(toStartOfYear(toDate('2017-06-15')))
// else if (ParserKeyword("WEEK").ignore(pos, expected))
// function_name = "toRelativeWeekNum";
return false;
case ParserInterval::IntervalKind::Month:
function_name = "toMonth";
break;
case ParserInterval::IntervalKind::Quarter:
function_name = "toQuarter";
break;
case ParserInterval::IntervalKind::Year:
function_name = "toYear";
break;
default:
return false;
}
ParserKeyword s_from("FROM");
if (!s_from.ignore(pos, expected))
return false;
@ -449,6 +714,168 @@ bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
return true;
}
bool ParserDateAddExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
const char * function_name = nullptr;
ASTPtr timestamp_node;
ASTPtr offset_node;
if (ParserKeyword("DATEADD").ignore(pos, expected) || ParserKeyword("DATE_ADD").ignore(pos, expected)
|| ParserKeyword("TIMESTAMPADD").ignore(pos, expected) || ParserKeyword("TIMESTAMP_ADD").ignore(pos, expected))
function_name = "plus";
else if (ParserKeyword("DATESUB").ignore(pos, expected) || ParserKeyword("DATE_SUB").ignore(pos, expected)
|| ParserKeyword("TIMESTAMPSUB").ignore(pos, expected) || ParserKeyword("TIMESTAMP_SUB").ignore(pos, expected))
function_name = "minus";
else
return false;
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
ParserInterval interval_parser;
if (interval_parser.ignore(pos, expected))
{
/// function(unit, offset, timestamp)
if (pos->type != TokenType::Comma)
return false;
++pos;
if (!ParserExpression().parse(pos, offset_node, expected))
return false;
if (pos->type != TokenType::Comma)
return false;
++pos;
if (!ParserExpression().parse(pos, timestamp_node, expected))
return false;
}
else
{
/// function(timestamp, INTERVAL offset unit)
if (!ParserExpression().parse(pos, timestamp_node, expected))
return false;
if (pos->type != TokenType::Comma)
return false;
++pos;
if (!ParserKeyword("INTERVAL").ignore(pos, expected))
return false;
if (!ParserExpression().parse(pos, offset_node, expected))
return false;
interval_parser.ignore(pos, expected);
}
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
const char * interval_function_name = interval_parser.getToIntervalKindFunctionName();
auto interval_expr_list_args = std::make_shared<ASTExpressionList>();
interval_expr_list_args->children = {offset_node};
auto interval_func_node = std::make_shared<ASTFunction>();
interval_func_node->name = interval_function_name;
interval_func_node->arguments = std::move(interval_expr_list_args);
interval_func_node->children.push_back(interval_func_node->arguments);
auto expr_list_args = std::make_shared<ASTExpressionList>();
expr_list_args->children = {timestamp_node, interval_func_node};
auto func_node = std::make_shared<ASTFunction>();
func_node->name = function_name;
func_node->arguments = std::move(expr_list_args);
func_node->children.push_back(func_node->arguments);
node = std::move(func_node);
return true;
}
bool ParserDateDiffExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
const char * interval_name = nullptr;
ASTPtr left_node;
ASTPtr right_node;
if (!(ParserKeyword("DATEDIFF").ignore(pos, expected) || ParserKeyword("DATE_DIFF").ignore(pos, expected)
|| ParserKeyword("TIMESTAMPDIFF").ignore(pos, expected) || ParserKeyword("TIMESTAMP_DIFF").ignore(pos, expected)))
return false;
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
ParserInterval interval_parser;
if (!interval_parser.ignore(pos, expected))
return false;
switch (interval_parser.interval_kind)
{
case ParserInterval::IntervalKind::Second:
interval_name = "second";
break;
case ParserInterval::IntervalKind::Minute:
interval_name = "minute";
break;
case ParserInterval::IntervalKind::Hour:
interval_name = "hour";
break;
case ParserInterval::IntervalKind::Day:
interval_name = "day";
break;
case ParserInterval::IntervalKind::Week:
interval_name = "week";
break;
case ParserInterval::IntervalKind::Month:
interval_name = "month";
break;
case ParserInterval::IntervalKind::Quarter:
interval_name = "quarter";
break;
case ParserInterval::IntervalKind::Year:
interval_name = "year";
break;
default:
return false;
}
if (pos->type != TokenType::Comma)
return false;
++pos;
if (!ParserExpression().parse(pos, left_node, expected))
return false;
if (pos->type != TokenType::Comma)
return false;
++pos;
if (!ParserExpression().parse(pos, right_node, expected))
return false;
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
auto expr_list_args = std::make_shared<ASTExpressionList>();
expr_list_args->children = {std::make_shared<ASTLiteral>(interval_name), left_node, right_node};
auto func_node = std::make_shared<ASTFunction>();
func_node->name = "dateDiff";
func_node->arguments = std::move(expr_list_args);
func_node->children.push_back(func_node->arguments);
node = std::move(func_node);
return true;
}
bool ParserNull::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
@ -750,7 +1177,12 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|| ParserLiteral().parse(pos, node, expected)
|| ParserCastExpression().parse(pos, node, expected)
|| ParserExtractExpression().parse(pos, node, expected)
|| ParserDateAddExpression().parse(pos, node, expected)
|| ParserDateDiffExpression().parse(pos, node, expected)
|| ParserSubstringExpression().parse(pos, node, expected)
|| ParserTrimExpression().parse(pos, node, expected)
|| ParserLeftExpression().parse(pos, node, expected)
|| ParserRightExpression().parse(pos, node, expected)
|| ParserCase().parse(pos, node, expected)
|| ParserFunction().parse(pos, node, expected)
|| ParserQualifiedAsterisk().parse(pos, node, expected)

View File

@ -103,6 +103,27 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserTrimExpression : public IParserBase
{
protected:
const char * getName() const override { return "TRIM expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserLeftExpression : public IParserBase
{
protected:
const char * getName() const override { return "LEFT expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserRightExpression : public IParserBase
{
protected:
const char * getName() const override { return "RIGHT expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserExtractExpression : public IParserBase
{
protected:
@ -110,6 +131,19 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserDateAddExpression : public IParserBase
{
protected:
const char * getName() const override { return "DATE_ADD expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserDateDiffExpression : public IParserBase
{
protected:
const char * getName() const override { return "DATE_DIFF expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** NULL literal.
*/

View File

@ -607,25 +607,13 @@ bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expec
if (!ParserExpressionWithOptionalAlias(false).parse(pos, expr, expected))
return false;
const char * function_name = nullptr;
if (ParserKeyword("SECOND").ignore(pos, expected))
function_name = "toIntervalSecond";
else if (ParserKeyword("MINUTE").ignore(pos, expected))
function_name = "toIntervalMinute";
else if (ParserKeyword("HOUR").ignore(pos, expected))
function_name = "toIntervalHour";
else if (ParserKeyword("DAY").ignore(pos, expected))
function_name = "toIntervalDay";
else if (ParserKeyword("WEEK").ignore(pos, expected))
function_name = "toIntervalWeek";
else if (ParserKeyword("MONTH").ignore(pos, expected))
function_name = "toIntervalMonth";
else if (ParserKeyword("YEAR").ignore(pos, expected))
function_name = "toIntervalYear";
else
ParserInterval interval_parser;
if (!interval_parser.ignore(pos, expected))
return false;
const char * function_name = interval_parser.getToIntervalKindFunctionName();
/// the function corresponding to the operator
auto function = std::make_shared<ASTFunction>();

View File

@ -0,0 +1,34 @@
<test>
<name>right</name>
<type>loop</type>
<preconditions>
<table_exists>hits_100m_single</table_exists>
</preconditions>
<stop_conditions>
<all_of>
<total_time_ms>10000</total_time_ms>
</all_of>
<any_of>
<average_speed_not_changing_for_ms>5000</average_speed_not_changing_for_ms>
<total_time_ms>20000</total_time_ms>
</any_of>
</stop_conditions>
<main_metric>
<min_time/>
</main_metric>
<substitutions>
<substitution>
<name>func</name>
<values>
<value>right(URL, 16)</value>
<value>substring(URL, greatest(minus(plus(length(URL), 1), 16), 1))</value>
</values>
</substitution>
</substitutions>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore({func})</query>
</test>

View File

@ -0,0 +1,34 @@
<test>
<name>trim_numbers</name>
<type>loop</type>
<stop_conditions>
<all_of>
<total_time_ms>10000</total_time_ms>
</all_of>
<any_of>
<average_speed_not_changing_for_ms>5000</average_speed_not_changing_for_ms>
<total_time_ms>20000</total_time_ms>
</any_of>
</stop_conditions>
<main_metric>
<min_time/>
</main_metric>
<substitutions>
<substitution>
<name>func</name>
<values>
<value>trim(</value>
<value>ltrim(</value>
<value>rtrim(</value>
<value>trim(LEADING '012345' FROM </value>
<value>trim(TRAILING '012345' FROM </value>
<value>trim(BOTH '012345' FROM </value>
</values>
</substitution>
</substitutions>
<query>SELECT count() FROM numbers(10000000) WHERE NOT ignore({func}toString(number)))</query>
</test>

View File

@ -0,0 +1,38 @@
<test>
<name>trim_urls</name>
<type>loop</type>
<preconditions>
<table_exists>hits_100m_single</table_exists>
</preconditions>
<stop_conditions>
<all_of>
<total_time_ms>10000</total_time_ms>
</all_of>
<any_of>
<average_speed_not_changing_for_ms>5000</average_speed_not_changing_for_ms>
<total_time_ms>20000</total_time_ms>
</any_of>
</stop_conditions>
<main_metric>
<min_time/>
</main_metric>
<substitutions>
<substitution>
<name>func</name>
<values>
<value>trim(</value>
<value>ltrim(</value>
<value>rtrim(</value>
<value>trim(LEADING 'htpsw:/' FROM </value>
<value>trim(TRAILING '/' FROM </value>
<value>trim(BOTH 'htpsw:/' FROM </value>
</values>
</substitution>
</substitutions>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore({func}URL))</query>
</test>

View File

@ -0,0 +1,35 @@
<test>
<name>trim_whitespaces</name>
<type>loop</type>
<preconditions>
<table_exists>whitespaces</table_exists>
</preconditions>
<stop_conditions>
<all_of>
<total_time_ms>30000</total_time_ms>
</all_of>
</stop_conditions>
<main_metric>
<min_time/>
</main_metric>
<substitutions>
<substitution>
<name>func</name>
<values>
<value>value</value>
<value>trimLeft(value)</value>
<value>trimRight(value)</value>
<value>trimBoth(value)</value>
<value>replaceRegexpOne(value, '^ *', '')</value>
<value>replaceRegexpOne(value, ' *$', '')</value>
<value>replaceRegexpAll(value, '^ *| *$', '')</value>
</values>
</substitution>
</substitutions>
<query>SELECT count() FROM whitespaces WHERE NOT ignore({func})</query>
</test>

View File

@ -0,0 +1,17 @@
CREATE TABLE whitespaces
(
value String
)
ENGINE = MergeTree()
PARTITION BY tuple()
ORDER BY tuple()
INSERT INTO whitespaces SELECT value
FROM
(
SELECT
arrayStringConcat(groupArray(' ')) AS spaces,
concat(spaces, toString(any(number)), spaces) AS value
FROM numbers(100000000)
GROUP BY pow(number, intHash32(number) % 4) % 12345678
) -- repeat something like this multiple times and/or just copy whitespaces table into itself

View File

@ -36,3 +36,4 @@
2029-02-28 01:02:03 2017-03-29 01:02:03
2030-02-28 01:02:03 2017-04-29 01:02:03
2031-02-28 01:02:03 2017-05-29 01:02:03
2015-11-29 01:02:03

View File

@ -2,3 +2,4 @@ SELECT toDateTime('2017-10-30 08:18:19') + INTERVAL 1 DAY + INTERVAL 1 MONTH - I
SELECT toDateTime('2017-10-30 08:18:19') + INTERVAL 1 HOUR + INTERVAL 1000 MINUTE + INTERVAL 10 SECOND;
SELECT toDateTime('2017-10-30 08:18:19') + INTERVAL 1 DAY + INTERVAL number MONTH FROM system.numbers LIMIT 20;
SELECT toDateTime('2016-02-29 01:02:03') + INTERVAL number YEAR, toDateTime('2016-02-29 01:02:03') + INTERVAL number MONTH FROM system.numbers LIMIT 16;
SELECT toDateTime('2016-02-29 01:02:03') - INTERVAL 1 QUARTER;

View File

@ -13,7 +13,7 @@ SELECT EXTRACT(year FROM toDateTime('2017-12-31 18:59:58'));
DROP TABLE IF EXISTS test.Orders;
CREATE TABLE test.Orders (OrderId UInt64, OrderName String, OrderDate DateTime) engine = Log;
insert into test.Orders values (1, 'Jarlsberg Cheese', toDateTime('2008-10-11 13:23:44'));
SELECT EXTRACT(YEAR FROM OrderDate) AS OrderYear, EXTRACT(MONTH FROM OrderDate) AS OrderMonth, EXTRACT(DAY FROM OrderDate) AS OrderDay,
SELECT EXTRACT(YYYY FROM OrderDate) AS OrderYear, EXTRACT(MONTH FROM OrderDate) AS OrderMonth, EXTRACT(DAY FROM OrderDate) AS OrderDay,
EXTRACT(HOUR FROM OrderDate), EXTRACT(MINUTE FROM OrderDate), EXTRACT(SECOND FROM OrderDate) FROM test.Orders WHERE OrderId=1;
DROP TABLE test.Orders;

View File

@ -10,3 +10,19 @@ o
1
oo
o
fo
foo
r
bar
foo
foo
xxfoo
fooabba
fooabbafoo
foo*
-11
-3
2021-01-01
2018-07-18 01:02:03
2018-04-01

View File

@ -12,3 +12,19 @@ select mid('foo', 3);
select IF(3>2, 1, 0);
select substring('foo' from 1 + 1);
select SUBSTRING('foo' FROM 2 FOR 1);
select left('foo', 2);
select LEFT('foo', 123);
select RIGHT('bar', 1);
select right('bar', 123);
select ltrim('') || rtrim('') || trim('');
select ltrim(' foo');
select RTRIM(' foo ');
select trim(TRAILING 'x' FROM 'xxfooxx');
select Trim(LEADING 'ab' FROM 'abbafooabba');
select TRIM(both 'ab' FROM 'abbafooabbafooabba');
select trim(LEADING '*[]{}|\\' FROM '\\|[[[}}}*foo*');
select DATE_DIFF(MONTH, toDate('2018-12-18'), toDate('2018-01-01'));
select DATE_DIFF(QQ, toDate('2018-12-18'), toDate('2018-01-01'));
select DATE_ADD(YEAR, 3, toDate('2018-01-01'));
select timestamp_sub(SQL_TSI_MONTH, 5, toDateTime('2018-12-18 01:02:03'));
select timestamp_ADD(toDate('2018-01-01'), INTERVAL 3 MONTH);

View File

@ -584,6 +584,16 @@ public:
}
}
inline time_t addQuarters(time_t t, Int64 delta) const
{
return addMonths(t, delta * 3);
}
inline DayNum addQuarters(DayNum d, Int64 delta) const
{
return addMonths(d, delta * 3);
}
/// Saturation can occur if 29 Feb is mapped to non-leap year.
inline time_t addYears(time_t t, Int64 delta) const
{