mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
add IColumn.isNullable()
This commit is contained in:
parent
f19dee706a
commit
0c634129da
@ -102,9 +102,4 @@ void ColumnConst::getPermutation(bool /*reverse*/, size_t /*limit*/, int /*nan_d
|
||||
res[i] = i;
|
||||
}
|
||||
|
||||
bool isColumnConst(const IColumn & column)
|
||||
{
|
||||
return checkColumn<ColumnConst>(column);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -197,6 +197,7 @@ public:
|
||||
return false;
|
||||
}
|
||||
|
||||
bool isNullable() const override { return isColumnNullable(*data); }
|
||||
bool onlyNull() const override { return data->isNullAt(0); }
|
||||
bool isNumeric() const override { return data->isNumeric(); }
|
||||
bool isFixedAndContiguous() const override { return data->isFixedAndContiguous(); }
|
||||
@ -216,6 +217,4 @@ public:
|
||||
T getValue() const { return getField().safeGet<NearestFieldType<T>>(); }
|
||||
};
|
||||
|
||||
bool isColumnConst(const IColumn & column);
|
||||
|
||||
}
|
||||
|
@ -146,6 +146,7 @@ public:
|
||||
size_t sizeOfValueIfFixed() const override { return getDictionary().sizeOfValueIfFixed(); }
|
||||
bool isNumeric() const override { return getDictionary().isNumeric(); }
|
||||
bool lowCardinality() const override { return true; }
|
||||
bool isNullable() const override { return isColumnNullable(*dictionary.getColumnUniquePtr()); }
|
||||
|
||||
const IColumnUnique & getDictionary() const { return dictionary.getColumnUnique(); }
|
||||
const ColumnPtr & getDictionaryPtr() const { return dictionary.getColumnUniquePtr(); }
|
||||
|
@ -451,10 +451,9 @@ void ColumnNullable::checkConsistency() const
|
||||
ErrorCodes::SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT);
|
||||
}
|
||||
|
||||
|
||||
ColumnPtr makeNullable(const ColumnPtr & column)
|
||||
{
|
||||
if (checkColumn<ColumnNullable>(*column))
|
||||
if (isColumnNullable(*column))
|
||||
return column;
|
||||
|
||||
if (isColumnConst(*column))
|
||||
@ -463,11 +462,4 @@ ColumnPtr makeNullable(const ColumnPtr & column)
|
||||
return ColumnNullable::create(column, ColumnUInt8::create(column->size(), 0));
|
||||
}
|
||||
|
||||
bool isNullable(const IColumn & column)
|
||||
{
|
||||
if (auto * column_const = checkAndGetColumn<ColumnConst>(column))
|
||||
return checkColumn<ColumnNullable>(column_const->getDataColumn());
|
||||
return checkColumn<ColumnNullable>(column);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -100,6 +100,7 @@ public:
|
||||
return false;
|
||||
}
|
||||
|
||||
bool isNullable() const override { return true; }
|
||||
bool isFixedAndContiguous() const override { return false; }
|
||||
bool valuesHaveFixedSize() const override { return nested_column->valuesHaveFixedSize(); }
|
||||
size_t sizeOfValueIfFixed() const override { return null_map->sizeOfValueIfFixed() + nested_column->sizeOfValueIfFixed(); }
|
||||
@ -141,8 +142,6 @@ private:
|
||||
void applyNullMapImpl(const ColumnUInt8 & map);
|
||||
};
|
||||
|
||||
|
||||
ColumnPtr makeNullable(const ColumnPtr & column);
|
||||
bool isNullable(const IColumn & column);
|
||||
|
||||
}
|
||||
|
@ -191,7 +191,7 @@ ColumnUnique<ColumnType>::ColumnUnique(MutableColumnPtr && holder, bool is_nulla
|
||||
{
|
||||
if (column_holder->size() < numSpecialValues())
|
||||
throw Exception("Too small holder column for ColumnUnique.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
if (checkColumn<ColumnNullable>(*column_holder))
|
||||
if (isColumnNullable(*column_holder))
|
||||
throw Exception("Holder column for ColumnUnique can't be nullable.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
index.setColumn(getRawColumnPtr());
|
||||
|
@ -1,6 +1,8 @@
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -22,4 +24,14 @@ String IColumn::dumpStructure() const
|
||||
return res.str();
|
||||
}
|
||||
|
||||
bool isColumnNullable(const IColumn & column)
|
||||
{
|
||||
return checkColumn<ColumnNullable>(column);
|
||||
}
|
||||
|
||||
bool isColumnConst(const IColumn & column)
|
||||
{
|
||||
return checkColumn<ColumnConst>(column);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -297,6 +297,9 @@ public:
|
||||
|
||||
/// Various properties on behaviour of column type.
|
||||
|
||||
/// True if column contains something nullable inside. It's true for ColumnNullable, can be true or false for ColumnConst, etc.
|
||||
virtual bool isNullable() const { return false; }
|
||||
|
||||
/// It's a special kind of column, that contain single value, but is not a ColumnConst.
|
||||
virtual bool isDummy() const { return false; }
|
||||
|
||||
@ -429,6 +432,10 @@ bool checkColumn(const IColumn * column)
|
||||
return checkAndGetColumn<Type>(column);
|
||||
}
|
||||
|
||||
bool isColumnConst(const IColumn & column); /// defined in ColumnConst.cpp
|
||||
/// True if column's an ColumnConst instance. It's just a syntax sugar for type check.
|
||||
bool isColumnConst(const IColumn & column);
|
||||
|
||||
/// True if column's an ColumnNullable instance. It's just a syntax sugar for type check.
|
||||
bool isColumnNullable(const IColumn & column);
|
||||
|
||||
}
|
||||
|
@ -749,12 +749,12 @@ void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & argu
|
||||
|
||||
col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
|
||||
if (col_array)
|
||||
is_array_of_nullable = checkColumn<ColumnNullable>(col_array->getData());
|
||||
is_array_of_nullable = isColumnNullable(col_array->getData());
|
||||
else
|
||||
{
|
||||
col_const_array = checkAndGetColumnConstData<ColumnArray>(block.getByPosition(arguments[0]).column.get());
|
||||
if (col_const_array)
|
||||
is_array_of_nullable = checkColumn<ColumnNullable>(col_const_array->getData());
|
||||
is_array_of_nullable = isColumnNullable(col_const_array->getData());
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -150,11 +151,14 @@ public:
|
||||
ColumnPtr res = std::move(temp_block.getByPosition(result).column);
|
||||
|
||||
/// if last argument is not nullable, result should be also not nullable
|
||||
if (!isNullable(*block.getByPosition(multi_if_args.back()).column) && isNullable(*res))
|
||||
if (!block.getByPosition(multi_if_args.back()).column->isNullable() && res->isNullable())
|
||||
{
|
||||
if (auto * column_const = checkAndGetColumn<ColumnConst>(*res))
|
||||
if (auto * column_lc = checkAndGetColumn<ColumnLowCardinality>(*res))
|
||||
res = checkAndGetColumn<ColumnNullable>(*column_lc->convertToFullColumn())->getNestedColumnPtr();
|
||||
else if (auto * column_const = checkAndGetColumn<ColumnConst>(*res))
|
||||
res = checkAndGetColumn<ColumnNullable>(column_const->getDataColumn())->getNestedColumnPtr();
|
||||
res = checkAndGetColumn<ColumnNullable>(*res)->getNestedColumnPtr();
|
||||
else
|
||||
res = checkAndGetColumn<ColumnNullable>(*res)->getNestedColumnPtr();
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(res);
|
||||
|
@ -693,7 +693,7 @@ private:
|
||||
|
||||
static ColumnPtr makeNullableColumnIfNot(const ColumnPtr & column)
|
||||
{
|
||||
if (checkColumn<ColumnNullable>(*column))
|
||||
if (isColumnNullable(*column))
|
||||
return column;
|
||||
|
||||
return ColumnNullable::create(
|
||||
@ -813,7 +813,7 @@ private:
|
||||
{
|
||||
if (cond_col)
|
||||
{
|
||||
if (checkColumn<ColumnNullable>(*arg_else.column))
|
||||
if (isColumnNullable(*arg_else.column))
|
||||
{
|
||||
auto arg_else_column = arg_else.column;
|
||||
auto result_column = (*std::move(arg_else_column)).mutate();
|
||||
@ -855,7 +855,7 @@ private:
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
negated_null_map_data[i] = !null_map_data[i];
|
||||
|
||||
if (checkColumn<ColumnNullable>(*arg_then.column))
|
||||
if (isColumnNullable(*arg_then.column))
|
||||
{
|
||||
auto arg_then_column = arg_then.column;
|
||||
auto result_column = (*std::move(arg_then_column)).mutate();
|
||||
|
@ -152,7 +152,7 @@ public:
|
||||
}
|
||||
else
|
||||
{
|
||||
if (checkColumn<ColumnNullable>(*cond_col.column))
|
||||
if (isColumnNullable(*cond_col.column))
|
||||
instruction.condition_is_nullable = true;
|
||||
|
||||
instruction.condition = cond_col.column.get();
|
||||
|
@ -308,7 +308,7 @@ struct AggregationMethodKeysFixed
|
||||
ColumnUInt8 * null_map;
|
||||
|
||||
/// If we have a nullable column, get its nested column and its null map.
|
||||
if (has_nullable_keys && checkColumn<ColumnNullable>(*key_columns[i]))
|
||||
if (has_nullable_keys && isColumnNullable(*key_columns[i]))
|
||||
{
|
||||
ColumnNullable & nullable_col = static_cast<ColumnNullable &>(*key_columns[i]);
|
||||
observed_column = &nullable_col.getNestedColumn();
|
||||
@ -321,7 +321,7 @@ struct AggregationMethodKeysFixed
|
||||
}
|
||||
|
||||
bool is_null;
|
||||
if (has_nullable_keys && checkColumn<ColumnNullable>(*key_columns[i]))
|
||||
if (has_nullable_keys && isColumnNullable(*key_columns[i]))
|
||||
{
|
||||
/// The current column is nullable. Check if the value of the
|
||||
/// corresponding key is nullable. Update the null map accordingly.
|
||||
|
@ -1411,7 +1411,7 @@ private:
|
||||
|
||||
const auto & dst = out_block.getByPosition(key_pos).column;
|
||||
const auto & src = sample_block_with_keys.getByPosition(i).column;
|
||||
if (checkColumn<ColumnNullable>(*dst) != checkColumn<ColumnNullable>(*src))
|
||||
if (isColumnNullable(*dst) != isColumnNullable(*src))
|
||||
nullability_changes.insert(key_pos);
|
||||
}
|
||||
|
||||
|
@ -2,3 +2,10 @@
|
||||
2 Nullable(UInt8)
|
||||
1 Nullable(UInt8)
|
||||
\N Nullable(Nothing)
|
||||
1 Nullable(UInt8)
|
||||
2 Nullable(UInt8)
|
||||
1 Nullable(UInt8)
|
||||
\N Nullable(Nothing)
|
||||
1 LowCardinality(Nullable(UInt8))
|
||||
2 LowCardinality(Nullable(UInt8))
|
||||
1 LowCardinality(Nullable(UInt8))
|
||||
|
@ -2,3 +2,12 @@ SELECT coalesce(toNullable(1), toNullable(2)) as x, toTypeName(x);
|
||||
SELECT coalesce(NULL, toNullable(2)) as x, toTypeName(x);
|
||||
SELECT coalesce(toNullable(1), NULL) as x, toTypeName(x);
|
||||
SELECT coalesce(NULL, NULL) as x, toTypeName(x);
|
||||
|
||||
SELECT coalesce(toNullable(materialize(1)), toNullable(materialize(2))) as x, toTypeName(x);
|
||||
SELECT coalesce(NULL, toNullable(materialize(2))) as x, toTypeName(x);
|
||||
SELECT coalesce(toNullable(materialize(1)), NULL) as x, toTypeName(x);
|
||||
SELECT coalesce(materialize(NULL), materialize(NULL)) as x, toTypeName(x);
|
||||
|
||||
SELECT coalesce(toLowCardinality(toNullable(1)), toLowCardinality(toNullable(2))) as x, toTypeName(x);
|
||||
SELECT coalesce(NULL, toLowCardinality(toNullable(2))) as x, toTypeName(x);
|
||||
SELECT coalesce(toLowCardinality(toNullable(1)), NULL) as x, toTypeName(x);
|
||||
|
Loading…
Reference in New Issue
Block a user