mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-20 05:05:38 +00:00
try to avoid field operations by providing new method in IColumn
This commit is contained in:
parent
cce61ac8ae
commit
19025fb4ff
@ -102,10 +102,35 @@ bool ConstantNode::requiresCastCall(Field::Types::Which type, const DataTypePtr
|
|||||||
return need_to_add_cast_function;
|
return need_to_add_cast_function;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool ConstantNode::requiresCastCall() const
|
bool ConstantNode::requiresCastCall(const DataTypePtr & field_type, const DataTypePtr & data_type)
|
||||||
{
|
{
|
||||||
const auto & [name, type, field_type] = getFieldAttributes();
|
bool need_to_add_cast_function = false;
|
||||||
return requiresCastCall(type, field_type, getResultType());
|
WhichDataType constant_value_literal_type(field_type);
|
||||||
|
WhichDataType constant_value_type(data_type);
|
||||||
|
|
||||||
|
switch (constant_value_literal_type.idx)
|
||||||
|
{
|
||||||
|
case TypeIndex::String:
|
||||||
|
{
|
||||||
|
need_to_add_cast_function = !constant_value_type.isString();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case TypeIndex::UInt64:
|
||||||
|
case TypeIndex::Int64:
|
||||||
|
case TypeIndex::Float64:
|
||||||
|
{
|
||||||
|
WhichDataType constant_value_field_type(field_type);
|
||||||
|
need_to_add_cast_function = constant_value_field_type.idx != constant_value_type.idx;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
default:
|
||||||
|
{
|
||||||
|
need_to_add_cast_function = true;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return need_to_add_cast_function;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool ConstantNode::receivedFromInitiatorServer() const
|
bool ConstantNode::receivedFromInitiatorServer() const
|
||||||
|
@ -92,10 +92,9 @@ public:
|
|||||||
return constant_value.getType();
|
return constant_value.getType();
|
||||||
}
|
}
|
||||||
|
|
||||||
static bool requiresCastCall(Field::Types::Which type, const DataTypePtr & field_type, const DataTypePtr & data_type);
|
|
||||||
|
|
||||||
/// Check if conversion to AST requires wrapping with _CAST function.
|
/// Check if conversion to AST requires wrapping with _CAST function.
|
||||||
bool requiresCastCall() const;
|
static bool requiresCastCall(Field::Types::Which type, const DataTypePtr & field_type, const DataTypePtr & data_type);
|
||||||
|
static bool requiresCastCall(const DataTypePtr & field_type, const DataTypePtr & data_type);
|
||||||
|
|
||||||
/// Check if constant is a result of _CAST function constant folding.
|
/// Check if constant is a result of _CAST function constant folding.
|
||||||
bool receivedFromInitiatorServer() const;
|
bool receivedFromInitiatorServer() const;
|
||||||
@ -109,10 +108,9 @@ public:
|
|||||||
|
|
||||||
void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override;
|
void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override;
|
||||||
|
|
||||||
std::tuple<String, Field::Types::Which, DataTypePtr> getFieldAttributes() const
|
std::pair<String, DataTypePtr> getValueNameAndType() const
|
||||||
{
|
{
|
||||||
const auto & [name, type, field_type] = constant_value.getFieldAttributes();
|
return constant_value.getValueNameAndType();
|
||||||
return {name + "_" + constant_value.getType()->getName(), type, field_type};
|
|
||||||
}
|
}
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
|
@ -21,7 +21,6 @@ public:
|
|||||||
ConstantValue(const Field & field_, DataTypePtr data_type_)
|
ConstantValue(const Field & field_, DataTypePtr data_type_)
|
||||||
: column(data_type_->createColumnConst(1, field_))
|
: column(data_type_->createColumnConst(1, field_))
|
||||||
, data_type(std::move(data_type_))
|
, data_type(std::move(data_type_))
|
||||||
, field_cache(applyVisitor(FieldVisitorToString(), field_), field_.getType(), applyVisitor(FieldToDataType<LeastSupertypeOnError::Variant>(), field_))
|
|
||||||
{}
|
{}
|
||||||
|
|
||||||
const ColumnPtr & getColumn() const
|
const ColumnPtr & getColumn() const
|
||||||
@ -34,21 +33,9 @@ public:
|
|||||||
return data_type;
|
return data_type;
|
||||||
}
|
}
|
||||||
|
|
||||||
const std::tuple<String, Field::Types::Which, DataTypePtr> & getFieldAttributes() const &
|
std::pair<String, DataTypePtr> getValueNameAndType() const
|
||||||
{
|
{
|
||||||
if (std::get<String>(field_cache).empty())
|
return column->getValueNameAndType(0);
|
||||||
{
|
|
||||||
Field field;
|
|
||||||
column->get(0, field);
|
|
||||||
field_cache = {applyVisitor(FieldVisitorToString(), field), field.getType(), applyVisitor(FieldToDataType<LeastSupertypeOnError::Variant>(), field)};
|
|
||||||
}
|
|
||||||
|
|
||||||
return field_cache;
|
|
||||||
}
|
|
||||||
|
|
||||||
std::tuple<String, Field::Types::Which, DataTypePtr> getFieldAttributes() const &&
|
|
||||||
{
|
|
||||||
return getFieldAttributes();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
@ -62,7 +49,6 @@ private:
|
|||||||
|
|
||||||
ColumnPtr column;
|
ColumnPtr column;
|
||||||
DataTypePtr data_type;
|
DataTypePtr data_type;
|
||||||
mutable std::tuple<String, Field::Types::Which, DataTypePtr> field_cache;
|
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,4 +1,6 @@
|
|||||||
|
#include <IO/WriteHelpers.h>
|
||||||
#include <Columns/ColumnAggregateFunction.h>
|
#include <Columns/ColumnAggregateFunction.h>
|
||||||
|
#include <DataTypes/DataTypeFactory.h>
|
||||||
|
|
||||||
#include <AggregateFunctions/IAggregateFunction.h>
|
#include <AggregateFunctions/IAggregateFunction.h>
|
||||||
#include <Columns/ColumnsCommon.h>
|
#include <Columns/ColumnsCommon.h>
|
||||||
@ -470,6 +472,19 @@ void ColumnAggregateFunction::get(size_t n, Field & res) const
|
|||||||
res = operator[](n);
|
res = operator[](n);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> ColumnAggregateFunction::getValueNameAndType(size_t n) const
|
||||||
|
{
|
||||||
|
String state;
|
||||||
|
{
|
||||||
|
WriteBufferFromOwnString buffer;
|
||||||
|
func->serialize(data[n], buffer, version);
|
||||||
|
WriteBufferFromString wb(state);
|
||||||
|
writeQuoted(buffer.str(), wb);
|
||||||
|
}
|
||||||
|
|
||||||
|
return {state, DataTypeFactory::instance().get(type_string)};
|
||||||
|
}
|
||||||
|
|
||||||
StringRef ColumnAggregateFunction::getDataAt(size_t n) const
|
StringRef ColumnAggregateFunction::getDataAt(size_t n) const
|
||||||
{
|
{
|
||||||
return StringRef(reinterpret_cast<const char *>(&data[n]), sizeof(data[n]));
|
return StringRef(reinterpret_cast<const char *>(&data[n]), sizeof(data[n]));
|
||||||
|
@ -136,6 +136,8 @@ public:
|
|||||||
|
|
||||||
void get(size_t n, Field & res) const override;
|
void get(size_t n, Field & res) const override;
|
||||||
|
|
||||||
|
virtual std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override;
|
||||||
|
|
||||||
bool isDefaultAt(size_t) const override
|
bool isDefaultAt(size_t) const override
|
||||||
{
|
{
|
||||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method isDefaultAt is not supported for ColumnAggregateFunction");
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method isDefaultAt is not supported for ColumnAggregateFunction");
|
||||||
|
@ -1,3 +1,5 @@
|
|||||||
|
#include <DataTypes/getLeastSupertype.h>
|
||||||
|
#include <DataTypes/DataTypeArray.h>
|
||||||
#include <Columns/ColumnArray.h>
|
#include <Columns/ColumnArray.h>
|
||||||
#include <Columns/ColumnsNumber.h>
|
#include <Columns/ColumnsNumber.h>
|
||||||
#include <Columns/ColumnDecimal.h>
|
#include <Columns/ColumnDecimal.h>
|
||||||
@ -148,6 +150,27 @@ void ColumnArray::get(size_t n, Field & res) const
|
|||||||
res_arr.push_back(getData()[offset + i]);
|
res_arr.push_back(getData()[offset + i]);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> ColumnArray::getValueNameAndType(size_t n) const
|
||||||
|
{
|
||||||
|
size_t offset = offsetAt(n);
|
||||||
|
size_t size = sizeAt(n);
|
||||||
|
|
||||||
|
String value_name {"["};
|
||||||
|
DataTypes element_types;
|
||||||
|
element_types.reserve(size);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
const auto & [value, type] = getData().getValueNameAndType(offset + i);
|
||||||
|
element_types.push_back(type);
|
||||||
|
if (i > 0)
|
||||||
|
value_name += ", ";
|
||||||
|
value_name += value;
|
||||||
|
}
|
||||||
|
value_name += "]";
|
||||||
|
|
||||||
|
return {value_name, std::make_shared<DataTypeArray>(getLeastSupertype<LeastSupertypeOnError::Variant>(element_types))};
|
||||||
|
}
|
||||||
|
|
||||||
StringRef ColumnArray::getDataAt(size_t n) const
|
StringRef ColumnArray::getDataAt(size_t n) const
|
||||||
{
|
{
|
||||||
|
@ -74,6 +74,7 @@ public:
|
|||||||
size_t size() const override;
|
size_t size() const override;
|
||||||
Field operator[](size_t n) const override;
|
Field operator[](size_t n) const override;
|
||||||
void get(size_t n, Field & res) const override;
|
void get(size_t n, Field & res) const override;
|
||||||
|
virtual std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override;
|
||||||
StringRef getDataAt(size_t n) const override;
|
StringRef getDataAt(size_t n) const override;
|
||||||
bool isDefaultAt(size_t n) const override;
|
bool isDefaultAt(size_t n) const override;
|
||||||
void insertData(const char * pos, size_t length) override;
|
void insertData(const char * pos, size_t length) override;
|
||||||
|
@ -82,6 +82,7 @@ public:
|
|||||||
TypeIndex getDataType() const override { throwMustBeDecompressed(); }
|
TypeIndex getDataType() const override { throwMustBeDecompressed(); }
|
||||||
Field operator[](size_t) const override { throwMustBeDecompressed(); }
|
Field operator[](size_t) const override { throwMustBeDecompressed(); }
|
||||||
void get(size_t, Field &) const override { throwMustBeDecompressed(); }
|
void get(size_t, Field &) const override { throwMustBeDecompressed(); }
|
||||||
|
virtual std::pair<String, DataTypePtr> getValueNameAndType(size_t) const override { throwMustBeDecompressed(); }
|
||||||
StringRef getDataAt(size_t) const override { throwMustBeDecompressed(); }
|
StringRef getDataAt(size_t) const override { throwMustBeDecompressed(); }
|
||||||
bool isDefaultAt(size_t) const override { throwMustBeDecompressed(); }
|
bool isDefaultAt(size_t) const override { throwMustBeDecompressed(); }
|
||||||
void insert(const Field &) override { throwMustBeDecompressed(); }
|
void insert(const Field &) override { throwMustBeDecompressed(); }
|
||||||
|
@ -78,6 +78,11 @@ public:
|
|||||||
data->get(0, res);
|
data->get(0, res);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
virtual std::pair<String, DataTypePtr> getValueNameAndType(size_t) const override
|
||||||
|
{
|
||||||
|
return data->getValueNameAndType(0);
|
||||||
|
}
|
||||||
|
|
||||||
StringRef getDataAt(size_t) const override
|
StringRef getDataAt(size_t) const override
|
||||||
{
|
{
|
||||||
return data->getDataAt(0);
|
return data->getDataAt(0);
|
||||||
|
@ -1,5 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <Common/FieldVisitorToString.h>
|
||||||
|
#include <DataTypes/FieldToDataType.h>
|
||||||
#include <base/sort.h>
|
#include <base/sort.h>
|
||||||
#include <base/TypeName.h>
|
#include <base/TypeName.h>
|
||||||
#include <Core/Field.h>
|
#include <Core/Field.h>
|
||||||
@ -121,6 +123,10 @@ public:
|
|||||||
|
|
||||||
Field operator[](size_t n) const override { return DecimalField(data[n], scale); }
|
Field operator[](size_t n) const override { return DecimalField(data[n], scale); }
|
||||||
void get(size_t n, Field & res) const override { res = (*this)[n]; }
|
void get(size_t n, Field & res) const override { res = (*this)[n]; }
|
||||||
|
std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override
|
||||||
|
{
|
||||||
|
return {FieldVisitorToString()(data[n], scale), FieldToDataType()(data[n], scale)};
|
||||||
|
}
|
||||||
bool getBool(size_t n) const override { return bool(data[n].value); }
|
bool getBool(size_t n) const override { return bool(data[n].value); }
|
||||||
Int64 getInt(size_t n) const override { return Int64(data[n].value); }
|
Int64 getInt(size_t n) const override { return Int64(data[n].value); }
|
||||||
UInt64 get64(size_t n) const override;
|
UInt64 get64(size_t n) const override;
|
||||||
|
@ -303,6 +303,22 @@ void ColumnDynamic::get(size_t n, Field & res) const
|
|||||||
type->getDefaultSerialization()->deserializeBinary(res, buf, getFormatSettings());
|
type->getDefaultSerialization()->deserializeBinary(res, buf, getFormatSettings());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> ColumnDynamic::getValueNameAndType(size_t n) const
|
||||||
|
{
|
||||||
|
const auto & variant_col = getVariantColumn();
|
||||||
|
/// Check if value is not in shared variant.
|
||||||
|
if (variant_col.globalDiscriminatorAt(n) != getSharedVariantDiscriminator())
|
||||||
|
return variant_col.getValueNameAndType(n);
|
||||||
|
|
||||||
|
/// We should deeserialize value from shared variant.
|
||||||
|
const auto & shared_variant = getSharedVariant();
|
||||||
|
auto value_data = shared_variant.getDataAt(variant_col.offsetAt(n));
|
||||||
|
ReadBufferFromMemory buf(value_data.data, value_data.size);
|
||||||
|
auto type = decodeDataType(buf);
|
||||||
|
const auto col = type->createColumn();
|
||||||
|
type->getDefaultSerialization()->deserializeBinary(*col, buf, getFormatSettings());
|
||||||
|
return col->getValueNameAndType(0);
|
||||||
|
}
|
||||||
|
|
||||||
#if !defined(DEBUG_OR_SANITIZER_BUILD)
|
#if !defined(DEBUG_OR_SANITIZER_BUILD)
|
||||||
void ColumnDynamic::insertFrom(const IColumn & src_, size_t n)
|
void ColumnDynamic::insertFrom(const IColumn & src_, size_t n)
|
||||||
|
@ -143,6 +143,8 @@ public:
|
|||||||
|
|
||||||
void get(size_t n, Field & res) const override;
|
void get(size_t n, Field & res) const override;
|
||||||
|
|
||||||
|
virtual std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override;
|
||||||
|
|
||||||
bool isDefaultAt(size_t n) const override
|
bool isDefaultAt(size_t n) const override
|
||||||
{
|
{
|
||||||
return variant_column_ptr->isDefaultAt(n);
|
return variant_column_ptr->isDefaultAt(n);
|
||||||
|
@ -1,5 +1,8 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <DataTypes/DataTypeString.h>
|
||||||
|
#include <IO/WriteHelpers.h>
|
||||||
|
#include <IO/WriteBufferFromString.h>
|
||||||
#include <Common/PODArray.h>
|
#include <Common/PODArray.h>
|
||||||
#include <Common/memcmpSmall.h>
|
#include <Common/memcmpSmall.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
@ -87,6 +90,13 @@ public:
|
|||||||
res = std::string_view{reinterpret_cast<const char *>(&chars[n * index]), n};
|
res = std::string_view{reinterpret_cast<const char *>(&chars[n * index]), n};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> getValueNameAndType(size_t index) const override
|
||||||
|
{
|
||||||
|
WriteBufferFromOwnString buf;
|
||||||
|
writeQuoted(std::string_view{reinterpret_cast<const char *>(&chars[n * index]), n}, buf);
|
||||||
|
return {buf.str(), std::make_shared<DataTypeString>()};
|
||||||
|
}
|
||||||
|
|
||||||
StringRef getDataAt(size_t index) const override
|
StringRef getDataAt(size_t index) const override
|
||||||
{
|
{
|
||||||
return StringRef(&chars[n * index], n);
|
return StringRef(&chars[n * index], n);
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
#include <Interpreters/ExpressionActions.h>
|
#include <Interpreters/ExpressionActions.h>
|
||||||
#include <Columns/ColumnFunction.h>
|
#include <Columns/ColumnFunction.h>
|
||||||
#include <Columns/ColumnsCommon.h>
|
#include <Columns/ColumnsCommon.h>
|
||||||
@ -91,6 +92,27 @@ void ColumnFunction::get(size_t n, Field & res) const
|
|||||||
res_tuple.push_back((*captured_columns[i].column)[n]);
|
res_tuple.push_back((*captured_columns[i].column)[n]);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> ColumnFunction::getValueNameAndType(size_t n) const
|
||||||
|
{
|
||||||
|
size_t size = captured_columns.size();
|
||||||
|
|
||||||
|
String value_name {size > 1 ? "(" : "tuple("};
|
||||||
|
DataTypes element_types;
|
||||||
|
element_types.reserve(size);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
const auto & [value, type] = captured_columns[i].column->getValueNameAndType(n);
|
||||||
|
element_types.push_back(type);
|
||||||
|
if (i > 0)
|
||||||
|
value_name += ", ";
|
||||||
|
value_name += value;
|
||||||
|
}
|
||||||
|
value_name += ")";
|
||||||
|
|
||||||
|
return {value_name, std::make_shared<DataTypeTuple>(element_types)};
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
#if !defined(DEBUG_OR_SANITIZER_BUILD)
|
#if !defined(DEBUG_OR_SANITIZER_BUILD)
|
||||||
void ColumnFunction::insertFrom(const IColumn & src, size_t n)
|
void ColumnFunction::insertFrom(const IColumn & src, size_t n)
|
||||||
|
@ -64,6 +64,8 @@ public:
|
|||||||
|
|
||||||
void get(size_t n, Field & res) const override;
|
void get(size_t n, Field & res) const override;
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override;
|
||||||
|
|
||||||
StringRef getDataAt(size_t) const override
|
StringRef getDataAt(size_t) const override
|
||||||
{
|
{
|
||||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get value from {}", getName());
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get value from {}", getName());
|
||||||
|
@ -58,6 +58,10 @@ public:
|
|||||||
|
|
||||||
Field operator[](size_t n) const override { return getDictionary()[getIndexes().getUInt(n)]; }
|
Field operator[](size_t n) const override { return getDictionary()[getIndexes().getUInt(n)]; }
|
||||||
void get(size_t n, Field & res) const override { getDictionary().get(getIndexes().getUInt(n), res); }
|
void get(size_t n, Field & res) const override { getDictionary().get(getIndexes().getUInt(n), res); }
|
||||||
|
std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override
|
||||||
|
{
|
||||||
|
return getDictionary().getValueNameAndType(getIndexes().getUInt(n));
|
||||||
|
}
|
||||||
|
|
||||||
StringRef getDataAt(size_t n) const override { return getDictionary().getDataAt(getIndexes().getUInt(n)); }
|
StringRef getDataAt(size_t n) const override { return getDictionary().getDataAt(getIndexes().getUInt(n)); }
|
||||||
|
|
||||||
|
@ -1,3 +1,5 @@
|
|||||||
|
#include <DataTypes/getLeastSupertype.h>
|
||||||
|
#include <DataTypes/DataTypeArray.h>
|
||||||
#include <Columns/ColumnMap.h>
|
#include <Columns/ColumnMap.h>
|
||||||
#include <Columns/ColumnCompressed.h>
|
#include <Columns/ColumnCompressed.h>
|
||||||
#include <IO/WriteBufferFromString.h>
|
#include <IO/WriteBufferFromString.h>
|
||||||
@ -79,6 +81,29 @@ void ColumnMap::get(size_t n, Field & res) const
|
|||||||
map.push_back(getNestedData()[offset + i]);
|
map.push_back(getNestedData()[offset + i]);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> ColumnMap::getValueNameAndType(size_t n) const
|
||||||
|
{
|
||||||
|
const auto & offsets = getNestedColumn().getOffsets();
|
||||||
|
size_t offset = offsets[n - 1];
|
||||||
|
size_t size = offsets[n] - offsets[n - 1];
|
||||||
|
|
||||||
|
String value_name {"["};
|
||||||
|
DataTypes element_types;
|
||||||
|
element_types.reserve(size);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
const auto & [value, type] = getNestedData().getValueNameAndType(offset + i);
|
||||||
|
element_types.push_back(type);
|
||||||
|
if (i > 0)
|
||||||
|
value_name += ", ";
|
||||||
|
value_name += value;
|
||||||
|
}
|
||||||
|
value_name += "]";
|
||||||
|
|
||||||
|
return {value_name, std::make_shared<DataTypeArray>(getLeastSupertype<LeastSupertypeOnError::Variant>(element_types))};
|
||||||
|
}
|
||||||
|
|
||||||
bool ColumnMap::isDefaultAt(size_t n) const
|
bool ColumnMap::isDefaultAt(size_t n) const
|
||||||
{
|
{
|
||||||
return nested->isDefaultAt(n);
|
return nested->isDefaultAt(n);
|
||||||
|
@ -51,6 +51,7 @@ public:
|
|||||||
|
|
||||||
Field operator[](size_t n) const override;
|
Field operator[](size_t n) const override;
|
||||||
void get(size_t n, Field & res) const override;
|
void get(size_t n, Field & res) const override;
|
||||||
|
std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override;
|
||||||
|
|
||||||
bool isDefaultAt(size_t n) const override;
|
bool isDefaultAt(size_t n) const override;
|
||||||
StringRef getDataAt(size_t n) const override;
|
StringRef getDataAt(size_t n) const override;
|
||||||
|
@ -1,3 +1,5 @@
|
|||||||
|
#include <DataTypes/DataTypeNothing.h>
|
||||||
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
#include <Common/Arena.h>
|
#include <Common/Arena.h>
|
||||||
#include <Common/HashTable/StringHashSet.h>
|
#include <Common/HashTable/StringHashSet.h>
|
||||||
#include <Common/SipHash.h>
|
#include <Common/SipHash.h>
|
||||||
@ -115,6 +117,14 @@ void ColumnNullable::get(size_t n, Field & res) const
|
|||||||
getNestedColumn().get(n, res);
|
getNestedColumn().get(n, res);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> ColumnNullable::getValueNameAndType(size_t n) const
|
||||||
|
{
|
||||||
|
if (isNullAt(n))
|
||||||
|
return {"NULL", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeNothing>())};
|
||||||
|
|
||||||
|
return getNestedColumn().getValueNameAndType(n);
|
||||||
|
}
|
||||||
|
|
||||||
Float64 ColumnNullable::getFloat64(size_t n) const
|
Float64 ColumnNullable::getFloat64(size_t n) const
|
||||||
{
|
{
|
||||||
if (isNullAt(n))
|
if (isNullAt(n))
|
||||||
|
@ -55,6 +55,7 @@ public:
|
|||||||
bool isNullAt(size_t n) const override { return assert_cast<const ColumnUInt8 &>(*null_map).getData()[n] != 0;}
|
bool isNullAt(size_t n) const override { return assert_cast<const ColumnUInt8 &>(*null_map).getData()[n] != 0;}
|
||||||
Field operator[](size_t n) const override;
|
Field operator[](size_t n) const override;
|
||||||
void get(size_t n, Field & res) const override;
|
void get(size_t n, Field & res) const override;
|
||||||
|
std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override;
|
||||||
bool getBool(size_t n) const override { return isNullAt(n) ? false : nested_column->getBool(n); }
|
bool getBool(size_t n) const override { return isNullAt(n) ? false : nested_column->getBool(n); }
|
||||||
UInt64 get64(size_t n) const override { return nested_column->get64(n); }
|
UInt64 get64(size_t n) const override { return nested_column->get64(n); }
|
||||||
Float64 getFloat64(size_t n) const override;
|
Float64 getFloat64(size_t n) const override;
|
||||||
|
@ -1,3 +1,5 @@
|
|||||||
|
#include <DataTypes/DataTypeObject.h>
|
||||||
|
#include <DataTypes/DataTypesBinaryEncoding.h>
|
||||||
#include <Columns/ColumnObject.h>
|
#include <Columns/ColumnObject.h>
|
||||||
#include <Columns/ColumnCompressed.h>
|
#include <Columns/ColumnCompressed.h>
|
||||||
#include <DataTypes/Serializations/SerializationDynamic.h>
|
#include <DataTypes/Serializations/SerializationDynamic.h>
|
||||||
@ -265,6 +267,81 @@ void ColumnObject::get(size_t n, Field & res) const
|
|||||||
res = (*this)[n];
|
res = (*this)[n];
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> ColumnObject::getValueNameAndType(size_t n) const
|
||||||
|
{
|
||||||
|
WriteBufferFromOwnString wb;
|
||||||
|
wb << '{';
|
||||||
|
|
||||||
|
bool first = true;
|
||||||
|
|
||||||
|
for (const auto & [path, column] : typed_paths)
|
||||||
|
{
|
||||||
|
const auto & [value, type] = column->getValueNameAndType(n);
|
||||||
|
|
||||||
|
if (first)
|
||||||
|
first = false;
|
||||||
|
else
|
||||||
|
wb << ", ";
|
||||||
|
|
||||||
|
writeDoubleQuoted(path, wb);
|
||||||
|
wb << ": " << value;
|
||||||
|
}
|
||||||
|
|
||||||
|
for (const auto & [path, column] : dynamic_paths_ptrs)
|
||||||
|
{
|
||||||
|
/// Output only non-null values from dynamic paths. We cannot distinguish cases when
|
||||||
|
/// dynamic path has Null value and when it's absent in the row and consider them equivalent.
|
||||||
|
if (column->isNullAt(n))
|
||||||
|
continue;
|
||||||
|
|
||||||
|
const auto & [value, type] = column->getValueNameAndType(n);
|
||||||
|
|
||||||
|
if (first)
|
||||||
|
first = false;
|
||||||
|
else
|
||||||
|
wb << ", ";
|
||||||
|
|
||||||
|
writeDoubleQuoted(path, wb);
|
||||||
|
wb << ": " << value;
|
||||||
|
}
|
||||||
|
|
||||||
|
const auto & shared_data_offsets = getSharedDataOffsets();
|
||||||
|
const auto [shared_paths, shared_values] = getSharedDataPathsAndValues();
|
||||||
|
size_t start = shared_data_offsets[static_cast<ssize_t>(n) - 1];
|
||||||
|
size_t end = shared_data_offsets[n];
|
||||||
|
for (size_t i = start; i != end; ++i)
|
||||||
|
{
|
||||||
|
if (first)
|
||||||
|
first = false;
|
||||||
|
else
|
||||||
|
wb << ", ";
|
||||||
|
|
||||||
|
String path = shared_paths->getDataAt(i).toString();
|
||||||
|
writeDoubleQuoted(path, wb);
|
||||||
|
|
||||||
|
auto value_data = shared_values->getDataAt(i);
|
||||||
|
ReadBufferFromMemory buf(value_data.data, value_data.size);
|
||||||
|
auto decoded_type = decodeDataType(buf);
|
||||||
|
|
||||||
|
if (isNothing(decoded_type))
|
||||||
|
{
|
||||||
|
wb << ": NULL";
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
const auto column = decoded_type->createColumn();
|
||||||
|
decoded_type->getDefaultSerialization()->deserializeBinary(*column, buf, getFormatSettings());
|
||||||
|
|
||||||
|
const auto & [value, type] = column->getValueNameAndType(n);
|
||||||
|
|
||||||
|
wb << ": " << value;
|
||||||
|
}
|
||||||
|
|
||||||
|
wb << "}";
|
||||||
|
|
||||||
|
return {wb.str(), std::make_shared<DataTypeObject>(DataTypeObject::SchemaFormat::JSON)};
|
||||||
|
}
|
||||||
|
|
||||||
bool ColumnObject::isDefaultAt(size_t n) const
|
bool ColumnObject::isDefaultAt(size_t n) const
|
||||||
{
|
{
|
||||||
for (const auto & [path, column] : typed_paths)
|
for (const auto & [path, column] : typed_paths)
|
||||||
|
@ -107,6 +107,7 @@ public:
|
|||||||
|
|
||||||
Field operator[](size_t n) const override;
|
Field operator[](size_t n) const override;
|
||||||
void get(size_t n, Field & res) const override;
|
void get(size_t n, Field & res) const override;
|
||||||
|
std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override;
|
||||||
|
|
||||||
bool isDefaultAt(size_t n) const override;
|
bool isDefaultAt(size_t n) const override;
|
||||||
StringRef getDataAt(size_t n) const override;
|
StringRef getDataAt(size_t n) const override;
|
||||||
|
@ -1,3 +1,6 @@
|
|||||||
|
#include <DataTypes/DataTypeObject.h>
|
||||||
|
#include <IO/WriteBufferFromString.h>
|
||||||
|
#include <IO/Operators.h>
|
||||||
#include <Core/Field.h>
|
#include <Core/Field.h>
|
||||||
#include <Columns/ColumnObjectDeprecated.h>
|
#include <Columns/ColumnObjectDeprecated.h>
|
||||||
#include <Columns/ColumnsNumber.h>
|
#include <Columns/ColumnsNumber.h>
|
||||||
@ -237,6 +240,33 @@ void ColumnObjectDeprecated::Subcolumn::get(size_t n, Field & res) const
|
|||||||
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Index ({}) for getting field is out of range", n);
|
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Index ({}) for getting field is out of range", n);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> ColumnObjectDeprecated::Subcolumn::getValueNameAndType(size_t n) const
|
||||||
|
{
|
||||||
|
if (isFinalized())
|
||||||
|
return getFinalizedColumn().getValueNameAndType(n);
|
||||||
|
|
||||||
|
size_t ind = n;
|
||||||
|
if (ind < num_of_defaults_in_prefix)
|
||||||
|
return least_common_type.get()->createColumnConstWithDefaultValue(1)->getValueNameAndType(0);
|
||||||
|
|
||||||
|
ind -= num_of_defaults_in_prefix;
|
||||||
|
for (const auto & part : data)
|
||||||
|
{
|
||||||
|
if (ind < part->size())
|
||||||
|
{
|
||||||
|
Field field;
|
||||||
|
part->get(ind, field);
|
||||||
|
const auto column = least_common_type.get()->createColumn();
|
||||||
|
column->insert(convertFieldToTypeOrThrow(field, *least_common_type.get()));
|
||||||
|
return column->getValueNameAndType(0);
|
||||||
|
}
|
||||||
|
|
||||||
|
ind -= part->size();
|
||||||
|
}
|
||||||
|
|
||||||
|
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Index ({}) for getting field is out of range", n);
|
||||||
|
}
|
||||||
|
|
||||||
void ColumnObjectDeprecated::Subcolumn::checkTypes() const
|
void ColumnObjectDeprecated::Subcolumn::checkTypes() const
|
||||||
{
|
{
|
||||||
DataTypes prefix_types;
|
DataTypes prefix_types;
|
||||||
@ -763,6 +793,30 @@ void ColumnObjectDeprecated::get(size_t n, Field & res) const
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> ColumnObjectDeprecated::getValueNameAndType(size_t n) const
|
||||||
|
{
|
||||||
|
WriteBufferFromOwnString wb;
|
||||||
|
wb << '{';
|
||||||
|
|
||||||
|
bool first = true;
|
||||||
|
|
||||||
|
for (const auto & entry : subcolumns)
|
||||||
|
{
|
||||||
|
if (first)
|
||||||
|
first = false;
|
||||||
|
else
|
||||||
|
wb << ", ";
|
||||||
|
|
||||||
|
writeDoubleQuoted(entry->path.getPath(), wb);
|
||||||
|
const auto & [value, type] = entry->data.getValueNameAndType(n);
|
||||||
|
wb << ": " << value;
|
||||||
|
}
|
||||||
|
|
||||||
|
wb << "}";
|
||||||
|
|
||||||
|
return {wb.str(), std::make_shared<DataTypeObject>(DataTypeObject::SchemaFormat::JSON)};
|
||||||
|
}
|
||||||
|
|
||||||
#if !defined(DEBUG_OR_SANITIZER_BUILD)
|
#if !defined(DEBUG_OR_SANITIZER_BUILD)
|
||||||
void ColumnObjectDeprecated::insertFrom(const IColumn & src, size_t n)
|
void ColumnObjectDeprecated::insertFrom(const IColumn & src, size_t n)
|
||||||
#else
|
#else
|
||||||
|
@ -69,6 +69,7 @@ public:
|
|||||||
size_t byteSize() const;
|
size_t byteSize() const;
|
||||||
size_t allocatedBytes() const;
|
size_t allocatedBytes() const;
|
||||||
void get(size_t n, Field & res) const;
|
void get(size_t n, Field & res) const;
|
||||||
|
std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const;
|
||||||
|
|
||||||
bool isFinalized() const;
|
bool isFinalized() const;
|
||||||
const DataTypePtr & getLeastCommonType() const { return least_common_type.get(); }
|
const DataTypePtr & getLeastCommonType() const { return least_common_type.get(); }
|
||||||
@ -220,6 +221,7 @@ public:
|
|||||||
void popBack(size_t length) override;
|
void popBack(size_t length) override;
|
||||||
Field operator[](size_t n) const override;
|
Field operator[](size_t n) const override;
|
||||||
void get(size_t n, Field & res) const override;
|
void get(size_t n, Field & res) const override;
|
||||||
|
std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override;
|
||||||
|
|
||||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||||
ColumnPtr filter(const Filter & filter, ssize_t result_size_hint) const override;
|
ColumnPtr filter(const Filter & filter, ssize_t result_size_hint) const override;
|
||||||
|
@ -94,6 +94,11 @@ void ColumnSparse::get(size_t n, Field & res) const
|
|||||||
values->get(getValueIndex(n), res);
|
values->get(getValueIndex(n), res);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> ColumnSparse::getValueNameAndType(size_t n) const
|
||||||
|
{
|
||||||
|
return values->getValueNameAndType(getValueIndex(n));
|
||||||
|
}
|
||||||
|
|
||||||
bool ColumnSparse::getBool(size_t n) const
|
bool ColumnSparse::getBool(size_t n) const
|
||||||
{
|
{
|
||||||
return values->getBool(getValueIndex(n));
|
return values->getBool(getValueIndex(n));
|
||||||
|
@ -65,6 +65,7 @@ public:
|
|||||||
bool isNullAt(size_t n) const override;
|
bool isNullAt(size_t n) const override;
|
||||||
Field operator[](size_t n) const override;
|
Field operator[](size_t n) const override;
|
||||||
void get(size_t n, Field & res) const override;
|
void get(size_t n, Field & res) const override;
|
||||||
|
virtual std::pair<String, DataTypePtr> getValueNameAndType(size_t) const override;
|
||||||
bool getBool(size_t n) const override;
|
bool getBool(size_t n) const override;
|
||||||
Float64 getFloat64(size_t n) const override;
|
Float64 getFloat64(size_t n) const override;
|
||||||
Float32 getFloat32(size_t n) const override;
|
Float32 getFloat32(size_t n) const override;
|
||||||
|
@ -2,6 +2,8 @@
|
|||||||
|
|
||||||
#include <cstring>
|
#include <cstring>
|
||||||
|
|
||||||
|
#include <DataTypes/DataTypeString.h>
|
||||||
|
#include <IO/WriteHelpers.h>
|
||||||
#include <Columns/IColumn.h>
|
#include <Columns/IColumn.h>
|
||||||
#include <Columns/IColumnImpl.h>
|
#include <Columns/IColumnImpl.h>
|
||||||
#include <Common/PODArray.h>
|
#include <Common/PODArray.h>
|
||||||
@ -109,6 +111,13 @@ public:
|
|||||||
res = std::string_view{reinterpret_cast<const char *>(&chars[offsetAt(n)]), sizeAt(n) - 1};
|
res = std::string_view{reinterpret_cast<const char *>(&chars[offsetAt(n)]), sizeAt(n) - 1};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override
|
||||||
|
{
|
||||||
|
WriteBufferFromOwnString wb;
|
||||||
|
writeQuoted(std::string_view{reinterpret_cast<const char *>(&chars[offsetAt(n)]), sizeAt(n) - 1}, wb);
|
||||||
|
return {wb.str(), std::make_shared<DataTypeString>()};
|
||||||
|
}
|
||||||
|
|
||||||
StringRef getDataAt(size_t n) const override
|
StringRef getDataAt(size_t n) const override
|
||||||
{
|
{
|
||||||
chassert(n < size());
|
chassert(n < size());
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
#include <Columns/ColumnTuple.h>
|
#include <Columns/ColumnTuple.h>
|
||||||
|
|
||||||
#include <Columns/ColumnCompressed.h>
|
#include <Columns/ColumnCompressed.h>
|
||||||
@ -148,6 +149,28 @@ void ColumnTuple::get(size_t n, Field & res) const
|
|||||||
res_tuple.push_back((*columns[i])[n]);
|
res_tuple.push_back((*columns[i])[n]);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> ColumnTuple::getValueNameAndType(size_t n) const
|
||||||
|
{
|
||||||
|
const size_t tuple_size = columns.size();
|
||||||
|
|
||||||
|
String value_name {tuple_size > 1 ? "(" : "tuple("};
|
||||||
|
|
||||||
|
DataTypes element_types;
|
||||||
|
element_types.reserve(tuple_size);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < tuple_size; ++i)
|
||||||
|
{
|
||||||
|
const auto & [value, type] = columns[i]->getValueNameAndType(n);
|
||||||
|
element_types.push_back(type);
|
||||||
|
if (i > 0)
|
||||||
|
value_name += ", ";
|
||||||
|
value_name += value;
|
||||||
|
}
|
||||||
|
value_name += ")";
|
||||||
|
|
||||||
|
return {value_name, std::make_shared<DataTypeTuple>(element_types)};
|
||||||
|
}
|
||||||
|
|
||||||
bool ColumnTuple::isDefaultAt(size_t n) const
|
bool ColumnTuple::isDefaultAt(size_t n) const
|
||||||
{
|
{
|
||||||
const size_t tuple_size = columns.size();
|
const size_t tuple_size = columns.size();
|
||||||
|
@ -59,6 +59,7 @@ public:
|
|||||||
|
|
||||||
Field operator[](size_t n) const override;
|
Field operator[](size_t n) const override;
|
||||||
void get(size_t n, Field & res) const override;
|
void get(size_t n, Field & res) const override;
|
||||||
|
virtual std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override;
|
||||||
|
|
||||||
bool isDefaultAt(size_t n) const override;
|
bool isDefaultAt(size_t n) const override;
|
||||||
StringRef getDataAt(size_t n) const override;
|
StringRef getDataAt(size_t n) const override;
|
||||||
|
@ -74,6 +74,10 @@ public:
|
|||||||
|
|
||||||
Field operator[](size_t n) const override { return (*getNestedColumn())[n]; }
|
Field operator[](size_t n) const override { return (*getNestedColumn())[n]; }
|
||||||
void get(size_t n, Field & res) const override { getNestedColumn()->get(n, res); }
|
void get(size_t n, Field & res) const override { getNestedColumn()->get(n, res); }
|
||||||
|
std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override
|
||||||
|
{
|
||||||
|
return getNestedColumn()->getValueNameAndType(n);
|
||||||
|
}
|
||||||
bool isDefaultAt(size_t n) const override { return n == 0; }
|
bool isDefaultAt(size_t n) const override { return n == 0; }
|
||||||
StringRef getDataAt(size_t n) const override { return getNestedColumn()->getDataAt(n); }
|
StringRef getDataAt(size_t n) const override { return getNestedColumn()->getDataAt(n); }
|
||||||
UInt64 get64(size_t n) const override { return getNestedColumn()->get64(n); }
|
UInt64 get64(size_t n) const override { return getNestedColumn()->get64(n); }
|
||||||
|
@ -1,3 +1,5 @@
|
|||||||
|
#include <DataTypes/DataTypeNothing.h>
|
||||||
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
#include <Columns/ColumnVariant.h>
|
#include <Columns/ColumnVariant.h>
|
||||||
|
|
||||||
#include <Columns/ColumnCompressed.h>
|
#include <Columns/ColumnCompressed.h>
|
||||||
@ -404,6 +406,15 @@ void ColumnVariant::get(size_t n, Field & res) const
|
|||||||
variants[discr]->get(offsetAt(n), res);
|
variants[discr]->get(offsetAt(n), res);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> ColumnVariant::getValueNameAndType(size_t n) const
|
||||||
|
{
|
||||||
|
Discriminator discr = localDiscriminatorAt(n);
|
||||||
|
if (discr == NULL_DISCRIMINATOR)
|
||||||
|
return {"NULL", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeNothing>())};
|
||||||
|
|
||||||
|
return variants[discr]->getValueNameAndType(offsetAt(n));
|
||||||
|
}
|
||||||
|
|
||||||
bool ColumnVariant::isDefaultAt(size_t n) const
|
bool ColumnVariant::isDefaultAt(size_t n) const
|
||||||
{
|
{
|
||||||
return localDiscriminatorAt(n) == NULL_DISCRIMINATOR;
|
return localDiscriminatorAt(n) == NULL_DISCRIMINATOR;
|
||||||
|
@ -173,6 +173,7 @@ public:
|
|||||||
|
|
||||||
Field operator[](size_t n) const override;
|
Field operator[](size_t n) const override;
|
||||||
void get(size_t n, Field & res) const override;
|
void get(size_t n, Field & res) const override;
|
||||||
|
virtual std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override;
|
||||||
|
|
||||||
bool isDefaultAt(size_t n) const override;
|
bool isDefaultAt(size_t n) const override;
|
||||||
bool isNullAt(size_t n) const override;
|
bool isNullAt(size_t n) const override;
|
||||||
|
@ -1,5 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <DataTypes/FieldToDataType.h>
|
||||||
|
#include <Common/FieldVisitorToString.h>
|
||||||
#include <Columns/ColumnFixedSizeHelper.h>
|
#include <Columns/ColumnFixedSizeHelper.h>
|
||||||
#include <Columns/IColumn.h>
|
#include <Columns/IColumn.h>
|
||||||
#include <Columns/IColumnImpl.h>
|
#include <Columns/IColumnImpl.h>
|
||||||
@ -207,6 +209,13 @@ public:
|
|||||||
res = (*this)[n];
|
res = (*this)[n];
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override
|
||||||
|
{
|
||||||
|
assert(n < data.size()); /// This assert is more strict than the corresponding assert inside PODArray.
|
||||||
|
const auto & val = castToNearestFieldType(data[n]);
|
||||||
|
return {FieldVisitorToString()(val), FieldToDataType()(val)};
|
||||||
|
}
|
||||||
|
|
||||||
UInt64 get64(size_t n) const override;
|
UInt64 get64(size_t n) const override;
|
||||||
|
|
||||||
Float64 getFloat64(size_t n) const override;
|
Float64 getFloat64(size_t n) const override;
|
||||||
|
@ -34,6 +34,8 @@ class ColumnGathererStream;
|
|||||||
class Field;
|
class Field;
|
||||||
class WeakHash32;
|
class WeakHash32;
|
||||||
class ColumnConst;
|
class ColumnConst;
|
||||||
|
class IDataType;
|
||||||
|
using DataTypePtr = std::shared_ptr<const IDataType>;
|
||||||
|
|
||||||
/// A range of column values between row indexes `from` and `to`. The name "equal range" is due to table sorting as its main use case: With
|
/// A range of column values between row indexes `from` and `to`. The name "equal range" is due to table sorting as its main use case: With
|
||||||
/// a PRIMARY KEY (c_pk1, c_pk2, ...), the first PK column is fully sorted. The second PK column is sorted within equal-value runs of the
|
/// a PRIMARY KEY (c_pk1, c_pk2, ...), the first PK column is fully sorted. The second PK column is sorted within equal-value runs of the
|
||||||
@ -144,6 +146,8 @@ public:
|
|||||||
/// Like the previous one, but avoids extra copying if Field is in a container, for example.
|
/// Like the previous one, but avoids extra copying if Field is in a container, for example.
|
||||||
virtual void get(size_t n, Field & res) const = 0;
|
virtual void get(size_t n, Field & res) const = 0;
|
||||||
|
|
||||||
|
virtual std::pair<String, DataTypePtr> getValueNameAndType(size_t) const = 0;
|
||||||
|
|
||||||
/// If possible, returns pointer to memory chunk which contains n-th element (if it isn't possible, throws an exception)
|
/// If possible, returns pointer to memory chunk which contains n-th element (if it isn't possible, throws an exception)
|
||||||
/// Is used to optimize some computations (in aggregation, for example).
|
/// Is used to optimize some computations (in aggregation, for example).
|
||||||
[[nodiscard]] virtual StringRef getDataAt(size_t n) const = 0;
|
[[nodiscard]] virtual StringRef getDataAt(size_t n) const = 0;
|
||||||
|
@ -25,6 +25,11 @@ void IColumnDummy::get(size_t, Field &) const
|
|||||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get value from {}", getName());
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get value from {}", getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<String, DataTypePtr> IColumnDummy::getValueNameAndType(size_t) const
|
||||||
|
{
|
||||||
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get value name and type from {}", getName());
|
||||||
|
}
|
||||||
|
|
||||||
void IColumnDummy::insert(const Field &)
|
void IColumnDummy::insert(const Field &)
|
||||||
{
|
{
|
||||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot insert element into {}", getName());
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot insert element into {}", getName());
|
||||||
|
@ -40,6 +40,7 @@ public:
|
|||||||
|
|
||||||
Field operator[](size_t) const override;
|
Field operator[](size_t) const override;
|
||||||
void get(size_t, Field &) const override;
|
void get(size_t, Field &) const override;
|
||||||
|
virtual std::pair<String, DataTypePtr> getValueNameAndType(size_t n) const override;
|
||||||
void insert(const Field &) override;
|
void insert(const Field &) override;
|
||||||
bool tryInsert(const Field &) override { return false; }
|
bool tryInsert(const Field &) override { return false; }
|
||||||
bool isDefaultAt(size_t) const override;
|
bool isDefaultAt(size_t) const override;
|
||||||
|
@ -51,6 +51,18 @@ static inline String formatQuoted(T x)
|
|||||||
return wb.str();
|
return wb.str();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
static inline String formatQuoted(const Decimal<T> & x, UInt32 scale)
|
||||||
|
{
|
||||||
|
WriteBufferFromOwnString wb;
|
||||||
|
|
||||||
|
writeChar('\'', wb);
|
||||||
|
writeText(x, scale, wb, {});
|
||||||
|
writeChar('\'', wb);
|
||||||
|
|
||||||
|
return wb.str();
|
||||||
|
}
|
||||||
|
|
||||||
/** In contrast to writeFloatText (and writeQuoted),
|
/** In contrast to writeFloatText (and writeQuoted),
|
||||||
* even if number looks like integer after formatting, prints decimal point nevertheless (for example, Float64(1) is printed as 1.).
|
* even if number looks like integer after formatting, prints decimal point nevertheless (for example, Float64(1) is printed as 1.).
|
||||||
* - because resulting text must be able to be parsed back as Float64 by query parser (otherwise it will be parsed as integer).
|
* - because resulting text must be able to be parsed back as Float64 by query parser (otherwise it will be parsed as integer).
|
||||||
@ -81,6 +93,10 @@ String FieldVisitorToString::operator() (const DecimalField<Decimal32> & x) cons
|
|||||||
String FieldVisitorToString::operator() (const DecimalField<Decimal64> & x) const { return formatQuoted(x); }
|
String FieldVisitorToString::operator() (const DecimalField<Decimal64> & x) const { return formatQuoted(x); }
|
||||||
String FieldVisitorToString::operator() (const DecimalField<Decimal128> & x) const { return formatQuoted(x); }
|
String FieldVisitorToString::operator() (const DecimalField<Decimal128> & x) const { return formatQuoted(x); }
|
||||||
String FieldVisitorToString::operator() (const DecimalField<Decimal256> & x) const { return formatQuoted(x); }
|
String FieldVisitorToString::operator() (const DecimalField<Decimal256> & x) const { return formatQuoted(x); }
|
||||||
|
String FieldVisitorToString::operator() (const Decimal32 & x, UInt32 scale) const { return formatQuoted(x, scale); }
|
||||||
|
String FieldVisitorToString::operator() (const Decimal64 & x, UInt32 scale) const { return formatQuoted(x, scale); }
|
||||||
|
String FieldVisitorToString::operator() (const Decimal128 & x, UInt32 scale) const { return formatQuoted(x, scale); }
|
||||||
|
String FieldVisitorToString::operator() (const Decimal256 & x, UInt32 scale) const { return formatQuoted(x, scale); }
|
||||||
String FieldVisitorToString::operator() (const Int128 & x) const { return formatQuoted(x); }
|
String FieldVisitorToString::operator() (const Int128 & x) const { return formatQuoted(x); }
|
||||||
String FieldVisitorToString::operator() (const UInt128 & x) const { return formatQuoted(x); }
|
String FieldVisitorToString::operator() (const UInt128 & x) const { return formatQuoted(x); }
|
||||||
String FieldVisitorToString::operator() (const UInt256 & x) const { return formatQuoted(x); }
|
String FieldVisitorToString::operator() (const UInt256 & x) const { return formatQuoted(x); }
|
||||||
|
@ -29,6 +29,10 @@ public:
|
|||||||
String operator() (const DecimalField<Decimal64> & x) const;
|
String operator() (const DecimalField<Decimal64> & x) const;
|
||||||
String operator() (const DecimalField<Decimal128> & x) const;
|
String operator() (const DecimalField<Decimal128> & x) const;
|
||||||
String operator() (const DecimalField<Decimal256> & x) const;
|
String operator() (const DecimalField<Decimal256> & x) const;
|
||||||
|
String operator() (const Decimal32 & x, UInt32 scale) const;
|
||||||
|
String operator() (const Decimal64 & x, UInt32 scale) const;
|
||||||
|
String operator() (const Decimal128 & x, UInt32 scale) const;
|
||||||
|
String operator() (const Decimal256 & x, UInt32 scale) const;
|
||||||
String operator() (const AggregateFunctionStateData & x) const;
|
String operator() (const AggregateFunctionStateData & x) const;
|
||||||
String operator() (const CustomType & x) const;
|
String operator() (const CustomType & x) const;
|
||||||
String operator() (const bool & x) const;
|
String operator() (const bool & x) const;
|
||||||
|
@ -130,6 +130,34 @@ DataTypePtr FieldToDataType<on_error>::operator() (const DecimalField<Decimal256
|
|||||||
return std::make_shared<Type>(Type::maxPrecision(), x.getScale());
|
return std::make_shared<Type>(Type::maxPrecision(), x.getScale());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
template <LeastSupertypeOnError on_error>
|
||||||
|
DataTypePtr FieldToDataType<on_error>::operator() (const Decimal32 &, UInt32 scale) const
|
||||||
|
{
|
||||||
|
using Type = DataTypeDecimal<Decimal32>;
|
||||||
|
return std::make_shared<Type>(Type::maxPrecision(), scale);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <LeastSupertypeOnError on_error>
|
||||||
|
DataTypePtr FieldToDataType<on_error>::operator() (const Decimal64 &, UInt32 scale) const
|
||||||
|
{
|
||||||
|
using Type = DataTypeDecimal<Decimal64>;
|
||||||
|
return std::make_shared<Type>(Type::maxPrecision(), scale);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <LeastSupertypeOnError on_error>
|
||||||
|
DataTypePtr FieldToDataType<on_error>::operator() (const Decimal128 &, UInt32 scale) const
|
||||||
|
{
|
||||||
|
using Type = DataTypeDecimal<Decimal128>;
|
||||||
|
return std::make_shared<Type>(Type::maxPrecision(), scale);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <LeastSupertypeOnError on_error>
|
||||||
|
DataTypePtr FieldToDataType<on_error>::operator() (const Decimal256 &, UInt32 scale) const
|
||||||
|
{
|
||||||
|
using Type = DataTypeDecimal<Decimal256>;
|
||||||
|
return std::make_shared<Type>(Type::maxPrecision(), scale);
|
||||||
|
}
|
||||||
|
|
||||||
template <LeastSupertypeOnError on_error>
|
template <LeastSupertypeOnError on_error>
|
||||||
DataTypePtr FieldToDataType<on_error>::operator() (const Array & x) const
|
DataTypePtr FieldToDataType<on_error>::operator() (const Array & x) const
|
||||||
{
|
{
|
||||||
|
@ -40,6 +40,10 @@ public:
|
|||||||
DataTypePtr operator() (const DecimalField<Decimal64> & x) const;
|
DataTypePtr operator() (const DecimalField<Decimal64> & x) const;
|
||||||
DataTypePtr operator() (const DecimalField<Decimal128> & x) const;
|
DataTypePtr operator() (const DecimalField<Decimal128> & x) const;
|
||||||
DataTypePtr operator() (const DecimalField<Decimal256> & x) const;
|
DataTypePtr operator() (const DecimalField<Decimal256> & x) const;
|
||||||
|
DataTypePtr operator() (const Decimal32 & x, UInt32 scale) const;
|
||||||
|
DataTypePtr operator() (const Decimal64 & x, UInt32 scale) const;
|
||||||
|
DataTypePtr operator() (const Decimal128 & x, UInt32 scale) const;
|
||||||
|
DataTypePtr operator() (const Decimal256 & x, UInt32 scale) const;
|
||||||
DataTypePtr operator() (const AggregateFunctionStateData & x) const;
|
DataTypePtr operator() (const AggregateFunctionStateData & x) const;
|
||||||
DataTypePtr operator() (const CustomType & x) const;
|
DataTypePtr operator() (const CustomType & x) const;
|
||||||
DataTypePtr operator() (const UInt256 & x) const;
|
DataTypePtr operator() (const UInt256 & x) const;
|
||||||
|
@ -60,8 +60,8 @@ namespace
|
|||||||
*/
|
*/
|
||||||
String calculateActionNodeNameWithCastIfNeeded(const ConstantNode & constant_node)
|
String calculateActionNodeNameWithCastIfNeeded(const ConstantNode & constant_node)
|
||||||
{
|
{
|
||||||
const auto & [name, type, field_type] = constant_node.getFieldAttributes();
|
const auto & [name, type] = constant_node.getValueNameAndType();
|
||||||
bool requires_cast_call = constant_node.hasSourceExpression() || ConstantNode::requiresCastCall(type, field_type, constant_node.getResultType());
|
bool requires_cast_call = constant_node.hasSourceExpression() || ConstantNode::requiresCastCall(type, constant_node.getResultType());
|
||||||
|
|
||||||
WriteBufferFromOwnString buffer;
|
WriteBufferFromOwnString buffer;
|
||||||
if (requires_cast_call)
|
if (requires_cast_call)
|
||||||
@ -336,7 +336,8 @@ public:
|
|||||||
|
|
||||||
static String calculateConstantActionNodeName(const ConstantNode & constant_node)
|
static String calculateConstantActionNodeName(const ConstantNode & constant_node)
|
||||||
{
|
{
|
||||||
return std::get<String>(constant_node.getFieldAttributes());
|
const auto & [name, type] = constant_node.getValueNameAndType();
|
||||||
|
return name + "_" + constant_node.getResultType()->getName();
|
||||||
}
|
}
|
||||||
|
|
||||||
String calculateWindowNodeActionName(const QueryTreeNodePtr & function_nodew_node_, const QueryTreeNodePtr & window_node_)
|
String calculateWindowNodeActionName(const QueryTreeNodePtr & function_nodew_node_, const QueryTreeNodePtr & window_node_)
|
||||||
|
Loading…
Reference in New Issue
Block a user