Merge branch 'alex-krash-function_format_date'

This commit is contained in:
Alexey Milovidov 2018-09-18 21:51:14 +03:00
commit dc7963a957
22 changed files with 777 additions and 181 deletions

View File

@ -360,31 +360,6 @@ void DataTypeArray::deserializeText(IColumn & column, ReadBuffer & istr, const F
});
}
void DataTypeArray::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
serializeText(column, row_num, ostr, settings);
}
void DataTypeArray::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
deserializeText(column, istr, settings);
}
void DataTypeArray::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
serializeText(column, row_num, ostr, settings);
}
void DataTypeArray::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
deserializeText(column, istr, settings);
}
void DataTypeArray::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
const ColumnArray & column_array = static_cast<const ColumnArray &>(column);

View File

@ -1,13 +1,13 @@
#pragma once
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeWithSimpleSerialization.h>
namespace DB
{
class DataTypeArray final : public IDataType
class DataTypeArray final : public DataTypeWithSimpleSerialization
{
private:
/// The type of array elements.
@ -41,13 +41,7 @@ public:
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;

View File

@ -14,7 +14,7 @@ void DataTypeDate::serializeText(const IColumn & column, size_t row_num, WriteBu
writeDateText(DayNum(static_cast<const ColumnUInt16 &>(column).getData()[row_num]), ostr);
}
static void deserializeText(IColumn & column, ReadBuffer & istr)
void DataTypeDate::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
DayNum x;
readDateText(x, istr);
@ -26,11 +26,6 @@ void DataTypeDate::serializeTextEscaped(const IColumn & column, size_t row_num,
serializeText(column, row_num, ostr, settings);
}
void DataTypeDate::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
deserializeText(column, istr);
}
void DataTypeDate::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
writeChar('\'', ostr);

View File

@ -19,14 +19,7 @@ void DataTypeNumberBase<T>::serializeText(const IColumn & column, size_t row_num
}
template <typename T>
void DataTypeNumberBase<T>::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
serializeText(column, row_num, ostr, settings);
}
template <typename T>
static void deserializeText(IColumn & column, ReadBuffer & istr)
void DataTypeNumberBase<T>::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
T x;
@ -38,26 +31,6 @@ static void deserializeText(IColumn & column, ReadBuffer & istr)
static_cast<ColumnVector<T> &>(column).getData().push_back(x);
}
template <typename T>
void DataTypeNumberBase<T>::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
deserializeText<T>(column, istr);
}
template <typename T>
void DataTypeNumberBase<T>::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
serializeText(column, row_num, ostr, settings);
}
template <typename T>
void DataTypeNumberBase<T>::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
deserializeText<T>(column, istr);
}
template <typename T>
static inline void writeDenormalNumber(T x, WriteBuffer & ostr)
{
@ -160,12 +133,6 @@ void DataTypeNumberBase<T>::deserializeTextJSON(IColumn & column, ReadBuffer & i
static_cast<ColumnVector<T> &>(column).getData().push_back(x);
}
template <typename T>
void DataTypeNumberBase<T>::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
serializeText(column, row_num, ostr, settings);
}
template <typename T>
void DataTypeNumberBase<T>::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{

View File

@ -1,6 +1,7 @@
#pragma once
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeWithSimpleSerialization.h>
namespace DB
@ -9,7 +10,7 @@ namespace DB
/** Implements part of the IDataType interface, common to all numbers and for Date and DateTime.
*/
template <typename T>
class DataTypeNumberBase : public IDataType
class DataTypeNumberBase : public DataTypeWithSimpleSerialization
{
static_assert(IsNumber<T>);
@ -21,13 +22,9 @@ public:
TypeIndex getTypeId() const override { return TypeId<T>::value; }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
Field getDefault() const override;

View File

@ -188,26 +188,6 @@ void DataTypeTuple::deserializeText(IColumn & column, ReadBuffer & istr, const F
assertChar(')', istr);
}
void DataTypeTuple::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
serializeText(column, row_num, ostr, settings);
}
void DataTypeTuple::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
deserializeText(column, istr, settings);
}
void DataTypeTuple::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
serializeText(column, row_num, ostr, settings);
}
void DataTypeTuple::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
deserializeText(column, istr, settings);
}
void DataTypeTuple::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
writeChar('[', ostr);

View File

@ -1,6 +1,6 @@
#pragma once
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeWithSimpleSerialization.h>
namespace DB
@ -16,7 +16,7 @@ namespace DB
*
* All tuples with same size and types of elements are equivalent for expressions, regardless to names of elements.
*/
class DataTypeTuple final : public IDataType
class DataTypeTuple final : public DataTypeWithSimpleSerialization
{
private:
DataTypes elems;
@ -39,11 +39,7 @@ public:
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -10,7 +10,7 @@ void DataTypeUUID::serializeText(const IColumn & column, size_t row_num, WriteBu
writeText(UUID(static_cast<const ColumnUInt128 &>(column).getData()[row_num]), ostr);
}
static void deserializeText(IColumn & column, ReadBuffer & istr)
void DataTypeUUID::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
UUID x;
readText(x, istr);
@ -22,11 +22,6 @@ void DataTypeUUID::serializeTextEscaped(const IColumn & column, size_t row_num,
serializeText(column, row_num, ostr, settings);
}
void DataTypeUUID::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
deserializeText(column, istr);
}
void DataTypeUUID::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
writeChar('\'', ostr);

View File

@ -0,0 +1,58 @@
#pragma once
#include <DataTypes/IDataType.h>
namespace DB
{
/// Helper class to define same IDataType text (de)serialization for all the variants (escaped, quoted, JSON, CSV).
/// You need to define serializeText() and deserializeText() in derived class.
class DataTypeWithSimpleSerialization : public IDataType
{
protected:
DataTypeWithSimpleSerialization()
{}
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{
serializeText(column, row_num, ostr, settings);
}
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{
serializeText(column, row_num, ostr, settings);
}
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{
serializeText(column, row_num, ostr, settings);
}
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{
serializeText(column, row_num, ostr, settings);
}
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{
deserializeText(column, istr, settings);
}
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{
deserializeText(column, istr, settings);
}
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{
deserializeText(column, istr, settings);
}
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{
deserializeText(column, istr, settings);
}
virtual void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
};
}

View File

@ -4,6 +4,7 @@
#include <Columns/ColumnDecimal.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeWithSimpleSerialization.h>
namespace DB
@ -16,53 +17,6 @@ namespace ErrorCodes
extern const int CANNOT_CONVERT_TYPE;
}
///
class DataTypeSimpleSerialization : public IDataType
{
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{
serializeText(column, row_num, ostr, settings);
}
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{
serializeText(column, row_num, ostr, settings);
}
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{
serializeText(column, row_num, ostr, settings);
}
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{
serializeText(column, row_num, ostr, settings);
}
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{
deserializeText(column, istr, settings);
}
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{
deserializeText(column, istr, settings);
}
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{
deserializeText(column, istr, settings);
}
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{
deserializeText(column, istr, settings);
}
virtual void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
};
class Context;
bool decimalCheckComparisonOverflow(const Context & context);
bool decimalCheckArithmeticOverflow(const Context & context);
@ -89,7 +43,7 @@ DataTypePtr createDecimal(UInt64 precision, UInt64 scale);
///
/// NOTE: It's possible to set scale as a template parameter then most of functions become static.
template <typename T>
class DataTypeDecimal final : public DataTypeSimpleSerialization
class DataTypeDecimal final : public DataTypeWithSimpleSerialization
{
static_assert(IsDecimalNumber<T>);

View File

@ -1,6 +1,6 @@
#pragma once
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeWithSimpleSerialization.h>
namespace DB
@ -11,7 +11,7 @@ namespace DB
*
* That is, this class is used just to distinguish the corresponding data type from the others.
*/
class IDataTypeDummy : public IDataType
class IDataTypeDummy : public DataTypeWithSimpleSerialization
{
private:
void throwNoSerialization() const
@ -26,15 +26,8 @@ public:
void deserializeBinary(IColumn &, ReadBuffer &) const override { throwNoSerialization(); }
void serializeBinaryBulk(const IColumn &, WriteBuffer &, size_t, size_t) const override { throwNoSerialization(); }
void deserializeBinaryBulk(IColumn &, ReadBuffer &, size_t, double) const override { throwNoSerialization(); }
void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void serializeTextEscaped(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void deserializeTextEscaped(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void serializeTextQuoted(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void deserializeTextQuoted(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void serializeTextJSON(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void deserializeTextJSON(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void serializeTextCSV(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void deserializeTextCSV(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void deserializeText(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
MutableColumnPtr createColumn() const override
{

View File

@ -115,6 +115,9 @@ void registerFunctionsDateTime(FunctionFactory & factory)
factory.registerFunction<FunctionDateDiff>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionToTimeZone>();
factory.registerFunction<FunctionFormatDateTime>();
}
}

View File

@ -1571,6 +1571,546 @@ public:
};
/** formatDateTime(time, 'pattern')
* Performs formatting of time, according to provided pattern
*/
class FunctionFormatDateTime : public IFunction
{
private:
class FormattingOperation
{
public:
FormattingOperation(const char * source, size_t copy_source, size_t copy_length)
: source(source), source_position_to_copy(copy_source), source_length_to_copy(copy_length) {}
void (*operation)(char *&, UInt32, const DateLUTImpl &) = nullptr;
private:
static constexpr char digits[] = "0123456789";
const char * source;
size_t source_position_to_copy = 0;
size_t source_length_to_copy = 0;
template <typename T>
static inline void writeNumber2(char *& ep, T v)
{
writeNumberWidth(ep, v, 2);
}
template <typename T>
static inline void writeNumberWidth(char *& ep, T v, int width)
{
auto width_copy = width;
ep += width_copy;
do
{
--width;
*--ep = digits[v % 10]; /// TODO v % 10 + '0' should be better (one arithmetic op vs. load from L1 cache)
} while (v /= 10);
while (--width >= 0)
*--ep = '0';
ep += width_copy;
}
public:
// format DateTime
void action(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
if (operation)
operation(target, source, timezone);
else
copy(target, source, timezone);
}
// format Date (convert to DateTime implicitly)
void action(char *& target, UInt16 date, const DateLUTImpl & timezone)
{
/// TODO This is not efficient.
const UInt32 datetime = timezone.fromDayNum(DayNum(date));
action(target, datetime, timezone);
}
void copy(char *& target, UInt32, const DateLUTImpl &)
{
memcpy(target, source + source_position_to_copy, source_length_to_copy);
target += source_length_to_copy;
}
static void format_C(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
auto year = ToYearImpl::execute(source, timezone);
auto century = year / 100;
writeNumber2(target, century);
}
static void format_d(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
writeNumber2(target, ToDayOfMonthImpl::execute(source, timezone));
}
static void format_D(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
auto month = ToMonthImpl::execute(source, timezone);
auto day = ToDayOfMonthImpl::execute(source, timezone);
auto year = ToYearImpl::execute(source, timezone) % 100;
writeNumber2(target, month);
*target++ = '/';
writeNumber2(target, day);
*target++ = '/';
writeNumber2(target, year);
}
static void format_e(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
auto day = ToDayOfMonthImpl::execute(source, timezone);
if (day < 10)
{
*target++ = ' ';
*target++ = '0' + day;
}
else
writeNumber2(target, day);
}
static inline void format_F(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
auto year = ToYearImpl::execute(source, timezone);
auto month = ToMonthImpl::execute(source, timezone);
auto day = ToDayOfMonthImpl::execute(source, timezone);
writeNumberWidth(target, year, 4);
*target++ = '-';
writeNumber2(target, month);
*target++ = '-';
writeNumber2(target, day);
}
static void format_H(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
auto x = ToHourImpl::execute(source, timezone);
writeNumber2(target, x);
}
static void format_I(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
auto x = ToHourImpl::execute(source, timezone);
writeNumber2(target, x == 0 ? 12 : (x > 12 ? x - 12 : x));
}
/// TODO format_l
static void format_j(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
writeNumberWidth(target, ToDayOfYearImpl::execute(source, timezone), 3);
}
static void format_m(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
auto x = ToMonthImpl::execute(source, timezone);
writeNumber2(target, x);
}
static void format_M(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
auto x = ToMinuteImpl::execute(source, timezone);
writeNumber2(target, x);
}
static void format_n(char *& target, UInt32 , const DateLUTImpl & )
{
*target++ = '\n';
}
static void format_p(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
*target++ = ToHourImpl::execute(source, timezone) < 12 ? 'A' : 'P';
*target++ = 'M';
}
// 24-hour HH:MM time, equivalent to %H:%M 14:55
static void format_R(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
auto hour = ToHourImpl::execute(source, timezone);
auto minute = ToMinuteImpl::execute(source, timezone);
writeNumber2(target, hour);
*target++ = ':';
writeNumber2(target, minute);
}
static void format_S(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
writeNumber2(target, ToSecondImpl::execute(source, timezone));
}
static void format_t(char *& target, UInt32 , const DateLUTImpl & )
{
*target++ = '\t';
}
// ISO 8601 time format (HH:MM:SS), equivalent to %H:%M:%S 14:55:02
static inline void format_T(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
writeNumber2(target, ToHourImpl::execute(source, timezone));
*target++ = ':';
writeNumber2(target, ToMinuteImpl::execute(source, timezone));
*target++ = ':';
writeNumber2(target, ToSecondImpl::execute(source, timezone));
}
// ISO 8601 weekday as number with Monday as 1 (1-7)
static void format_u(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
*target++ = '0' + ToDayOfWeekImpl::execute(source, timezone);
}
// Week number with the first Sunday as the first day of week one (00-53)
static void format_U(char *& , UInt32 , const DateLUTImpl & )
{
throw Exception("Handler %U is not implemented", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
// ISO 8601 week number (01-53)
static void format_V(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
writeNumber2(target, ToISOWeekImpl::execute(source, timezone));
}
// Week number with the first Monday as the first day of week one (00-53)
static void format_W(char *& , UInt32 , const DateLUTImpl & )
{
throw Exception("Handler %W is not implemented", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
// Weekday as a decimal number with Sunday as 0 (0-6)
static void format_w(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
auto day = ToDayOfWeekImpl::execute(source, timezone);
*target++ = '0' + (day == 7 ? 0 : day);
}
// Year, last two digits (00-99)
static void format_y(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
writeNumber2(target, ToYearImpl::execute(source, timezone) % 100);
}
// Year, 4 digits
static void format_Y(char *& target, UInt32 source, const DateLUTImpl & timezone)
{
writeNumberWidth(target, ToYearImpl::execute(source, timezone), 4);
}
static void format_Percent(char *& target, UInt32 , const DateLUTImpl & )
{
*target++ = '%';
}
};
public:
static constexpr auto name = "formatDateTime";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionFormatDateTime>(); }
String getName() const override
{
return name;
}
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1,2}; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 2 && arguments.size() != 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 2 or 3",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!WhichDataType(arguments[0].type).isDateOrDateTime())
throw Exception("Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() +
". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!WhichDataType(arguments[1].type).isString())
throw Exception("Illegal type " + arguments[1].type->getName() + " of 2 argument of function " + getName() + ". Must be String.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (arguments.size() == 3)
{
if (!WhichDataType(arguments[2].type).isString())
throw Exception("Illegal type " + arguments[2].type->getName() + " of 3 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
{
if (!executeType<UInt32>(block, arguments, result) && !executeType<UInt16>(block, arguments, result))
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of function " + getName() + ", must be Date or DateTime",
ErrorCodes::ILLEGAL_COLUMN);
}
template <typename T>
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
{
if (auto * times = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
{
const ColumnConst * pattern_column = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
if (!pattern_column)
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
+ " of second ('format') argument of function " + getName()
+ ". Must be constant string.",
ErrorCodes::ILLEGAL_COLUMN);
String pattern = pattern_column->getValue<String>();
std::vector<FormattingOperation> instructions = {};
size_t result_length = parsePattern(pattern, instructions);
const DateLUTImpl * time_zone_tmp = nullptr;
if (arguments.size() == 3)
time_zone_tmp = &extractTimeZoneFromFunctionArguments(block, arguments, 2, 0);
else
time_zone_tmp = &DateLUT::instance();
const DateLUTImpl & time_zone = *time_zone_tmp;
const typename ColumnVector<T>::Container & vec = times->getData();
auto col_res = ColumnString::create();
auto & dst_data = col_res->getChars();
auto & dst_offsets = col_res->getOffsets();
dst_data.resize(vec.size() * (result_length + 1));
dst_offsets.resize(vec.size());
auto begin = reinterpret_cast<char *>(dst_data.data());
auto pos = begin;
for (size_t i = 0; i < vec.size(); ++i)
{
for(auto & instruction : instructions)
instruction.action(pos, vec[i], time_zone);
*pos++ = '\0';
dst_offsets[i] = pos - begin;
}
dst_data.resize(pos - begin);
block.getByPosition(result).column = std::move(col_res);
return true;
}
return false;
}
size_t parsePattern(String & pattern, std::vector<FormattingOperation> & instructions)
{
char * begin_of_pattern = pattern.data();
size_t result_size = 0;
size_t last_pos = 0;
for (size_t s = 0; s < pattern.length(); ++s)
{
if (pattern[s] == '%')
{
if (last_pos > 0)
{
auto length = 1 + s - last_pos;
instructions.emplace_back(begin_of_pattern, last_pos - 1, length);
last_pos = 0;
result_size += length;
}
FormattingOperation formatting_operation(begin_of_pattern, 0, 0);
auto & operation = formatting_operation.operation;
if (++s == pattern.length())
throw Exception("Sign '%' is last in pattern, if you need it, use '%%'", ErrorCodes::BAD_ARGUMENTS);
switch (pattern[s])
{
// Year, divided by 100, zero-padded
case 'C':
operation = &FormattingOperation::format_C;
result_size += 2;
break;
// Day of the month, zero-padded (01-31)
case 'd':
operation = &FormattingOperation::format_d;
result_size += 2;
break;
// Short MM/DD/YY date, equivalent to %m/%d/%y
case 'D':
operation = &FormattingOperation::format_D;
result_size += 8;
break;
// Day of the month, space-padded ( 1-31) 23
case 'e':
operation = &FormattingOperation::format_e;
result_size += 2;
break;
// Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23
case 'F':
operation = &FormattingOperation::format_F;
result_size += 10;
break;
// Hour in 24h format (00-23)
case 'H':
operation = &FormattingOperation::format_H;
result_size += 2;
break;
// Hour in 12h format (01-12)
case 'I':
operation = &FormattingOperation::format_I;
result_size += 2;
break;
// Day of the year (001-366) 235
case 'j':
operation = &FormattingOperation::format_j;
result_size += 3;
break;
// Month as a decimal number (01-12)
case 'm':
operation = &FormattingOperation::format_m;
result_size += 2;
break;
// Minute (00-59)
case 'M':
operation = &FormattingOperation::format_M;
result_size += 2;
break;
// New line character "\n"
case 'n':
operation = &FormattingOperation::format_n;
result_size += 1;
break;
// AM or PM
case 'p':
operation = &FormattingOperation::format_p;
result_size += 2;
break;
// 24-hour HH:MM time, equivalent to %H:%M 14:55
case 'R':
operation = &FormattingOperation::format_R;
result_size += 5;
break;
// Seconds
case 'S':
operation = &FormattingOperation::format_S;
result_size += 2;
break;
// Horizontal-tab character ('\t')
case 't':
operation = &FormattingOperation::format_t;
result_size += 1;
break;
// ISO 8601 time format (HH:MM:SS), equivalent to %H:%M:%S 14:55:02
case 'T':
operation = &FormattingOperation::format_T;
result_size += 8;
break;
// ISO 8601 weekday as number with Monday as 1 (1-7)
case 'u':
operation = &FormattingOperation::format_u;
result_size += 1;
break;
// Week number with the first Sunday as the first day of week one (00-53)
case 'U':
operation = &FormattingOperation::format_U;
result_size += 2;
break;
// ISO 8601 week number (01-53)
case 'V':
operation = &FormattingOperation::format_V;
result_size += 2;
break;
// Weekday as a decimal number with Sunday as 0 (0-6) 4
case 'w':
operation = &FormattingOperation::format_w;
result_size += 1;
break;
// Week number with the first Monday as the first day of week one (00-53)
case 'W':
operation = &FormattingOperation::format_W;
result_size += 2;
break;
// Four digits year
case 'y':
operation = &FormattingOperation::format_y;
result_size += 2;
break;
// Four digits year
case 'Y':
operation = &FormattingOperation::format_Y;
result_size += 4;
break;
case '%':
operation = &FormattingOperation::format_Percent;
result_size += 1;
break;
default:
throw Exception(
"Wrong pattern '" + pattern + "', unexpected symbol '" + pattern[s] + "' for function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
}
instructions.emplace_back(std::move(formatting_operation));
}
else
{
if (last_pos == 0)
last_pos = s + 1;
}
}
if (last_pos > 0)
{
auto length = 1 + pattern.length() - last_pos;
instructions.emplace_back(begin_of_pattern, last_pos - 1, length);
result_size += length;
}
return result_size;
}
};
using FunctionToYear = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToYearImpl>;
using FunctionToQuarter = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToQuarterImpl>;
using FunctionToMonth = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToMonthImpl>;

View File

@ -0,0 +1,31 @@
20
02
01/02/18
2
2018-01-02
22
02
10
11
12
001
366
01
33
\n
AM
AM
PM
22:33
44
\t
22:33:44
1 7
01 01 53 52
1 0
18
2018
%
no formatting pattern
2018-01-01 00:00:00
2018-01-01 01:00:00 2018-01-01 04:00:00

View File

@ -0,0 +1,45 @@
SET send_logs_level = 'none';
SELECT formatDateTime(); -- { serverError 42 }
SELECT formatDateTime('not a datetime', 'IGNORED'); -- { serverError 43 }
SELECT formatDateTime(now(), now()); -- { serverError 43 }
SELECT formatDateTime(now(), 'good format pattern', now()); -- { serverError 43 }
SELECT formatDateTime(now(), 'unescaped %'); -- { serverError 36 }
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%U'); -- { serverError 43 }
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%W'); -- { serverError 43 }
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%C');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%d');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%D');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%e');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%F');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%H');
SELECT formatDateTime(toDateTime('2018-01-02 02:33:44'), '%H');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%I');
SELECT formatDateTime(toDateTime('2018-01-02 11:33:44'), '%I');
SELECT formatDateTime(toDateTime('2018-01-02 00:33:44'), '%I');
SELECT formatDateTime(toDateTime('2018-01-01 00:33:44'), '%j');
SELECT formatDateTime(toDateTime('2000-12-31 00:33:44'), '%j');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%m');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%M');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%n');
SELECT formatDateTime(toDateTime('2018-01-02 00:33:44'), '%p');
SELECT formatDateTime(toDateTime('2018-01-02 11:33:44'), '%p');
SELECT formatDateTime(toDateTime('2018-01-02 12:33:44'), '%p');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%R');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%S');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%t');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%T');
SELECT formatDateTime(toDateTime('2018-01-01 22:33:44'), '%u'), formatDateTime(toDateTime('2018-01-07 22:33:44'), '%u');
SELECT formatDateTime(toDateTime('1996-01-01 22:33:44'), '%V'), formatDateTime(toDateTime('1996-12-31 22:33:44'), '%V'),
formatDateTime(toDateTime('1999-01-01 22:33:44'), '%V'), formatDateTime(toDateTime('1999-12-31 22:33:44'), '%V');
SELECT formatDateTime(toDateTime('2018-01-01 22:33:44'), '%w'), formatDateTime(toDateTime('2018-01-07 22:33:44'), '%w');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%y');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%Y');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%%');
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), 'no formatting pattern');
SELECT formatDateTime(toDate('2018-01-01'), '%F %T');
SELECT
formatDateTime(toDateTime('2018-01-01 01:00:00', 'UTC'), '%F %T', 'UTC'),
formatDateTime(toDateTime('2018-01-01 01:00:00', 'UTC'), '%F %T', 'Europe/Moscow')

4
debian/.pbuilderrc vendored
View File

@ -160,7 +160,7 @@ case "$DIST" in
esac
if [ "$ARCH" != arm64 ]; then
case "$DIST" in
case "$DIST" in
"cosmic" | "bionic" | "experimental" | "unstable" | "testing")
EXTRAPACKAGES+=" liblld-6.0-dev libclang-6.0-dev liblld-6.0 "
export CMAKE_FLAGS="-DLLVM_VERSION_POSTFIX=-6.0 $CMAKE_FLAGS"
@ -168,7 +168,7 @@ if [ "$ARCH" != arm64 ]; then
"artful" )
EXTRAPACKAGES+=" liblld-5.0-dev libclang-5.0-dev liblld-5.0 "
;;
esac
esac
else
export CMAKE_FLAGS="-DENABLE_EMBEDDED_COMPILER=0 $CMAKE_FLAGS"
fi

7
debian/rules vendored
View File

@ -47,7 +47,12 @@ else
endif
endif
CMAKE_FLAGS += -DCMAKE_CXX_COMPILER=`which $(CXX)` -DCMAKE_C_COMPILER=`which $(CC)`
ifdef CXX
CMAKE_FLAGS += -DCMAKE_CXX_COMPILER=`which $(CXX)`
endif
ifdef CC
CMAKE_FLAGS += -DCMAKE_C_COMPILER=`which $(CC)`
endif
ifndef DISABLE_NINJA
NINJA=$(shell which ninja)

View File

@ -47,7 +47,8 @@ Clusters are set like this:
</replica>
<replica>
<host>example01-02-2</host>
<port>9000</port>
<secure>1</secure>
<port>9440</port>
</replica>
</shard>
</logs>
@ -60,12 +61,14 @@ Replicas are duplicating servers (in order to read all the data, you can access
Cluster names must not contain dots.
The parameters `host`, `port`, and optionally `user` and `password` are specified for each server:
The parameters `host`, `port`, and optionally `user`, `password`, `secure`, `compression` are specified for each server:
: - `host` The address of the remote server. You can use either the domain or the IPv4 or IPv6 address. If you specify the domain, the server makes a DNS request when it starts, and the result is stored as long as the server is running. If the DNS request fails, the server doesn't start. If you change the DNS record, restart the server.
- `port` The TCP port for messenger activity ('tcp_port' in the config, usually set to 9000). Do not confuse it with http_port.
- `user` Name of the user for connecting to a remote server. Default value: default. This user must have access to connect to the specified server. Access is configured in the users.xml file. For more information, see the section "Access rights".
- `password` The password for connecting to a remote server (not masked). Default value: empty string.
- `password` The password for connecting to a remote server (not masked). Default value: empty string.
- `secure` - Use ssl for connection, usually you also should define `port` = 9440. Server should listen on <tcp_port_secure>9440</tcp_port_secure> and have correct certificates.
- `compression` - Use data compression. Default value: true.
When specifying replicas, one of the available replicas will be selected for each of the shards when reading. You can configure the algorithm for load balancing (the preference for which replica to access) see the 'load_balancing' setting.
If the connection with the server is not established, there will be an attempt to connect with a short timeout. If the connection failed, the next replica will be selected, and so on for all the replicas. If the connection attempt failed for all the replicas, the attempt will be repeated the same way, several times.

View File

@ -151,3 +151,34 @@ For a time interval starting at 'StartTime' and continuing for 'Duration' second
For example, `timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]`.
This is necessary for searching for pageviews in the corresponding session.
## formatDateTime(Time, Format\[, Timezone\])
Function formats a Time according given Format string. N.B.: Format is a constant expression, e.g. you can not have multiple formats for single result column.
Supported modifiers for Format:
("Example" column shows formatting result for time `2018-01-02 22:33:44`)
| Modifier | Description | Example |
| ----------- | -------- | --------------- |
|%C|year divided by 100 and truncated to integer (00-99)|20
|%d|day of the month, zero-padded (01-31)|02
|%D|Short MM/DD/YY date, equivalent to %m/%d/%y|01/02/2018|
|%e|day of the month, space-padded ( 1-31)| 2|
|%F|short YYYY-MM-DD date, equivalent to %Y-%m-%d|2018-01-02
|%H|hour in 24h format (00-23)|22|
|%I|hour in 12h format (01-12)|10|
|%j|day of the year (001-366)|002|
|%m|month as a decimal number (01-12)|01|
|%M|minute (00-59)|33|
|%n|new-line character ('\n')||
|%p|AM or PM designation|PM|
|%R|24-hour HH:MM time, equivalent to %H:%M|22:33|
|%S|second (00-59)|44|
|%t|horizontal-tab character ('\t')||
|%T|ISO 8601 time format (HH:MM:SS), equivalent to %H:%M:%S|22:33:44|
|%u|ISO 8601 weekday as number with Monday as 1 (1-7)|2|
|%V|ISO 8601 week number (01-53)|01|
|%w|weekday as a decimal number with Sunday as 0 (0-6)|2|
|%y|Year, last two digits (00-99)|18|
|%Y|Year|2018|
|%%|a % sign|%|

View File

@ -60,12 +60,15 @@ logs - имя кластера в конфигурационном файле с
Имя кластера не должно содержать точки.
В качестве параметров для каждого сервера указываются `host`, `port` и, не обязательно, `user`, `password`:
В качестве параметров для каждого сервера указываются `host`, `port` и, не обязательно, `user`, `password`, `secure`, `compression`:
: - `host` - адрес удалённого сервера. Может быть указан домен, или IPv4 или IPv6 адрес. В случае указания домена, при старте сервера делается DNS запрос, и результат запоминается на всё время работы сервера. Если DNS запрос неуспешен, то сервер не запускается. Если вы изменяете DNS-запись, перезапустите сервер.
- `port` - TCP-порт для межсерверного взаимодействия (в конфиге - tcp_port, обычно 9000). Не перепутайте с http_port.
- `user` - имя пользователя для соединения с удалённым сервером. по умолчанию - default. Этот пользователь должен иметь доступ для соединения с указанным сервером. Доступы настраиваются в файле users.xml, подробнее смотрите в разделе "Права доступа".
- `password` - пароль для соединения с удалённым сервером, в открытом виде. по умолчанию - пустая строка.
- `secure` - Использовать шифрованое соединение ssl, Обычно используется с портом `port` = 9440. Сервер должен слушать порт <tcp_port_secure>9440</tcp_port_secure> с корректными настройками сертификатов.
- `compression` - Использовать сжатие данных. По умолчанию: true.
При указании реплик, для каждого из шардов, при чтении, будет выбрана одна из доступных реплик. Можно настроить алгоритм балансировки нагрузки (то есть, предпочтения, на какую из реплик идти) - см. настройку load_balancing.
Если соединение с сервером не установлено, то будет произведена попытка соединения с небольшим таймаутом. Если соединиться не удалось, то будет выбрана следующая реплика, и так для всех реплик. Если попытка соединения для всех реплик не удалась, то будут снова произведены попытки соединения по кругу, и так несколько раз.

View File

@ -122,3 +122,34 @@ SELECT
Для интервала времени, начинающегося в StartTime и продолжающегося Duration секунд, возвращает массив моментов времени, состоящий из округлений вниз до получаса точек из этого интервала.
Например, `timeSlots(toDateTime('2012-01-01 12:20:00'), toUInt32(600)) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]`.
Это нужно для поиска хитов, входящих в соответствующий визит.
## formatDateTime(Time, Format\[, Timezone\])
Функция преобразования даты-с-временем в String согласно заданному шаблону. Важно - шаблон является константным выражением, т.е. невозможно использование разных шаблонов в одной колонке.
Поддерживаемые модификаторы в шаблоне Format:
(колонка "Пример" показана для времени `2018-01-02 22:33:44`)
| Модификатор | Описание | Пример |
| ----------- | -------- | --------------- |
|%C|номер года, поделённый на 100 (00-99)|20
|%d|день месяца, с ведущим нулём (01-31)|02
|%D|короткая запись %m/%d/%y|01/02/2018|
|%e|день месяца, с ведущим пробелом ( 1-31)| 2|
|%F|короткая запись %Y-%m-%d|2018-01-02
|%H|час в 24-часовом формате (00-23)|22|
|%I|час в 12-часовом формате (01-12)|10|
|%j|номер дня в году, с ведущими нулями (001-366)|002|
|%m|месяц, с ведущим нулём (01-12)|01|
|%M|минуты, с ведущим нулём (00-59)|33|
|%n|символ переноса строки ('\n')||
|%p|обозначения AM или PM|PM|
|%R|короткая запись %H:%M|22:33|
|%S|секунды, с ведущими нулями (00-59)|44|
|%t|символ табуляции ('\t')||
|%T|формат времени ISO 8601, одинаковый с %H:%M:%S|22:33:44|
|%u|номер дня недели согласно ISO 8601, понедельник - 1, воскресенье - 7|2|
|%V|номер недели согласно ISO 8601 (01-53)|01|
|%w|номер дня недели, начиная с воскресенья (0-6)|2|
|%y|год, последние 2 цифры (00-99)|18|
|%Y|год, 4 цифры|2018|
|%%|символ %|%|

View File

@ -34,8 +34,6 @@ cd $CURDIR
source "./release_lib.sh"
DEB_CC=${DEB_CC:=gcc-7}
DEB_CXX=${DEB_CXX:=g++-7}
PBUILDER_AUTOUPDATE=${PBUILDER_AUTOUPDATE=4320}
DEBUILD_NOSIGN_OPTIONS="-us -uc"
@ -110,6 +108,8 @@ echo -e "\nCurrent version is $VERSION_STRING"
gen_changelog "$VERSION_STRING" "" "$AUTHOR" ""
if [ -z "$USE_PBUILDER" ] ; then
DEB_CC=${DEB_CC:=`which gcc-7 gcc-8 gcc | head -n1`}
DEB_CXX=${DEB_CXX:=`which g++-7 g++-8 g++ | head -n1`}
# Build (only binary packages).
debuild -e PATH -e SSH_AUTH_SOCK \
-e DEB_CC=$DEB_CC -e DEB_CXX=$DEB_CXX -e CMAKE_FLAGS="$CMAKE_FLAGS" \