Fixed error after merge [#CLICKHOUSE-3150].

This commit is contained in:
Alexey Milovidov 2017-07-21 23:20:22 +03:00
parent 6d47981e86
commit 31d8ba3b34
8 changed files with 40 additions and 41 deletions

View File

@ -2,12 +2,21 @@
#include <IO/WriteBufferFromString.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnNullable.h>
#include <Core/FieldVisitors.h>
#include <Common/typeid_cast.h>
namespace DB
{
bool ColumnConst::isNull() const
{
const ColumnNullable * column_nullable = typeid_cast<const ColumnNullable *>(data.get());
return column_nullable && column_nullable->isNullAt(0);
}
String ColumnConst::dump() const
{
String res;

View File

@ -66,10 +66,7 @@ public:
return false;
}
bool isNull() const override
{
return data->isNullable() && (*data)[0].isNull();
}
bool isNull() const override;
bool isFixed() const override
{

View File

@ -99,7 +99,7 @@ class ArraySource final : public IArraySource<TResult>
{
public:
ArraySource(const PaddedPODArray<TType> & data_, const ColumnArray::Offsets_t & offsets_)
: data{data_}, offsets{offsets_}
: data(data_), offsets(offsets_)
{
}
@ -146,7 +146,7 @@ class ConstArraySource final : public IArraySource<TResult>
{
public:
ConstArraySource(const Array & array_)
: array{array_}
: array(array_)
{
}
@ -190,7 +190,7 @@ class ArraySink
public:
ArraySink(PaddedPODArray<TResult> & data_, ColumnArray::Offsets_t & offsets_,
size_t data_size_, size_t offsets_size_)
: data{data_}, offsets{offsets_}
: data(data_), offsets(offsets_)
{
offsets.resize(offsets_size_);
data.reserve(data_size_);
@ -245,7 +245,7 @@ public:
{
const ColumnVector<TType> * content = typeid_cast<const ColumnVector<TType> *>(&col_array->getData());
if (!content)
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
throw Exception{"Unexpected type of Array column in function multiIf with numeric Array arguments", ErrorCodes::LOGICAL_ERROR};
source = std::make_unique<ArraySource<TResult, TType>>(content->getData(), col_array->getOffsets());
}
else if (auto col_const_array = checkAndGetColumnConst<ColumnArray>(col))
@ -260,8 +260,6 @@ public:
}
};
const Array null_array{Null()};
/// Case for null sources.
template <typename TResult>
class ArraySourceCreator<TResult, Null> final
@ -274,7 +272,7 @@ public:
if (TypeName<Null>::get() == type_name)
{
IArraySourcePtr<TResult> source;
source = std::make_unique<ConstArraySource<TResult, Null>>(null_array);
source = std::make_unique<ConstArraySource<TResult, Null>>(Array());
sources.push_back(std::move(source));
return true;
@ -363,7 +361,7 @@ private:
|| ArraySourceCreator<TResult, Float32>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, Float64>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, Null>::execute(sources, block, args, br)))
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
throw Exception{"Unexpected type of Array column in function multiIf with numeric Array arguments", ErrorCodes::LOGICAL_ERROR};
}
return sources;

View File

@ -10,6 +10,7 @@
#include <Columns/ColumnFixedString.h>
#include <Common/typeid_cast.h>
/// NOTE: this code is quite complicated and ugly because it handles
/// the internals of arrays of strings.
/// Arrays of fixed strings are currently unsupported.
@ -77,8 +78,8 @@ public:
const ColumnString::Offsets_t & string_offsets_,
const ColumnArray::Offsets_t & array_offsets_,
size_t index_)
: data{data_}, string_offsets{string_offsets_}, array_offsets{array_offsets_},
index{index_}
: data(data_), string_offsets(string_offsets_), array_offsets(array_offsets_),
index(index_)
{
}
@ -159,7 +160,7 @@ class ConstStringArraySource : public StringArraySource
{
public:
ConstStringArraySource(const Array & data_, size_t index_)
: data{data_}, index{index_}
: data(data_), index(index_)
{
data_size = 0;
for (const auto & s : data)
@ -221,7 +222,6 @@ private:
return array_size;
};
};
@ -236,7 +236,7 @@ public:
size_t data_size_,
size_t offsets_size_,
size_t row_count)
: data{data_}, string_offsets{string_offsets_}, array_offsets{array_offsets_}
: data(data_), string_offsets(string_offsets_), array_offsets(array_offsets_)
{
array_offsets.resize(row_count);
string_offsets.reserve(offsets_size_);
@ -274,8 +274,6 @@ CondSources createConds(const Block & block, const ColumnNumbers & args)
}
const Array null_array{String()};
/// Create accessors for branch values.
bool createStringArraySources(StringArraySources & sources, const Block & block,
const ColumnNumbers & args)
@ -287,14 +285,7 @@ bool createStringArraySources(StringArraySources & sources, const Block & block,
const ColumnString * var_col = col_arr ? checkAndGetColumn<ColumnString>(&col_arr->getData()) : nullptr;
const ColumnConst * const_col = checkAndGetColumnConst<ColumnArray>(col);
if (col->isNull())
{
StringArraySourcePtr source;
source = std::make_unique<ConstStringArraySource>(null_array, args[i]);
sources.push_back(std::move(source));
return true;
}
else if ((col_arr && var_col) || const_col)
if ((col_arr && var_col) || const_col)
{
StringArraySourcePtr source;
@ -304,7 +295,8 @@ bool createStringArraySources(StringArraySources & sources, const Block & block,
else if (const_col)
source = std::make_unique<ConstStringArraySource>(const_col->getValue<Array>(), args[i]);
else
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
throw Exception{"Unexpected type of column in then or else condition of multiIf function with Array(String) arguments",
ErrorCodes::LOGICAL_ERROR};
sources.push_back(std::move(source));

View File

@ -14,9 +14,6 @@ class NullMapBuilder;
class StringArrayEvaluator
{
public:
/// The tracker parameter is an index to a column that tracks the originating column of each value of
/// the result column. Calling this function with result == tracker means that no such tracking is
/// required, which happens if multiIf is called with no nullable parameters.
static bool perform(Block & block, const ColumnNumbers & args, size_t result, NullMapBuilder & builder);
};

View File

@ -121,7 +121,7 @@ class FixedStringSource final : public StringSource
{
public:
FixedStringSource(const ColumnFixedString::Chars_t & data_, size_t size_, size_t index_)
: data{data_}, size{size_}, index{index_}
: data(data_), size(size_), index(index_)
{
}
@ -174,7 +174,7 @@ class VarStringSource final : public StringSource
public:
VarStringSource(const ColumnString::Chars_t & data_,
const ColumnString::Offsets_t & offsets_, size_t index_)
: data{data_}, offsets{offsets_}, index{index_}
: data(data_), offsets(offsets_), index(index_)
{
}
@ -235,7 +235,7 @@ class FixedStringSink : public StringSink
{
public:
FixedStringSink(ColumnFixedString::Chars_t & data_, size_t size_, size_t data_size_)
: data{data_}, size{size_}
: data(data_), size(size_)
{
data.reserve(data_size_);
}
@ -267,7 +267,7 @@ class VarStringSink : public StringSink
public:
VarStringSink(ColumnString::Chars_t & data_, ColumnString::Offsets_t & offsets_,
size_t data_size_, size_t offsets_size_)
: data{data_}, offsets{offsets_}
: data(data_), offsets(offsets_)
{
offsets.resize(offsets_size_);
data.reserve(data_size_);

View File

@ -1,3 +1,4 @@
Trivial case
2
A
A\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0
@ -6,6 +7,7 @@ A\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0
[]
[]
Numeric branches
1
3
1
@ -6766,6 +6768,7 @@ A\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0
3
1
2
Numeric array branches
[1,2]
[3,3]
[1,2]
@ -13526,6 +13529,7 @@ A\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0
[3,3]
[1,2]
[2,3]
String branches
ABC
B
CD
@ -16086,6 +16090,7 @@ foo
foo
foo
foo
String array branches
['ABCDE','ABCDEF']
['B','BC']
['CDE','CDEF']
@ -18646,6 +18651,7 @@ foo
['foo','bar']
['foo','bar']
['foo','bar']
Miscellaneous
11225 Москва
20273 Москва
213 Мир

View File

@ -1,5 +1,5 @@
/* Trivial case */
SELECT 'Trivial case';
SELECT multiIf(1, 2, 1, 3, 4);
SELECT multiIf(1, 'A', 1, 'BC', 'DEF');
@ -10,7 +10,7 @@ SELECT multiIf(rand() % 2 = 0, emptyArrayString(), emptyArrayString());
SELECT multiIf(rand() % 2 = 0, emptyArrayUInt8(), emptyArrayUInt8());
SELECT multiIf(rand() % 2 = 0, '', '');
/* Numeric branches */
SELECT 'Numeric branches';
SELECT multiIf((number % 2) = 0, toInt8(1), (number % 3) = 0, toInt8(2), toInt8(3)) FROM system.numbers LIMIT 10;
SELECT multiIf((number % 2) = 0, toInt8(1), (number % 3) = 0, toInt8(2), toInt16(3)) FROM system.numbers LIMIT 10;
@ -689,7 +689,7 @@ SELECT multiIf((number % 2) = 0, toFloat64(1), (number % 3) = 0, toFloat64(2), t
SELECT multiIf((number % 2) = 0, toFloat64(1), (number % 3) = 0, toFloat64(2), toFloat32(3)) FROM system.numbers LIMIT 10;
SELECT multiIf((number % 2) = 0, toFloat64(1), (number % 3) = 0, toFloat64(2), toFloat64(3)) FROM system.numbers LIMIT 10;
/* Numeric array branches */
SELECT 'Numeric array branches';
SELECT multiIf((number % 2) = 0, [toInt8(1), toInt8(2)], (number % 3) = 0, [toInt8(2), toInt8(3)], [toInt8(3), toInt8(3)]) FROM system.numbers LIMIT 10;
SELECT multiIf((number % 2) = 0, [toInt8(1), toInt8(2)], (number % 3) = 0, [toInt8(2), toInt8(3)], [toInt16(3), toInt16(3)]) FROM system.numbers LIMIT 10;
@ -1368,7 +1368,7 @@ SELECT multiIf((number % 2) = 0, [toFloat64(1), toFloat64(2)], (number % 3) = 0,
SELECT multiIf((number % 2) = 0, [toFloat64(1), toFloat64(2)], (number % 3) = 0, [toFloat64(2), toFloat64(3)], [toFloat32(3), toFloat32(3)]) FROM system.numbers LIMIT 10;
SELECT multiIf((number % 2) = 0, [toFloat64(1), toFloat64(2)], (number % 3) = 0, [toFloat64(2), toFloat64(3)], [toFloat64(3), toFloat64(3)]) FROM system.numbers LIMIT 10;
/* String branches */
SELECT 'String branches';
DROP TABLE IF EXISTS test.multi_if_check;
CREATE TABLE test.multi_if_check(col1 UInt64, col2 String, col3 String, col4 String) ENGINE=TinyLog;
@ -1642,7 +1642,7 @@ SELECT multiIf(1, 'foo', 1, 'bar', 'baz') FROM test.multi_if_check;
DROP TABLE IF EXISTS test.multi_if_check;
/* String array branches */
SELECT 'String array branches';
CREATE TABLE test.multi_if_check(col1 UInt64, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=TinyLog;
INSERT INTO test.multi_if_check(col1, col2, col3, col4, col5, col6, col7) VALUES(1, 'A', 'AB', 'ABC', 'ABCD', 'ABCDE', 'ABCDEF');
@ -1915,7 +1915,7 @@ SELECT multiIf(1, ['foo', 'bar'], 1, ['foo', 'bar'], ['foo', 'bar']) FROM test.m
DROP TABLE IF EXISTS test.multi_if_check;
/* Miscellaneous */
SELECT 'Miscellaneous';
CREATE TABLE test.multi_if_check(col1 UInt64) ENGINE=TinyLog;
INSERT INTO test.multi_if_check(col1) VALUES (11225),(20273),(213),(240),(12),(187),(29252);