Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2018-07-16 03:33:21 +03:00
commit 9381af3bbb
12 changed files with 335 additions and 118 deletions

View File

@ -302,40 +302,64 @@ struct ConvertImplGenericToString
/** Conversion of strings to numbers, dates, datetimes: through parsing. /** Conversion of strings to numbers, dates, datetimes: through parsing.
*/ */
template <typename DataType> void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) template <typename DataType>
void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
{ {
readText(x, rb); readText(x, rb);
} }
template <> inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) template <>
inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
{ {
DayNum tmp(0); DayNum tmp(0);
readDateText(tmp, rb); readDateText(tmp, rb);
x = tmp; x = tmp;
} }
template <> inline void parseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) template <>
inline void parseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
{ {
time_t tmp = 0; time_t tmp = 0;
readDateTimeText(tmp, rb, *time_zone); readDateTimeText(tmp, rb, *time_zone);
x = tmp; x = tmp;
} }
template <> inline void parseImpl<DataTypeUUID>(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) template <>
inline void parseImpl<DataTypeUUID>(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
{ {
UUID tmp; UUID tmp;
readText(tmp, rb); readText(tmp, rb);
x = tmp; x = tmp;
} }
template <typename DataType> template <typename DataType>
bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb) bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
{ {
if constexpr (std::is_integral_v<typename DataType::FieldType>) if constexpr (std::is_integral_v<typename DataType::FieldType>)
return tryReadIntText(x, rb); return tryReadIntText(x, rb);
else if constexpr (std::is_floating_point_v<typename DataType::FieldType>) else if constexpr (std::is_floating_point_v<typename DataType::FieldType>)
return tryReadFloatText(x, rb); return tryReadFloatText(x, rb);
/// NOTE Need to implement for Date and DateTime too. }
template <>
inline bool tryParseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
{
DayNum tmp(0);
if (!tryReadDateText(tmp, rb))
return false;
x = tmp;
return true;
}
template <>
inline bool tryParseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
{
time_t tmp = 0;
if (!tryReadDateTimeText(tmp, rb, *time_zone))
return false;
x = tmp;
return true;
} }
@ -475,7 +499,7 @@ struct ConvertThroughParsing
} }
else else
{ {
parsed = tryParseImpl<ToDataType>(vec_to[i], read_buffer) && isAllRead(read_buffer); parsed = tryParseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone) && isAllRead(read_buffer);
} }
if (!parsed) if (!parsed)
@ -861,14 +885,11 @@ public:
const IDataType * from_type = block.getByPosition(arguments[0]).type.get(); const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
if (checkAndGetDataType<DataTypeString>(from_type)) if (checkAndGetDataType<DataTypeString>(from_type))
ConvertThroughParsing<DataTypeString, ToDataType, Name, exception_mode, parsing_mode>::execute(block, ConvertThroughParsing<DataTypeString, ToDataType, Name, exception_mode, parsing_mode>::execute(
arguments, block, arguments, result, input_rows_count);
result, input_rows_count);
else if (checkAndGetDataType<DataTypeFixedString>(from_type)) else if (checkAndGetDataType<DataTypeFixedString>(from_type))
ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, exception_mode, parsing_mode>::execute(block, ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, exception_mode, parsing_mode>::execute(
arguments, block, arguments, result, input_rows_count);
result,
input_rows_count);
else else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName() throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName()
+ ". Only String or FixedString argument is accepted for try-conversion function. For other arguments, use function without 'orZero' or 'orNull'.", + ". Only String or FixedString argument is accepted for try-conversion function. For other arguments, use function without 'orZero' or 'orNull'.",
@ -1231,6 +1252,9 @@ private:
const char * name; const char * name;
}; };
struct NameCast { static constexpr auto name = "CAST"; };
class FunctionCast final : public IFunctionBase class FunctionCast final : public IFunctionBase
{ {
public: public:
@ -1274,11 +1298,33 @@ private:
DataTypePtr return_type; DataTypePtr return_type;
template <typename DataType> template <typename DataType>
WrapperType createWrapper(const DataTypePtr & from_type, const DataType * const) const WrapperType createWrapper(const DataTypePtr & from_type, const DataType * const, bool requested_result_is_nullable) const
{ {
using FunctionType = typename FunctionTo<DataType>::Type; FunctionPtr function;
auto function = FunctionType::create(context); if (requested_result_is_nullable && checkAndGetDataType<DataTypeString>(from_type.get()))
{
/// In case when converting to Nullable type, we apply different parsing rule,
/// that will not throw an exception but return NULL in case of malformed input.
function = FunctionConvertFromString<DataType, NameCast, ConvertFromStringExceptionMode::Null>::create(context);
}
else
function = FunctionTo<DataType>::Type::create(context);
/// Check conversion using underlying function
{
function->getReturnType(ColumnsWithTypeAndName(1, { nullptr, from_type, "" }));
}
return [function] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
function->execute(block, arguments, result, input_rows_count);
};
}
WrapperType createStringWrapper(const DataTypePtr & from_type) const
{
FunctionPtr function = FunctionToString::create(context);
/// Check conversion using underlying function /// Check conversion using underlying function
{ {
@ -1530,24 +1576,16 @@ private:
}; };
} }
/// Actions to be taken when performing a conversion.
struct NullableConversion
{
bool source_is_nullable = false;
bool result_is_nullable = false;
};
WrapperType prepare(const DataTypePtr & from_type, const DataTypePtr & to_type) const WrapperType prepare(const DataTypePtr & from_type, const DataTypePtr & to_type) const
{ {
/// Determine whether pre-processing and/or post-processing must take place during conversion. /// Determine whether pre-processing and/or post-processing must take place during conversion.
NullableConversion nullable_conversion; bool source_is_nullable = from_type->isNullable();
nullable_conversion.source_is_nullable = from_type->isNullable(); bool result_is_nullable = to_type->isNullable();
nullable_conversion.result_is_nullable = to_type->isNullable();
if (from_type->onlyNull()) if (from_type->onlyNull())
{ {
if (!nullable_conversion.result_is_nullable) if (!result_is_nullable)
throw Exception{"Cannot convert NULL to a non-nullable type", ErrorCodes::CANNOT_CONVERT_TYPE}; throw Exception{"Cannot convert NULL to a non-nullable type", ErrorCodes::CANNOT_CONVERT_TYPE};
return [](Block & block, const ColumnNumbers &, const size_t result, size_t input_rows_count) return [](Block & block, const ColumnNumbers &, const size_t result, size_t input_rows_count)
@ -1557,14 +1595,12 @@ private:
}; };
} }
DataTypePtr from_inner_type = removeNullable(from_type); auto wrapper = prepareImpl(removeNullable(from_type), removeNullable(to_type), result_is_nullable);
DataTypePtr to_inner_type = removeNullable(to_type);
auto wrapper = prepareImpl(from_inner_type, to_inner_type); if (result_is_nullable)
if (nullable_conversion.result_is_nullable)
{ {
return [wrapper, nullable_conversion] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) return [wrapper, source_is_nullable]
(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{ {
/// Create a temporary block on which to perform the operation. /// Create a temporary block on which to perform the operation.
auto & res = block.getByPosition(result); auto & res = block.getByPosition(result);
@ -1573,7 +1609,7 @@ private:
const auto & nested_type = nullable_type.getNestedType(); const auto & nested_type = nullable_type.getNestedType();
Block tmp_block; Block tmp_block;
if (nullable_conversion.source_is_nullable) if (source_is_nullable)
tmp_block = createBlockWithNestedColumns(block, arguments); tmp_block = createBlockWithNestedColumns(block, arguments);
else else
tmp_block = block; tmp_block = block;
@ -1584,29 +1620,12 @@ private:
/// Perform the requested conversion. /// Perform the requested conversion.
wrapper(tmp_block, arguments, tmp_res_index, input_rows_count); wrapper(tmp_block, arguments, tmp_res_index, input_rows_count);
/// Wrap the result into a nullable column.
ColumnPtr null_map;
if (nullable_conversion.source_is_nullable)
{
/// This is a conversion from a nullable to a nullable type.
/// So we just keep the null map of the input argument.
const auto & col = block.getByPosition(arguments[0]).column;
const auto & nullable_col = static_cast<const ColumnNullable &>(*col);
null_map = nullable_col.getNullMapColumnPtr();
}
else
{
/// This is a conversion from an ordinary type to a nullable type.
/// So we create a trivial null map.
null_map = ColumnUInt8::create(input_rows_count, 0);
}
const auto & tmp_res = tmp_block.getByPosition(tmp_res_index); const auto & tmp_res = tmp_block.getByPosition(tmp_res_index);
res.column = ColumnNullable::create(tmp_res.column, null_map);
res.column = wrapInNullable(tmp_res.column, Block({block.getByPosition(arguments[0]), tmp_res}), {0}, 1, input_rows_count);
}; };
} }
else if (nullable_conversion.source_is_nullable) else if (source_is_nullable)
{ {
/// Conversion from Nullable to non-Nullable. /// Conversion from Nullable to non-Nullable.
@ -1632,38 +1651,39 @@ private:
return wrapper; return wrapper;
} }
WrapperType prepareImpl(const DataTypePtr & from_type, const DataTypePtr & to_type) const /// 'from_type' and 'to_type' are nested types in case of Nullable. 'requested_result_is_nullable' is true if CAST to Nullable type is requested.
WrapperType prepareImpl(const DataTypePtr & from_type, const DataTypePtr & to_type, bool requested_result_is_nullable) const
{ {
if (from_type->equals(*to_type)) if (from_type->equals(*to_type))
return createIdentityWrapper(from_type); return createIdentityWrapper(from_type);
else if (checkDataType<DataTypeNothing>(from_type.get())) else if (checkDataType<DataTypeNothing>(from_type.get()))
return createNothingWrapper(to_type.get()); return createNothingWrapper(to_type.get());
else if (const auto to_actual_type = checkAndGetDataType<DataTypeUInt8>(to_type.get())) else if (const auto to_actual_type = checkAndGetDataType<DataTypeUInt8>(to_type.get()))
return createWrapper(from_type, to_actual_type); return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeUInt16>(to_type.get())) else if (const auto to_actual_type = checkAndGetDataType<DataTypeUInt16>(to_type.get()))
return createWrapper(from_type, to_actual_type); return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeUInt32>(to_type.get())) else if (const auto to_actual_type = checkAndGetDataType<DataTypeUInt32>(to_type.get()))
return createWrapper(from_type, to_actual_type); return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeUInt64>(to_type.get())) else if (const auto to_actual_type = checkAndGetDataType<DataTypeUInt64>(to_type.get()))
return createWrapper(from_type, to_actual_type); return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeInt8>(to_type.get())) else if (const auto to_actual_type = checkAndGetDataType<DataTypeInt8>(to_type.get()))
return createWrapper(from_type, to_actual_type); return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeInt16>(to_type.get())) else if (const auto to_actual_type = checkAndGetDataType<DataTypeInt16>(to_type.get()))
return createWrapper(from_type, to_actual_type); return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeInt32>(to_type.get())) else if (const auto to_actual_type = checkAndGetDataType<DataTypeInt32>(to_type.get()))
return createWrapper(from_type, to_actual_type); return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeInt64>(to_type.get())) else if (const auto to_actual_type = checkAndGetDataType<DataTypeInt64>(to_type.get()))
return createWrapper(from_type, to_actual_type); return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeFloat32>(to_type.get())) else if (const auto to_actual_type = checkAndGetDataType<DataTypeFloat32>(to_type.get()))
return createWrapper(from_type, to_actual_type); return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeFloat64>(to_type.get())) else if (const auto to_actual_type = checkAndGetDataType<DataTypeFloat64>(to_type.get()))
return createWrapper(from_type, to_actual_type); return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeDate>(to_type.get())) else if (const auto to_actual_type = checkAndGetDataType<DataTypeDate>(to_type.get()))
return createWrapper(from_type, to_actual_type); return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeDateTime>(to_type.get())) else if (const auto to_actual_type = checkAndGetDataType<DataTypeDateTime>(to_type.get()))
return createWrapper(from_type, to_actual_type); return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeString>(to_type.get())) else if (const auto to_actual_type = checkAndGetDataType<DataTypeString>(to_type.get()))
return createWrapper(from_type, to_actual_type); return createStringWrapper(from_type);
else if (const auto type_fixed_string = checkAndGetDataType<DataTypeFixedString>(to_type.get())) else if (const auto type_fixed_string = checkAndGetDataType<DataTypeFixedString>(to_type.get()))
return createFixedStringWrapper(from_type, type_fixed_string->getN()); return createFixedStringWrapper(from_type, type_fixed_string->getN());
else if (const auto type_array = checkAndGetDataType<DataTypeArray>(to_type.get())) else if (const auto type_array = checkAndGetDataType<DataTypeArray>(to_type.get()))

View File

@ -33,13 +33,6 @@ namespace ErrorCodes
} }
namespace
{
/** Return ColumnNullable of src, with null map as OR-ed null maps of args columns in blocks.
* Or ColumnConst(ColumnNullable) if the result is always NULL or if the result is constant and always not NULL.
*/
ColumnPtr wrapInNullable(const ColumnPtr & src, const Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) ColumnPtr wrapInNullable(const ColumnPtr & src, const Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
{ {
ColumnPtr result_null_map_column; ColumnPtr result_null_map_column;
@ -101,6 +94,9 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, const Block & block, const Colum
} }
namespace
{
struct NullPresence struct NullPresence
{ {
bool has_nullable = false; bool has_nullable = false;

View File

@ -448,4 +448,10 @@ private:
using FunctionPtr = std::shared_ptr<IFunction>; using FunctionPtr = std::shared_ptr<IFunction>;
/** Return ColumnNullable of src, with null map as OR-ed null maps of args columns in blocks.
* Or ColumnConst(ColumnNullable) if the result is always NULL or if the result is constant and always not NULL.
*/
ColumnPtr wrapInNullable(const ColumnPtr & src, const Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count);
} }

View File

@ -18,6 +18,8 @@ namespace ErrorCodes
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; extern const int CANNOT_PARSE_ESCAPE_SEQUENCE;
extern const int CANNOT_PARSE_QUOTED_STRING; extern const int CANNOT_PARSE_QUOTED_STRING;
extern const int CANNOT_PARSE_DATETIME;
extern const int CANNOT_PARSE_DATE;
extern const int INCORRECT_DATA; extern const int INCORRECT_DATA;
} }
@ -255,7 +257,7 @@ static ReturnType parseJSONEscapeSequence(Vector & s, ReadBuffer & buf)
auto error = [](const char * message, int code) auto error = [](const char * message, int code)
{ {
if (throw_exception) if constexpr (throw_exception)
throw Exception(message, code); throw Exception(message, code);
return ReturnType(false); return ReturnType(false);
}; };
@ -593,7 +595,7 @@ ReturnType readJSONStringInto(Vector & s, ReadBuffer & buf)
auto error = [](const char * message, int code) auto error = [](const char * message, int code)
{ {
if (throw_exception) if constexpr (throw_exception)
throw Exception(message, code); throw Exception(message, code);
return ReturnType(false); return ReturnType(false);
}; };
@ -636,38 +638,77 @@ template bool readJSONStringInto<PaddedPODArray<UInt8>, bool>(PaddedPODArray<UIn
template void readJSONStringInto<NullSink>(NullSink & s, ReadBuffer & buf); template void readJSONStringInto<NullSink>(NullSink & s, ReadBuffer & buf);
void readDateTextFallback(LocalDate & date, ReadBuffer & buf) template <typename ReturnType>
ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf)
{ {
char chars_year[4]; static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
readPODBinary(chars_year, buf);
UInt16 year = (chars_year[0] - '0') * 1000 + (chars_year[1] - '0') * 100 + (chars_year[2] - '0') * 10 + (chars_year[3] - '0');
buf.ignore(); auto error = []
char chars_month[2];
readPODBinary(chars_month, buf);
UInt8 month = chars_month[0] - '0';
if (isNumericASCII(chars_month[1]))
{ {
month = month * 10 + chars_month[1] - '0'; if constexpr (throw_exception)
buf.ignore(); throw Exception("Cannot parse date: value is too short", ErrorCodes::CANNOT_PARSE_DATE);
return ReturnType(false);
};
auto ignore_delimiter = [&]
{
if (!buf.eof())
{
++buf.position();
return true;
} }
else
return false;
};
char char_day; auto append_digit = [&](auto & x)
readChar(char_day, buf); {
UInt8 day = char_day - '0';
if (!buf.eof() && isNumericASCII(*buf.position())) if (!buf.eof() && isNumericASCII(*buf.position()))
{ {
day = day * 10 + *buf.position() - '0'; x = x * 10 + (*buf.position() - '0');
++buf.position(); ++buf.position();
return true;
} }
else
return false;
};
UInt16 year = 0;
if (!append_digit(year)
|| !append_digit(year)
|| !append_digit(year)
|| !append_digit(year))
return error();
if (!ignore_delimiter())
return error();
UInt8 month = 0;
if (!append_digit(month))
return error();
append_digit(month);
if (!ignore_delimiter())
return error();
UInt8 day = 0;
if (!append_digit(day))
return error();
append_digit(day);
date = LocalDate(year, month, day); date = LocalDate(year, month, day);
return ReturnType(true);
} }
template void readDateTextFallback<void>(LocalDate &, ReadBuffer &);
template bool readDateTextFallback<bool>(LocalDate &, ReadBuffer &);
void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut)
template <typename ReturnType>
ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut)
{ {
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
static constexpr auto DATE_TIME_BROKEN_DOWN_LENGTH = 19; static constexpr auto DATE_TIME_BROKEN_DOWN_LENGTH = 19;
static constexpr auto UNIX_TIMESTAMP_MAX_LENGTH = 10; static constexpr auto UNIX_TIMESTAMP_MAX_LENGTH = 10;
@ -690,7 +731,11 @@ void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUT
if (remaining_size != size) if (remaining_size != size)
{ {
s_pos[size] = 0; s_pos[size] = 0;
if constexpr (throw_exception)
throw Exception(std::string("Cannot parse datetime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME); throw Exception(std::string("Cannot parse datetime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME);
else
return false;
} }
UInt16 year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'); UInt16 year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0');
@ -707,8 +752,29 @@ void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUT
datetime = date_lut.makeDateTime(year, month, day, hour, minute, second); datetime = date_lut.makeDateTime(year, month, day, hour, minute, second);
} }
else else
datetime = parse<time_t>(s, s_pos - s); {
/// Only unix timestamp of 5-10 characters is supported. For consistency. See readDateTimeTextImpl.
if (s_pos - s >= 5 && s_pos - s <= 10)
{
/// Not very efficient.
datetime = 0;
for (const char * digit_pos = s; digit_pos < s_pos; ++digit_pos)
datetime = datetime * 10 + *digit_pos - '0';
} }
else
{
if constexpr (throw_exception)
throw Exception("Cannot parse datetime", ErrorCodes::CANNOT_PARSE_DATETIME);
else
return false;
}
}
return ReturnType(true);
}
template void readDateTimeTextFallback<void>(time_t &, ReadBuffer &, const DateLUTImpl &);
template bool readDateTimeTextFallback<bool>(time_t &, ReadBuffer &, const DateLUTImpl &);
void skipJSONFieldPlain(ReadBuffer & buf, const StringRef & name_of_filed) void skipJSONFieldPlain(ReadBuffer & buf, const StringRef & name_of_filed)

View File

@ -246,7 +246,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
x = 0; x = 0;
if (buf.eof()) if (buf.eof())
{ {
if (throw_exception) if constexpr (throw_exception)
throwReadAfterEOF(); throwReadAfterEOF();
else else
return ReturnType(false); return ReturnType(false);
@ -263,7 +263,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
negative = true; negative = true;
else else
{ {
if (throw_exception) if constexpr (throw_exception)
throw Exception("Unsigned type must not contain '-' symbol", ErrorCodes::CANNOT_PARSE_NUMBER); throw Exception("Unsigned type must not contain '-' symbol", ErrorCodes::CANNOT_PARSE_NUMBER);
else else
return ReturnType(false); return ReturnType(false);
@ -459,12 +459,14 @@ template <typename IteratorSrc, typename IteratorDst>
void formatHex(IteratorSrc src, IteratorDst dst, const size_t num_bytes); void formatHex(IteratorSrc src, IteratorDst dst, const size_t num_bytes);
void readDateTextFallback(LocalDate & date, ReadBuffer & buf); template <typename ReturnType>
ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf);
/// In YYYY-MM-DD format. /// In YYYY-MM-DD format.
/// For convenience, Month and Day parts can have single digit instead of two digits. /// For convenience, Month and Day parts can have single digit instead of two digits.
/// Any separators other than '-' are supported. /// Any separators other than '-' are supported.
inline void readDateText(LocalDate & date, ReadBuffer & buf) template <typename ReturnType = void>
inline ReturnType readDateTextImpl(LocalDate & date, ReadBuffer & buf)
{ {
/// Optimistic path, when whole value is in buffer. /// Optimistic path, when whole value is in buffer.
if (buf.position() + 10 <= buf.buffer().end()) if (buf.position() + 10 <= buf.buffer().end())
@ -491,16 +493,47 @@ inline void readDateText(LocalDate & date, ReadBuffer & buf)
buf.position() += 1; buf.position() += 1;
date = LocalDate(year, month, day); date = LocalDate(year, month, day);
return ReturnType(true);
} }
else else
readDateTextFallback(date, buf); return readDateTextFallback<ReturnType>(date, buf);
}
template <typename ReturnType = void>
inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf)
{
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
LocalDate local_date;
if constexpr (throw_exception)
readDateTextImpl<ReturnType>(local_date, buf);
else if (!readDateTextImpl<ReturnType>(local_date, buf))
return false;
date = DateLUT::instance().makeDayNum(local_date.year(), local_date.month(), local_date.day());
return ReturnType(true);
}
inline void readDateText(LocalDate & date, ReadBuffer & buf)
{
readDateTextImpl<void>(date, buf);
} }
inline void readDateText(DayNum & date, ReadBuffer & buf) inline void readDateText(DayNum & date, ReadBuffer & buf)
{ {
LocalDate local_date; readDateTextImpl<void>(date, buf);
readDateText(local_date, buf); }
date = DateLUT::instance().makeDayNum(local_date.year(), local_date.month(), local_date.day());
inline bool tryReadDateText(LocalDate & date, ReadBuffer & buf)
{
return readDateTextImpl<bool>(date, buf);
}
inline bool tryReadDateText(DayNum & date, ReadBuffer & buf)
{
return readDateTextImpl<bool>(date, buf);
} }
@ -523,12 +556,14 @@ template <typename T>
inline T parse(const char * data, size_t size); inline T parse(const char * data, size_t size);
void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut); template <typename ReturnType = void>
ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut);
/** In YYYY-MM-DD hh:mm:ss format, according to specified time zone. /** In YYYY-MM-DD hh:mm:ss format, according to specified time zone.
* As an exception, also supported parsing of unix timestamp in form of decimal number. * As an exception, also supported parsing of unix timestamp in form of decimal number.
*/ */
inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut) template <typename ReturnType = void>
inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut)
{ {
/** Read 10 characters, that could represent unix timestamp. /** Read 10 characters, that could represent unix timestamp.
* Only unix timestamp of 5-10 characters is supported. * Only unix timestamp of 5-10 characters is supported.
@ -556,18 +591,24 @@ inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTI
datetime = date_lut.makeDateTime(year, month, day, hour, minute, second); datetime = date_lut.makeDateTime(year, month, day, hour, minute, second);
buf.position() += 19; buf.position() += 19;
return ReturnType(true);
} }
else else
/// Why not readIntTextUnsafe? Because for needs of AdFox, parsing of unix timestamp with leading zeros is supported: 000...NNNN. /// Why not readIntTextUnsafe? Because for needs of AdFox, parsing of unix timestamp with leading zeros is supported: 000...NNNN.
readIntText(datetime, buf); return readIntTextImpl<time_t, ReturnType>(datetime, buf);
} }
else else
readDateTimeTextFallback(datetime, buf, date_lut); return readDateTimeTextFallback<ReturnType>(datetime, buf, date_lut);
} }
inline void readDateTimeText(time_t & datetime, ReadBuffer & buf) inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
{ {
readDateTimeText(datetime, buf, DateLUT::instance()); readDateTimeTextImpl<void>(datetime, buf, date_lut);
}
inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
{
return readDateTimeTextImpl<bool>(datetime, buf, date_lut);
} }
inline void readDateTimeText(LocalDateTime & datetime, ReadBuffer & buf) inline void readDateTimeText(LocalDateTime & datetime, ReadBuffer & buf)

View File

@ -1,6 +1,6 @@
# http://clang.llvm.org/extra/clang-tidy/ # http://clang.llvm.org/extra/clang-tidy/
# Install latest clang with extra tools. # Install latest clang with extra tools.
# Look at utils/prepare-environment/install-clang.sh # Look at ci/build-clang-from-sources.sh
sudo apt-get install jq sudo apt-get install jq

View File

@ -0,0 +1,49 @@
\N
\N
123
\N
\N
\N
\N
2018-01-02
2018-01-02
2018-01-02
2018-01-02
\N
\N
\N
\N
2018-01-02 01:02:03
2018-01-02 01:02:03
\N
abc
123
123a
-123
0000-00-00
0000-00-00
2018-01-02
2018-01-02
2018-01-02
2018-01-02
0000-00-00
0000-00-00
\N
\N
2018-01-02
2018-01-02
2018-01-02
2018-01-02
\N
\N
0000-00-00 00:00:00
0000-00-00 00:00:00
2018-01-02 01:02:03
2018-01-02 01:02:03
0000-00-00 00:00:00
\N
\N
2018-01-02 01:02:03
2018-01-02 01:02:03
\N

View File

@ -0,0 +1,11 @@
SELECT CAST(arrayJoin(['', 'abc', '123', '123a', '-123']) AS Nullable(UInt8));
SELECT CAST(arrayJoin(['', '2018', '2018-01-02', '2018-1-2', '2018-01-2', '2018-1-02', '2018-ab-cd', '2018-01-02a']) AS Nullable(Date));
SELECT CAST(arrayJoin(['', '2018', '2018-01-02 01:02:03', '2018-01-02T01:02:03', '2018-01-02 01:02:03 abc']) AS Nullable(DateTime));
SELECT CAST(arrayJoin(['', 'abc', '123', '123a', '-123']) AS Nullable(String));
SELECT toDateOrZero(arrayJoin(['', '2018', '2018-01-02', '2018-1-2', '2018-01-2', '2018-1-02', '2018-ab-cd', '2018-01-02a']));
SELECT toDateOrNull(arrayJoin(['', '2018', '2018-01-02', '2018-1-2', '2018-01-2', '2018-1-02', '2018-ab-cd', '2018-01-02a']));
SELECT toDateTimeOrZero(arrayJoin(['', '2018', '2018-01-02 01:02:03', '2018-01-02T01:02:03', '2018-01-02 01:02:03 abc']));
SELECT toDateTimeOrNull(arrayJoin(['', '2018', '2018-01-02 01:02:03', '2018-01-02T01:02:03', '2018-01-02 01:02:03 abc']));

View File

@ -0,0 +1,12 @@
\N
123
123
\N
\N
\N
123
123
\N
\N

View File

@ -0,0 +1,12 @@
DROP TABLE IF EXISTS test.alter;
CREATE TABLE test.alter (`boolean_false` Nullable(String)) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO test.alter (`boolean_false`) VALUES (NULL), (''), ('123');
SELECT * FROM test.alter;
SELECT * FROM test.alter ORDER BY boolean_false NULLS LAST;
ALTER TABLE test.alter MODIFY COLUMN `boolean_false` Nullable(UInt8);
SELECT * FROM test.alter;
SELECT * FROM test.alter ORDER BY boolean_false NULLS LAST;
DROP TABLE test.alter;

View File

@ -34,7 +34,7 @@ sudo apt-get install gcc-7 g++-7
### Install from sources ### Install from sources
Look at [https://github.com/yandex/ClickHouse/blob/master/utils/prepare-environment/install-gcc.sh] Look at [ci/build-gcc-from-sources.sh](https://github.com/yandex/ClickHouse/blob/master/ci/build-gcc-from-sources.sh)
## Use GCC 7 for builds ## Use GCC 7 for builds
@ -79,7 +79,7 @@ Install the most recent version of Clang.
Clang is embedded into the ClickHouse package and used at runtime. The minimum version is 5.0. It is optional. Clang is embedded into the ClickHouse package and used at runtime. The minimum version is 5.0. It is optional.
To install clang, see `utils/prepare-environment/install-clang.sh` To install clang, see [ci/build-clang-from-sources.sh](https://github.com/yandex/ClickHouse/blob/master/ci/build-clang-from-sources.sh)
You may also build ClickHouse with Clang for development purposes. You may also build ClickHouse with Clang for development purposes.
For production releases, GCC is used. For production releases, GCC is used.

View File

@ -596,6 +596,10 @@ OPTIMIZE TABLE [db.]name [PARTITION partition] [FINAL]
Если указан `PARTITION`, то оптимизация будет производиться только для указаной партиции. Если указан `PARTITION`, то оптимизация будет производиться только для указаной партиции.
Если указан `FINAL`, то оптимизация будет производиться даже когда все данные уже лежат в одном куске. Если указан `FINAL`, то оптимизация будет производиться даже когда все данные уже лежат в одном куске.
<div class="admonition warning">
Запрос OPTIMIZE не может устранить причину появления ошибки "Too many parts".
</div>
<a name="queries-insert"></a> <a name="queries-insert"></a>
## INSERT ## INSERT