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.
*/
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()))

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 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;

View File

@ -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);
}

View File

@ -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)
{

View File

@ -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)

View File

@ -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

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
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.

View File

@ -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