mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge branch 'master' of github.com:yandex/ClickHouse
This commit is contained in:
commit
9381af3bbb
@ -302,40 +302,64 @@ struct ConvertImplGenericToString
|
||||
|
||||
/** 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);
|
||||
}
|
||||
|
||||
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);
|
||||
readDateText(tmp, rb);
|
||||
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;
|
||||
readDateTimeText(tmp, rb, *time_zone);
|
||||
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;
|
||||
readText(tmp, rb);
|
||||
x = tmp;
|
||||
}
|
||||
|
||||
|
||||
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>)
|
||||
return tryReadIntText(x, rb);
|
||||
else if constexpr (std::is_floating_point_v<typename DataType::FieldType>)
|
||||
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
|
||||
{
|
||||
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)
|
||||
@ -861,14 +885,11 @@ public:
|
||||
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
||||
|
||||
if (checkAndGetDataType<DataTypeString>(from_type))
|
||||
ConvertThroughParsing<DataTypeString, ToDataType, Name, exception_mode, parsing_mode>::execute(block,
|
||||
arguments,
|
||||
result, input_rows_count);
|
||||
ConvertThroughParsing<DataTypeString, ToDataType, Name, exception_mode, parsing_mode>::execute(
|
||||
block, arguments, result, input_rows_count);
|
||||
else if (checkAndGetDataType<DataTypeFixedString>(from_type))
|
||||
ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, exception_mode, parsing_mode>::execute(block,
|
||||
arguments,
|
||||
result,
|
||||
input_rows_count);
|
||||
ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, exception_mode, parsing_mode>::execute(
|
||||
block, arguments, result, input_rows_count);
|
||||
else
|
||||
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'.",
|
||||
@ -1231,6 +1252,9 @@ private:
|
||||
const char * name;
|
||||
};
|
||||
|
||||
|
||||
struct NameCast { static constexpr auto name = "CAST"; };
|
||||
|
||||
class FunctionCast final : public IFunctionBase
|
||||
{
|
||||
public:
|
||||
@ -1274,11 +1298,33 @@ private:
|
||||
DataTypePtr return_type;
|
||||
|
||||
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
|
||||
{
|
||||
@ -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
|
||||
{
|
||||
/// Determine whether pre-processing and/or post-processing must take place during conversion.
|
||||
|
||||
NullableConversion nullable_conversion;
|
||||
nullable_conversion.source_is_nullable = from_type->isNullable();
|
||||
nullable_conversion.result_is_nullable = to_type->isNullable();
|
||||
bool source_is_nullable = from_type->isNullable();
|
||||
bool result_is_nullable = to_type->isNullable();
|
||||
|
||||
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};
|
||||
|
||||
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);
|
||||
DataTypePtr to_inner_type = removeNullable(to_type);
|
||||
auto wrapper = prepareImpl(removeNullable(from_type), removeNullable(to_type), result_is_nullable);
|
||||
|
||||
auto wrapper = prepareImpl(from_inner_type, to_inner_type);
|
||||
|
||||
if (nullable_conversion.result_is_nullable)
|
||||
if (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.
|
||||
auto & res = block.getByPosition(result);
|
||||
@ -1573,7 +1609,7 @@ private:
|
||||
const auto & nested_type = nullable_type.getNestedType();
|
||||
|
||||
Block tmp_block;
|
||||
if (nullable_conversion.source_is_nullable)
|
||||
if (source_is_nullable)
|
||||
tmp_block = createBlockWithNestedColumns(block, arguments);
|
||||
else
|
||||
tmp_block = block;
|
||||
@ -1584,29 +1620,12 @@ private:
|
||||
/// Perform the requested conversion.
|
||||
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);
|
||||
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.
|
||||
|
||||
@ -1632,38 +1651,39 @@ private:
|
||||
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))
|
||||
return createIdentityWrapper(from_type);
|
||||
else if (checkDataType<DataTypeNothing>(from_type.get()))
|
||||
return createNothingWrapper(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()))
|
||||
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()))
|
||||
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()))
|
||||
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()))
|
||||
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()))
|
||||
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()))
|
||||
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()))
|
||||
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()))
|
||||
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()))
|
||||
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()))
|
||||
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()))
|
||||
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()))
|
||||
return createWrapper(from_type, to_actual_type);
|
||||
return createStringWrapper(from_type);
|
||||
else if (const auto type_fixed_string = checkAndGetDataType<DataTypeFixedString>(to_type.get()))
|
||||
return createFixedStringWrapper(from_type, type_fixed_string->getN());
|
||||
else if (const auto type_array = checkAndGetDataType<DataTypeArray>(to_type.get()))
|
||||
|
@ -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 result_null_map_column;
|
||||
@ -101,6 +94,9 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, const Block & block, const Colum
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
struct NullPresence
|
||||
{
|
||||
bool has_nullable = false;
|
||||
|
@ -448,4 +448,10 @@ private:
|
||||
|
||||
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);
|
||||
|
||||
}
|
||||
|
@ -18,6 +18,8 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
|
||||
extern const int CANNOT_PARSE_ESCAPE_SEQUENCE;
|
||||
extern const int CANNOT_PARSE_QUOTED_STRING;
|
||||
extern const int CANNOT_PARSE_DATETIME;
|
||||
extern const int CANNOT_PARSE_DATE;
|
||||
extern const int INCORRECT_DATA;
|
||||
}
|
||||
|
||||
@ -255,7 +257,7 @@ static ReturnType parseJSONEscapeSequence(Vector & s, ReadBuffer & buf)
|
||||
|
||||
auto error = [](const char * message, int code)
|
||||
{
|
||||
if (throw_exception)
|
||||
if constexpr (throw_exception)
|
||||
throw Exception(message, code);
|
||||
return ReturnType(false);
|
||||
};
|
||||
@ -593,7 +595,7 @@ ReturnType readJSONStringInto(Vector & s, ReadBuffer & buf)
|
||||
|
||||
auto error = [](const char * message, int code)
|
||||
{
|
||||
if (throw_exception)
|
||||
if constexpr (throw_exception)
|
||||
throw Exception(message, code);
|
||||
return ReturnType(false);
|
||||
};
|
||||
@ -636,38 +638,77 @@ template bool readJSONStringInto<PaddedPODArray<UInt8>, bool>(PaddedPODArray<UIn
|
||||
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];
|
||||
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');
|
||||
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
|
||||
|
||||
buf.ignore();
|
||||
|
||||
char chars_month[2];
|
||||
readPODBinary(chars_month, buf);
|
||||
UInt8 month = chars_month[0] - '0';
|
||||
if (isNumericASCII(chars_month[1]))
|
||||
auto error = []
|
||||
{
|
||||
month = month * 10 + chars_month[1] - '0';
|
||||
buf.ignore();
|
||||
}
|
||||
if constexpr (throw_exception)
|
||||
throw Exception("Cannot parse date: value is too short", ErrorCodes::CANNOT_PARSE_DATE);
|
||||
return ReturnType(false);
|
||||
};
|
||||
|
||||
char char_day;
|
||||
readChar(char_day, buf);
|
||||
UInt8 day = char_day - '0';
|
||||
if (!buf.eof() && isNumericASCII(*buf.position()))
|
||||
auto ignore_delimiter = [&]
|
||||
{
|
||||
day = day * 10 + *buf.position() - '0';
|
||||
++buf.position();
|
||||
}
|
||||
if (!buf.eof())
|
||||
{
|
||||
++buf.position();
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
};
|
||||
|
||||
auto append_digit = [&](auto & x)
|
||||
{
|
||||
if (!buf.eof() && isNumericASCII(*buf.position()))
|
||||
{
|
||||
x = x * 10 + (*buf.position() - '0');
|
||||
++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);
|
||||
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 UNIX_TIMESTAMP_MAX_LENGTH = 10;
|
||||
|
||||
@ -690,7 +731,11 @@ void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUT
|
||||
if (remaining_size != size)
|
||||
{
|
||||
s_pos[size] = 0;
|
||||
throw Exception(std::string("Cannot parse datetime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME);
|
||||
|
||||
if constexpr (throw_exception)
|
||||
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');
|
||||
@ -707,9 +752,30 @@ void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUT
|
||||
datetime = date_lut.makeDateTime(year, month, day, hour, minute, second);
|
||||
}
|
||||
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)
|
||||
{
|
||||
|
@ -246,7 +246,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
|
||||
x = 0;
|
||||
if (buf.eof())
|
||||
{
|
||||
if (throw_exception)
|
||||
if constexpr (throw_exception)
|
||||
throwReadAfterEOF();
|
||||
else
|
||||
return ReturnType(false);
|
||||
@ -263,7 +263,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
|
||||
negative = true;
|
||||
else
|
||||
{
|
||||
if (throw_exception)
|
||||
if constexpr (throw_exception)
|
||||
throw Exception("Unsigned type must not contain '-' symbol", ErrorCodes::CANNOT_PARSE_NUMBER);
|
||||
else
|
||||
return ReturnType(false);
|
||||
@ -459,12 +459,14 @@ template <typename IteratorSrc, typename IteratorDst>
|
||||
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.
|
||||
/// For convenience, Month and Day parts can have single digit instead of two digits.
|
||||
/// 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.
|
||||
if (buf.position() + 10 <= buf.buffer().end())
|
||||
@ -491,16 +493,47 @@ inline void readDateText(LocalDate & date, ReadBuffer & buf)
|
||||
buf.position() += 1;
|
||||
|
||||
date = LocalDate(year, month, day);
|
||||
return ReturnType(true);
|
||||
}
|
||||
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)
|
||||
{
|
||||
LocalDate local_date;
|
||||
readDateText(local_date, buf);
|
||||
date = DateLUT::instance().makeDayNum(local_date.year(), local_date.month(), local_date.day());
|
||||
readDateTextImpl<void>(date, buf);
|
||||
}
|
||||
|
||||
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);
|
||||
|
||||
|
||||
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.
|
||||
* 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.
|
||||
* 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);
|
||||
|
||||
buf.position() += 19;
|
||||
return ReturnType(true);
|
||||
}
|
||||
else
|
||||
/// 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
|
||||
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)
|
||||
|
@ -1,6 +1,6 @@
|
||||
# http://clang.llvm.org/extra/clang-tidy/
|
||||
# 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
|
||||
|
||||
|
@ -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
|
@ -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']));
|
||||
|
@ -0,0 +1,12 @@
|
||||
\N
|
||||
|
||||
123
|
||||
|
||||
123
|
||||
\N
|
||||
\N
|
||||
\N
|
||||
123
|
||||
123
|
||||
\N
|
||||
\N
|
@ -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;
|
@ -34,7 +34,7 @@ sudo apt-get install gcc-7 g++-7
|
||||
|
||||
### 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
|
||||
|
||||
@ -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.
|
||||
|
||||
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.
|
||||
For production releases, GCC is used.
|
||||
|
@ -596,6 +596,10 @@ OPTIMIZE TABLE [db.]name [PARTITION partition] [FINAL]
|
||||
Если указан `PARTITION`, то оптимизация будет производиться только для указаной партиции.
|
||||
Если указан `FINAL`, то оптимизация будет производиться даже когда все данные уже лежат в одном куске.
|
||||
|
||||
<div class="admonition warning">
|
||||
Запрос OPTIMIZE не может устранить причину появления ошибки "Too many parts".
|
||||
</div>
|
||||
|
||||
<a name="queries-insert"></a>
|
||||
|
||||
## INSERT
|
||||
|
Loading…
Reference in New Issue
Block a user