Merge branch 'master' into decimal-too-large-negative-exponent

This commit is contained in:
Alexey Milovidov 2020-08-02 22:48:26 +03:00
commit 0ce54187e1
16 changed files with 92 additions and 37 deletions

View File

@ -16,4 +16,4 @@ ClickHouse is an open-source column-oriented database management system that all
## Upcoming Events
* [ClickHouse at ByteDance (in Chinese)](https://mp.weixin.qq.com/s/Em-HjPylO8D7WPui4RREAQ) on July 31, 2020.
* [ClickHouse at ByteDance (in Chinese)](https://mp.weixin.qq.com/s/Em-HjPylO8D7WPui4RREAQ) on August 14, 2020.

View File

@ -54,7 +54,7 @@ endif ()
# Example: DwarfInstructions.hpp: register unsigned long long x16 __asm("x16") = cfa;
check_cxx_compiler_flag(-Wregister HAVE_WARNING_REGISTER)
if (HAVE_WARNING_REGISTER)
target_compile_options(unwind PRIVATE -Wno-register)
target_compile_options(unwind PRIVATE "$<$<STREQUAL:$<TARGET_PROPERTY:LANGUAGE>,CXX>:-Wno-register>")
endif ()
install(

View File

@ -780,8 +780,10 @@ void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & argu
ArrayImpl::NullMapBuilder builder;
Block source_block;
const auto & input_type = typeid_cast<const DataTypeNullable &>(*typeid_cast<const DataTypeArray &>(*block.getByPosition(arguments[0]).type).getNestedType()).getNestedType();
const auto & tmp_ret_type = typeid_cast<const DataTypeNullable &>(*block.getByPosition(result).type).getNestedType();
const DataTypePtr & input_type = typeid_cast<const DataTypeNullable &>(
*typeid_cast<const DataTypeArray &>(*block.getByPosition(arguments[0]).type).getNestedType()).getNestedType();
DataTypePtr tmp_ret_type = removeNullable(block.getByPosition(result).type);
if (col_array)
{

View File

@ -59,24 +59,39 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
template <typename LonAndLatType, typename PrecisionType>
void execute(const IColumn * lon_min_column,
const IColumn * lat_min_column,
const IColumn * lon_max_column,
const IColumn * lat_max_column,
const IColumn * precision_column,
ColumnPtr & result) const
void execute(
const IColumn * lon_min_column,
const IColumn * lat_min_column,
const IColumn * lon_max_column,
const IColumn * lat_max_column,
const IColumn * precision_column,
ColumnPtr & result,
size_t input_rows_count) const
{
static constexpr size_t max_array_size = 10'000'000;
const auto * lon_min_const = typeid_cast<const ColumnConst *>(lon_min_column);
const auto * lat_min_const = typeid_cast<const ColumnConst *>(lat_min_column);
const auto * lon_max_const = typeid_cast<const ColumnConst *>(lon_max_column);
const auto * lat_max_const = typeid_cast<const ColumnConst *>(lat_max_column);
const auto * precision_const = typeid_cast<const ColumnConst *>(precision_column);
if (lon_min_const)
lon_min_column = &lon_min_const->getDataColumn();
if (lat_min_const)
lat_min_column = &lat_min_const->getDataColumn();
if (lon_max_const)
lon_max_column = &lon_max_const->getDataColumn();
if (lat_max_const)
lat_max_column = &lat_max_const->getDataColumn();
if (precision_const)
precision_column = &precision_const->getDataColumn();
const auto * lon_min = checkAndGetColumn<ColumnVector<LonAndLatType>>(lon_min_column);
const auto * lat_min = checkAndGetColumn<ColumnVector<LonAndLatType>>(lat_min_column);
const auto * lon_max = checkAndGetColumn<ColumnVector<LonAndLatType>>(lon_max_column);
const auto * lat_max = checkAndGetColumn<ColumnVector<LonAndLatType>>(lat_max_column);
auto * precision = checkAndGetColumn<ColumnVector<PrecisionType>>(precision_column);
if (precision == nullptr)
{
precision = checkAndGetColumnConstData<ColumnVector<PrecisionType>>(precision_column);
}
const auto * precision = checkAndGetColumn<ColumnVector<PrecisionType>>(precision_column);
if (!lon_min || !lat_min || !lon_max || !lat_max || !precision)
{
@ -88,24 +103,24 @@ public:
ErrorCodes::LOGICAL_ERROR);
}
const size_t total_rows = lat_min->size();
auto col_res = ColumnArray::create(ColumnString::create());
ColumnString & res_strings = typeid_cast<ColumnString &>(col_res->getData());
ColumnArray::Offsets & res_offsets = col_res->getOffsets();
ColumnString::Chars & res_strings_chars = res_strings.getChars();
ColumnString::Offsets & res_strings_offsets = res_strings.getOffsets();
for (size_t row = 0; row < total_rows; ++row)
for (size_t row = 0; row < input_rows_count; ++row)
{
const Float64 lon_min_value = lon_min->getElement(row);
const Float64 lat_min_value = lat_min->getElement(row);
const Float64 lon_max_value = lon_max->getElement(row);
const Float64 lat_max_value = lat_max->getElement(row);
const Float64 lon_min_value = lon_min->getElement(lon_min_const ? 0 : row);
const Float64 lat_min_value = lat_min->getElement(lat_min_const ? 0 : row);
const Float64 lon_max_value = lon_max->getElement(lon_max_const ? 0 : row);
const Float64 lat_max_value = lat_max->getElement(lat_max_const ? 0 : row);
const PrecisionType precision_value = precision->getElement(precision_const ? 0 : row);
const auto prepared_args = geohashesInBoxPrepare(
lon_min_value, lat_min_value, lon_max_value, lat_max_value,
precision->getElement(row % precision->size()));
lon_min_value, lat_min_value, lon_max_value, lat_max_value,
precision_value);
if (prepared_args.items_count > max_array_size)
{
throw Exception(getName() + " would produce " + std::to_string(prepared_args.items_count) +
@ -123,8 +138,9 @@ public:
for (UInt64 i = 1; i <= prepared_args.items_count ; ++i)
res_strings_offsets.push_back(starting_offset + (prepared_args.precision + 1) * i);
res_offsets.push_back((res_offsets.empty() ? 0 : res_offsets.back()) + prepared_args.items_count);
res_offsets.push_back(res_offsets.back() + prepared_args.items_count);
}
if (!res_strings_offsets.empty() && res_strings_offsets.back() != res_strings_chars.size())
{
throw Exception("String column size mismatch (internal logical error)", ErrorCodes::LOGICAL_ERROR);
@ -140,23 +156,19 @@ public:
result = std::move(col_res);
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const IColumn * lon_min = block.getByPosition(arguments[0]).column.get();
const IColumn * lat_min = block.getByPosition(arguments[1]).column.get();
const IColumn * lon_max = block.getByPosition(arguments[2]).column.get();
const IColumn * lat_max = block.getByPosition(arguments[3]).column.get();
const IColumn * prec = block.getByPosition(arguments[4]).column.get();
const IColumn * precision = block.getByPosition(arguments[4]).column.get();
ColumnPtr & res = block.getByPosition(result).column;
if (checkColumn<ColumnVector<Float32>>(lon_min))
{
execute<Float32, UInt8>(lon_min, lat_min, lon_max, lat_max, prec, res);
}
execute<Float32, UInt8>(lon_min, lat_min, lon_max, lat_max, precision, res, input_rows_count);
else
{
execute<Float64, UInt8>(lon_min, lat_min, lon_max, lat_max, prec, res);
}
execute<Float64, UInt8>(lon_min, lat_min, lon_max, lat_max, precision, res, input_rows_count);
}
};

View File

@ -553,7 +553,7 @@ ReturnType parseDateTimeBestEffortImpl(
};
if (!check_date(is_leap_year, month, day_of_month))
return on_error("Cannot read DateTime: logical error, unexpected date: " + std::to_string(year) + "-" + std::to_string(month) + "-" + std::to_string(day_of_month), ErrorCodes::LOGICAL_ERROR);
return on_error("Cannot read DateTime: unexpected date: " + std::to_string(year) + "-" + std::to_string(month) + "-" + std::to_string(day_of_month), ErrorCodes::CANNOT_PARSE_DATETIME);
if (is_pm && hour < 12)
hour += 12;

View File

@ -161,6 +161,8 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, co
{
if (num_columns == 1)
{
/// One column at the left of IN.
Field value = extractValueFromNode(elem, *types[0], context);
if (!value.isNull() || context.getSettingsRef().transform_null_in)
@ -168,15 +170,20 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, co
}
else if (elem->as<ASTFunction>() || elem->as<ASTLiteral>())
{
/// Multiple columns at the left of IN.
/// The right hand side of in should be a set of tuples.
Field function_result;
const Tuple * tuple = nullptr;
/// Tuple can be represented as a function in AST.
auto * func = elem->as<ASTFunction>();
if (func && func->name != "tuple")
{
if (!tuple_type)
tuple_type = std::make_shared<DataTypeTuple>(types);
/// If the function is not a tuple, treat it as a constant expression that returns tuple and extract it.
function_result = extractValueFromNode(elem, *tuple_type, context);
if (function_result.getType() != Field::Types::Tuple)
throw Exception("Invalid type of set. Expected tuple, got " + String(function_result.getTypeName()),
@ -185,10 +192,12 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, co
tuple = &function_result.get<Tuple>();
}
/// Tuple can be represented as a literal in AST.
auto * literal = elem->as<ASTLiteral>();
if (literal)
{
if (literal->value.getType() != Field::Types::Tuple)
/// The literal must be tuple.
if (literal->value.getType() != Field::Types::Tuple)
throw Exception("Invalid type in set. Expected tuple, got "
+ String(literal->value.getTypeName()), ErrorCodes::INCORRECT_ELEMENT_OF_SET);
@ -203,13 +212,15 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, co
if (tuple_values.empty())
tuple_values.resize(tuple_size);
/// Fill tuple values by evaluation of constant expressions.
size_t i = 0;
for (; i < tuple_size; ++i)
{
Field value = tuple ? (*tuple)[i]
Field value = tuple ? convertFieldToType((*tuple)[i], *types[i])
: extractValueFromNode(func->arguments->children[i], *types[i], context);
/// If at least one of the elements of the tuple has an impossible (outside the range of the type) value, then the entire tuple too.
/// If at least one of the elements of the tuple has an impossible (outside the range of the type) value,
/// then the entire tuple too.
if (value.isNull() && !context.getSettings().transform_null_in)
break;

View File

@ -0,0 +1,4 @@
[1] 1
[1] 1
[1] 1
[1] 1

View File

@ -0,0 +1,4 @@
SELECT [toNullable(1)] AS x, x[toNullable(1)] AS y;
SELECT materialize([toNullable(1)]) AS x, x[toNullable(1)] AS y;
SELECT [toNullable(1)] AS x, x[materialize(toNullable(1))] AS y;
SELECT materialize([toNullable(1)]) AS x, x[materialize(toNullable(1))] AS y;

View File

@ -0,0 +1 @@
SELECT (1, 2) IN ((1, (2, 3)), (1 + 1, 1)); -- { serverError 53 }

View File

@ -0,0 +1,3 @@
SELECT [1, NULL][toNullable(1)];
SELECT [toNullable(1)][toNullable(1)];
SELECT [NULL][toNullable(1)];

View File

@ -0,0 +1,2 @@
select parseDateTime64BestEffort('2.55'); -- { serverError 41 }
select parseDateTime64BestEffortOrNull('2.55');

View File

@ -0,0 +1,6 @@
['s']
['s0']
['s02','s08','s03','s09','s06','s0d']
['s']
['s0']
['s02','s08','s03','s09','s06','s0d']

View File

@ -0,0 +1,6 @@
SELECT geohashesInBox(1., 2., 3., 4., 1);
SELECT geohashesInBox(materialize(1.), 2., 3., 4., 2);
SELECT geohashesInBox(1., materialize(2.), 3., 4., 3);
SELECT geohashesInBox(1., 2., materialize(3.), 4., 1);
SELECT geohashesInBox(1., 2., 3., materialize(4.), 2);
SELECT geohashesInBox(1., 2., 3., 4., materialize(3));