Merge pull request #66178 from Avogar/variant-dynamic-null-subcolumn

Support null map subcolumn for Variant and Dynamic typed subcolumns
This commit is contained in:
Alexey Milovidov 2024-07-07 15:10:23 +00:00 committed by GitHub
commit 29affadfb6
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
26 changed files with 2299 additions and 1380 deletions

View File

@ -2,9 +2,11 @@
#include <DataTypes/Serializations/SerializationDynamic.h>
#include <DataTypes/Serializations/SerializationDynamicElement.h>
#include <DataTypes/Serializations/SerializationVariantElement.h>
#include <DataTypes/Serializations/SerializationVariantElementNullMap.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnDynamic.h>
#include <Columns/ColumnVariant.h>
#include <Core/Field.h>
@ -110,28 +112,58 @@ std::unique_ptr<IDataType::SubstreamData> DataTypeDynamic::getDynamicSubcolumnDa
}
/// Extract nested subcolumn of requested dynamic subcolumn if needed.
if (!subcolumn_nested_name.empty())
/// If requested subcolumn is null map, it's processed separately as there is no Nullable type yet.
bool is_null_map_subcolumn = subcolumn_nested_name == "null";
if (is_null_map_subcolumn)
{
res->type = std::make_shared<DataTypeUInt8>();
}
else if (!subcolumn_nested_name.empty())
{
res = getSubcolumnData(subcolumn_nested_name, *res, throw_if_null);
if (!res)
return nullptr;
}
res->serialization = std::make_shared<SerializationDynamicElement>(res->serialization, subcolumn_type->getName());
res->type = makeNullableOrLowCardinalityNullableSafe(res->type);
res->serialization = std::make_shared<SerializationDynamicElement>(res->serialization, subcolumn_type->getName(), is_null_map_subcolumn);
/// Make resulting subcolumn Nullable only if type subcolumn can be inside Nullable or can be LowCardinality(Nullable()).
bool make_subcolumn_nullable = subcolumn_type->canBeInsideNullable() || subcolumn_type->lowCardinality();
if (!is_null_map_subcolumn && make_subcolumn_nullable)
res->type = makeNullableOrLowCardinalityNullableSafe(res->type);
if (data.column)
{
if (discriminator)
{
/// Provided Dynamic column has subcolumn of this type, we should use VariantSubcolumnCreator to
/// Provided Dynamic column has subcolumn of this type, we should use VariantSubcolumnCreator/VariantNullMapSubcolumnCreator to
/// create full subcolumn from variant according to discriminators.
const auto & variant_column = assert_cast<const ColumnDynamic &>(*data.column).getVariantColumn();
auto creator = SerializationVariantElement::VariantSubcolumnCreator(variant_column.getLocalDiscriminatorsPtr(), "", *discriminator, variant_column.localDiscriminatorByGlobal(*discriminator));
res->column = creator.create(res->column);
std::unique_ptr<ISerialization::ISubcolumnCreator> creator;
if (is_null_map_subcolumn)
creator = std::make_unique<SerializationVariantElementNullMap::VariantNullMapSubcolumnCreator>(
variant_column.getLocalDiscriminatorsPtr(),
"",
*discriminator,
variant_column.localDiscriminatorByGlobal(*discriminator));
else
creator = std::make_unique<SerializationVariantElement::VariantSubcolumnCreator>(
variant_column.getLocalDiscriminatorsPtr(),
"",
*discriminator,
variant_column.localDiscriminatorByGlobal(*discriminator),
make_subcolumn_nullable);
res->column = creator->create(res->column);
}
/// Provided Dynamic column doesn't have subcolumn of this type, just create column filled with default values.
else if (is_null_map_subcolumn)
{
/// Fill null map with 1 when there is no such Dynamic subcolumn.
auto column = ColumnUInt8::create();
assert_cast<ColumnUInt8 &>(*column).getData().resize_fill(data.column->size(), 1);
res->column = std::move(column);
}
else
{
/// Provided Dynamic column doesn't have subcolumn of this type, just create column filled with default values.
auto column = res->type->createColumn();
column->insertManyDefaults(data.column->size());
res->column = std::move(column);

View File

@ -173,7 +173,7 @@ bool IDataType::hasDynamicSubcolumns() const
auto data = SubstreamData(getDefaultSerialization()).withType(getPtr());
auto callback = [&](const SubstreamPath &, const String &, const SubstreamData & subcolumn_data)
{
has_dynamic_subcolumns |= subcolumn_data.type->hasDynamicSubcolumnsData();
has_dynamic_subcolumns |= subcolumn_data.type && subcolumn_data.type->hasDynamicSubcolumnsData();
};
forEachSubcolumn(callback, data);
return has_dynamic_subcolumns;

View File

@ -64,6 +64,9 @@ String ISerialization::Substream::toString() const
if (type == VariantElement)
return fmt::format("VariantElement({})", variant_element_name);
if (type == VariantElementNullMap)
return fmt::format("VariantElementNullMap({}.null)", variant_element_name);
return String(magic_enum::enum_name(type));
}
@ -195,6 +198,8 @@ String getNameForSubstreamPath(
stream_name += ".variant_offsets";
else if (it->type == Substream::VariantElement)
stream_name += "." + it->variant_element_name;
else if (it->type == Substream::VariantElementNullMap)
stream_name += "." + it->variant_element_name + ".null";
else if (it->type == SubstreamType::DynamicStructure)
stream_name += ".dynamic_structure";
}
@ -395,7 +400,8 @@ bool ISerialization::hasSubcolumnForPath(const SubstreamPath & path, size_t pref
return path[last_elem].type == Substream::NullMap
|| path[last_elem].type == Substream::TupleElement
|| path[last_elem].type == Substream::ArraySizes
|| path[last_elem].type == Substream::VariantElement;
|| path[last_elem].type == Substream::VariantElement
|| path[last_elem].type == Substream::VariantElementNullMap;
}
ISerialization::SubstreamData ISerialization::createFromPath(const SubstreamPath & path, size_t prefix_len)

View File

@ -184,6 +184,7 @@ public:
VariantOffsets,
VariantElements,
VariantElement,
VariantElementNullMap,
DynamicData,
DynamicStructure,
@ -436,6 +437,9 @@ protected:
template <typename State, typename StatePtr>
State * checkAndGetState(const StatePtr & state) const;
template <typename State, typename StatePtr>
static State * checkAndGetState(const StatePtr & state, const ISerialization * serialization);
[[noreturn]] void throwUnexpectedDataAfterParsedValue(IColumn & column, ReadBuffer & istr, const FormatSettings &, const String & type_name) const;
};
@ -446,10 +450,16 @@ using SubstreamType = ISerialization::Substream::Type;
template <typename State, typename StatePtr>
State * ISerialization::checkAndGetState(const StatePtr & state) const
{
return checkAndGetState<State, StatePtr>(state, this);
}
template <typename State, typename StatePtr>
State * ISerialization::checkAndGetState(const StatePtr & state, const ISerialization * serialization)
{
if (!state)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Got empty state for {}", demangle(typeid(*this).name()));
"Got empty state for {}", demangle(typeid(*serialization).name()));
auto * state_concrete = typeid_cast<State *>(state.get());
if (!state_concrete)
@ -457,7 +467,7 @@ State * ISerialization::checkAndGetState(const StatePtr & state) const
auto & state_ref = *state;
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Invalid State for {}. Expected: {}, got {}",
demangle(typeid(*this).name()),
demangle(typeid(*serialization).name()),
demangle(typeid(State).name()),
demangle(typeid(state_ref).name()));
}

View File

@ -1,5 +1,6 @@
#include <DataTypes/Serializations/SerializationDynamicElement.h>
#include <DataTypes/Serializations/SerializationVariantElement.h>
#include <DataTypes/Serializations/SerializationVariantElementNullMap.h>
#include <DataTypes/Serializations/SerializationDynamic.h>
#include <DataTypes/DataTypeVariant.h>
#include <DataTypes/DataTypeFactory.h>
@ -77,7 +78,10 @@ void SerializationDynamicElement::deserializeBinaryBulkStatePrefix(
if (auto global_discr = assert_cast<const DataTypeVariant &>(*variant_type).tryGetVariantDiscriminator(dynamic_element_name))
{
settings.path.push_back(Substream::DynamicData);
dynamic_element_state->variant_serialization = std::make_shared<SerializationVariantElement>(nested_serialization, dynamic_element_name, *global_discr);
if (is_null_map_subcolumn)
dynamic_element_state->variant_serialization = std::make_shared<SerializationVariantElementNullMap>(dynamic_element_name, *global_discr);
else
dynamic_element_state->variant_serialization = std::make_shared<SerializationVariantElement>(nested_serialization, dynamic_element_name, *global_discr);
dynamic_element_state->variant_serialization->deserializeBinaryBulkStatePrefix(settings, dynamic_element_state->variant_element_state, cache);
settings.path.pop_back();
}
@ -98,7 +102,16 @@ void SerializationDynamicElement::deserializeBinaryBulkWithMultipleStreams(
SubstreamsCache * cache) const
{
if (!state)
{
if (is_null_map_subcolumn)
{
auto mutable_column = result_column->assumeMutable();
auto & data = assert_cast<ColumnUInt8 &>(*mutable_column).getData();
data.resize_fill(data.size() + limit, 1);
}
return;
}
auto * dynamic_element_state = checkAndGetState<DeserializeBinaryBulkStateDynamicElement>(state);
@ -108,6 +121,12 @@ void SerializationDynamicElement::deserializeBinaryBulkWithMultipleStreams(
dynamic_element_state->variant_serialization->deserializeBinaryBulkWithMultipleStreams(result_column, limit, settings, dynamic_element_state->variant_element_state, cache);
settings.path.pop_back();
}
else if (is_null_map_subcolumn)
{
auto mutable_column = result_column->assumeMutable();
auto & data = assert_cast<ColumnUInt8 &>(*mutable_column).getData();
data.resize_fill(data.size() + limit, 1);
}
else
{
auto mutable_column = result_column->assumeMutable();

View File

@ -13,11 +13,11 @@ private:
/// To be able to deserialize Dynamic element as a subcolumn
/// we need its type name and global discriminator.
String dynamic_element_name;
bool is_null_map_subcolumn;
public:
SerializationDynamicElement(const SerializationPtr & nested_, const String & dynamic_element_name_)
: SerializationWrapper(nested_)
, dynamic_element_name(dynamic_element_name_)
SerializationDynamicElement(const SerializationPtr & nested_, const String & dynamic_element_name_, bool is_null_map_subcolumn_ = false)
: SerializationWrapper(nested_), dynamic_element_name(dynamic_element_name_), is_null_map_subcolumn(is_null_map_subcolumn_)
{
}

View File

@ -1,5 +1,6 @@
#include <DataTypes/Serializations/SerializationVariant.h>
#include <DataTypes/Serializations/SerializationVariantElement.h>
#include <DataTypes/Serializations/SerializationVariantElementNullMap.h>
#include <DataTypes/Serializations/SerializationNumber.h>
#include <DataTypes/Serializations/SerializationNullable.h>
#include <DataTypes/Serializations/SerializationNamed.h>
@ -71,10 +72,16 @@ void SerializationVariant::enumerateStreams(
for (size_t i = 0; i < variants.size(); ++i)
{
settings.path.back().creator = std::make_shared<SerializationVariantElement::VariantSubcolumnCreator>(local_discriminators, variant_names[i], i, column_variant ? column_variant->localDiscriminatorByGlobal(i) : i);
DataTypePtr type = type_variant ? type_variant->getVariant(i) : nullptr;
settings.path.back().creator = std::make_shared<SerializationVariantElement::VariantSubcolumnCreator>(
local_discriminators,
variant_names[i],
i,
column_variant ? column_variant->localDiscriminatorByGlobal(i) : i,
!type || type->canBeInsideNullable() || type->lowCardinality());
auto variant_data = SubstreamData(variants[i])
.withType(type_variant ? type_variant->getVariant(i) : nullptr)
.withType(type)
.withColumn(column_variant ? column_variant->getVariantPtrByGlobalDiscriminator(i) : nullptr)
.withSerializationInfo(data.serialization_info)
.withDeserializeState(variant_deserialize_state ? variant_deserialize_state->variant_states[i] : nullptr);
@ -85,6 +92,24 @@ void SerializationVariant::enumerateStreams(
settings.path.pop_back();
}
/// Variant subcolumns like variant.Type have type Nullable(Type), so we want to support reading null map subcolumn from it: variant.Type.null.
/// Nullable column is created during deserialization of a variant subcolumn according to the discriminators, so we don't have actual Nullable
/// serialization with null map subcolumn. To be able to read null map subcolumn from the variant subcolumn we use special serialization
/// SerializationVariantElementNullMap.
auto null_map_data = SubstreamData(std::make_shared<SerializationNumber<UInt8>>())
.withType(type_variant ? std::make_shared<DataTypeUInt8>() : nullptr)
.withColumn(column_variant ? ColumnUInt8::create() : nullptr);
for (size_t i = 0; i < variants.size(); ++i)
{
settings.path.back().creator = std::make_shared<SerializationVariantElementNullMap::VariantNullMapSubcolumnCreator>(local_discriminators, variant_names[i], i, column_variant ? column_variant->localDiscriminatorByGlobal(i) : i);
settings.path.push_back(Substream::VariantElementNullMap);
settings.path.back().variant_element_name = variant_names[i];
settings.path.back().data = null_map_data;
callback(settings.path);
settings.path.pop_back();
}
settings.path.pop_back();
}

View File

@ -2,6 +2,7 @@
#include <DataTypes/Serializations/ISerialization.h>
#include <DataTypes/Serializations/SerializationVariantElement.h>
#include <DataTypes/Serializations/SerializationVariantElementNullMap.h>
namespace DB
{
@ -161,6 +162,7 @@ public:
private:
friend SerializationVariantElement;
friend SerializationVariantElementNullMap;
void addVariantElementToPath(SubstreamPath & path, size_t i) const;

View File

@ -115,7 +115,14 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams(
if (discriminators_state->mode.value == SerializationVariant::DiscriminatorsSerializationMode::BASIC)
SerializationNumber<ColumnVariant::Discriminator>().deserializeBinaryBulk(*variant_element_state->discriminators->assumeMutable(), *discriminators_stream, limit, 0);
else
variant_limit = deserializeCompactDiscriminators(variant_element_state->discriminators, limit, discriminators_stream, settings.continuous_reading, *variant_element_state);
variant_limit = deserializeCompactDiscriminators(
variant_element_state->discriminators,
variant_discriminator,
limit,
discriminators_stream,
settings.continuous_reading,
variant_element_state->discriminators_state,
this);
addToSubstreamsCache(cache, settings.path, variant_element_state->discriminators);
}
@ -224,12 +231,14 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams(
size_t SerializationVariantElement::deserializeCompactDiscriminators(
DB::ColumnPtr & discriminators_column,
ColumnVariant::Discriminator variant_discriminator,
size_t limit,
DB::ReadBuffer * stream,
bool continuous_reading,
DeserializeBinaryBulkStateVariantElement & variant_element_state) const
DeserializeBinaryBulkStatePtr & discriminators_state_,
const ISerialization * serialization)
{
auto * discriminators_state = checkAndGetState<SerializationVariant::DeserializeBinaryBulkStateVariantDiscriminators>(variant_element_state.discriminators_state);
auto * discriminators_state = checkAndGetState<SerializationVariant::DeserializeBinaryBulkStateVariantDiscriminators>(discriminators_state_, serialization);
auto & discriminators = assert_cast<ColumnVariant::ColumnDiscriminators &>(*discriminators_column->assumeMutable());
auto & discriminators_data = discriminators.getData();
@ -290,17 +299,19 @@ SerializationVariantElement::VariantSubcolumnCreator::VariantSubcolumnCreator(
const ColumnPtr & local_discriminators_,
const String & variant_element_name_,
ColumnVariant::Discriminator global_variant_discriminator_,
ColumnVariant::Discriminator local_variant_discriminator_)
ColumnVariant::Discriminator local_variant_discriminator_,
bool make_nullable_)
: local_discriminators(local_discriminators_)
, variant_element_name(variant_element_name_)
, global_variant_discriminator(global_variant_discriminator_)
, local_variant_discriminator(local_variant_discriminator_)
, make_nullable(make_nullable_)
{
}
DataTypePtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB::DataTypePtr & prev) const
{
return makeNullableOrLowCardinalityNullableSafe(prev);
return make_nullable ? makeNullableOrLowCardinalityNullableSafe(prev) : prev;
}
SerializationPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB::SerializationPtr & prev) const
@ -313,12 +324,12 @@ ColumnPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB:
/// Case when original Variant column contained only one non-empty variant and no NULLs.
/// In this case just use this variant.
if (prev->size() == local_discriminators->size())
return makeNullableOrLowCardinalityNullableSafe(prev);
return make_nullable ? makeNullableOrLowCardinalityNullableSafe(prev) : prev;
/// If this variant is empty, fill result column with default values.
if (prev->empty())
{
auto res = makeNullableOrLowCardinalityNullableSafe(prev)->cloneEmpty();
auto res = make_nullable ? makeNullableOrLowCardinalityNullableSafe(prev)->cloneEmpty() : prev->cloneEmpty();
res->insertManyDefaults(local_discriminators->size());
return res;
}
@ -333,16 +344,16 @@ ColumnPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB:
/// Now we can create new column from null-map and variant column using IColumn::expand.
auto res_column = IColumn::mutate(prev);
/// Special case for LowCardinality. We want the result to be LowCardinality(Nullable),
/// Special case for LowCardinality when we want the result to be LowCardinality(Nullable),
/// but we don't have a good way to apply null-mask for LowCardinality(), so, we first
/// convert our column to LowCardinality(Nullable()) and then use expand which will
/// fill rows with 0 in mask with default value (that is NULL).
if (prev->lowCardinality())
if (make_nullable && prev->lowCardinality())
res_column = assert_cast<ColumnLowCardinality &>(*res_column).cloneNullable();
res_column->expand(null_map, /*inverted = */ true);
if (res_column->canBeInsideNullable())
if (make_nullable && prev->canBeInsideNullable())
{
auto null_map_col = ColumnUInt8::create();
null_map_col->getData() = std::move(null_map);

View File

@ -9,6 +9,7 @@ namespace DB
{
class SerializationVariant;
class SerializationVariantElementNullMap;
/// Serialization for Variant element when we read it as a subcolumn.
class SerializationVariantElement final : public SerializationWrapper
@ -66,12 +67,14 @@ public:
const String variant_element_name;
const ColumnVariant::Discriminator global_variant_discriminator;
const ColumnVariant::Discriminator local_variant_discriminator;
bool make_nullable;
VariantSubcolumnCreator(
const ColumnPtr & local_discriminators_,
const String & variant_element_name_,
ColumnVariant::Discriminator global_variant_discriminator_,
ColumnVariant::Discriminator local_variant_discriminator_);
ColumnVariant::Discriminator local_variant_discriminator_,
bool make_nullable_);
DataTypePtr create(const DataTypePtr & prev) const override;
ColumnPtr create(const ColumnPtr & prev) const override;
@ -79,15 +82,18 @@ public:
};
private:
friend SerializationVariant;
friend SerializationVariantElementNullMap;
struct DeserializeBinaryBulkStateVariantElement;
size_t deserializeCompactDiscriminators(
static size_t deserializeCompactDiscriminators(
ColumnPtr & discriminators_column,
ColumnVariant::Discriminator variant_discriminator,
size_t limit,
ReadBuffer * stream,
bool continuous_reading,
DeserializeBinaryBulkStateVariantElement & variant_element_state) const;
DeserializeBinaryBulkStatePtr & discriminators_state_,
const ISerialization * serialization);
void addVariantToPath(SubstreamPath & path) const;
void removeVariantFromPath(SubstreamPath & path) const;

View File

@ -0,0 +1,190 @@
#include <Columns/ColumnLowCardinality.h>
#include <Columns/ColumnNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/Serializations/SerializationNumber.h>
#include <DataTypes/Serializations/SerializationVariant.h>
#include <DataTypes/Serializations/SerializationVariantElement.h>
#include <DataTypes/Serializations/SerializationVariantElementNullMap.h>
#include <IO/ReadHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
struct DeserializeBinaryBulkStateVariantElementNullMap : public ISerialization::DeserializeBinaryBulkState
{
/// During deserialization discriminators streams can be shared.
/// For example we can read several variant elements together: "select v.UInt32, v.String.null from table",
/// or we can read the whole variant and some of variant elements or their subcolumns: "select v, v.UInt32.null from table".
/// To read the same column from the same stream more than once we use substream cache,
/// but this cache stores the whole column, not only the current range.
/// During deserialization of variant elements or their subcolumns discriminators column is not stored
/// in the result column, so we need to store them inside deserialization state, so we can use
/// substream cache correctly.
ColumnPtr discriminators;
ISerialization::DeserializeBinaryBulkStatePtr discriminators_state;
};
void SerializationVariantElementNullMap::enumerateStreams(
DB::ISerialization::EnumerateStreamsSettings & settings,
const DB::ISerialization::StreamCallback & callback,
const DB::ISerialization::SubstreamData &) const
{
/// We will need stream for discriminators during deserialization.
settings.path.push_back(Substream::VariantDiscriminators);
callback(settings.path);
settings.path.pop_back();
}
void SerializationVariantElementNullMap::serializeBinaryBulkStatePrefix(
const IColumn &, SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const
{
throw Exception(
ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkStatePrefix is not implemented for SerializationVariantElementNullMap");
}
void SerializationVariantElementNullMap::serializeBinaryBulkStateSuffix(SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const
{
throw Exception(
ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkStateSuffix is not implemented for SerializationVariantElementNullMap");
}
void SerializationVariantElementNullMap::deserializeBinaryBulkStatePrefix(
DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsDeserializeStatesCache * cache) const
{
DeserializeBinaryBulkStatePtr discriminators_state = SerializationVariant::deserializeDiscriminatorsStatePrefix(settings, cache);
if (!discriminators_state)
return;
auto variant_element_null_map_state = std::make_shared<DeserializeBinaryBulkStateVariantElementNullMap>();
variant_element_null_map_state->discriminators_state = std::move(discriminators_state);
state = std::move(variant_element_null_map_state);
}
void SerializationVariantElementNullMap::serializeBinaryBulkWithMultipleStreams(
const IColumn &, size_t, size_t, SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const
{
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Method serializeBinaryBulkWithMultipleStreams is not implemented for SerializationVariantElementNullMap");
}
void SerializationVariantElementNullMap::deserializeBinaryBulkWithMultipleStreams(
ColumnPtr & result_column,
size_t limit,
DeserializeBinaryBulkSettings & settings,
DeserializeBinaryBulkStatePtr & state,
SubstreamsCache * cache) const
{
/// Deserialize discriminators from Variant column.
settings.path.push_back(Substream::VariantDiscriminators);
DeserializeBinaryBulkStateVariantElementNullMap * variant_element_null_map_state = nullptr;
std::optional<size_t> variant_limit;
if (auto cached_discriminators = getFromSubstreamsCache(cache, settings.path))
{
variant_element_null_map_state = checkAndGetState<DeserializeBinaryBulkStateVariantElementNullMap>(state);
variant_element_null_map_state->discriminators = cached_discriminators;
}
else if (auto * discriminators_stream = settings.getter(settings.path))
{
variant_element_null_map_state = checkAndGetState<DeserializeBinaryBulkStateVariantElementNullMap>(state);
auto * discriminators_state = checkAndGetState<SerializationVariant::DeserializeBinaryBulkStateVariantDiscriminators>(
variant_element_null_map_state->discriminators_state);
/// If we started to read a new column, reinitialize discriminators column in deserialization state.
if (!variant_element_null_map_state->discriminators || result_column->empty())
variant_element_null_map_state->discriminators = ColumnVariant::ColumnDiscriminators::create();
/// Deserialize discriminators according to serialization mode.
if (discriminators_state->mode.value == SerializationVariant::DiscriminatorsSerializationMode::BASIC)
SerializationNumber<ColumnVariant::Discriminator>().deserializeBinaryBulk(
*variant_element_null_map_state->discriminators->assumeMutable(), *discriminators_stream, limit, 0);
else
variant_limit = SerializationVariantElement::deserializeCompactDiscriminators(
variant_element_null_map_state->discriminators,
variant_discriminator,
limit,
discriminators_stream,
settings.continuous_reading,
variant_element_null_map_state->discriminators_state,
this);
addToSubstreamsCache(cache, settings.path, variant_element_null_map_state->discriminators);
}
else
{
/// There is no such stream or cached data, it means that there is no Variant column in this part (it could happen after alter table add column).
/// In such cases columns are filled with default values, but for null-map column default value should be 1, not 0. Fill column with 1 here instead.
MutableColumnPtr mutable_column = result_column->assumeMutable();
auto & data = assert_cast<ColumnUInt8 &>(*mutable_column).getData();
data.resize_fill(data.size() + limit, 1);
settings.path.pop_back();
return;
}
settings.path.pop_back();
MutableColumnPtr mutable_column = result_column->assumeMutable();
auto & data = assert_cast<ColumnUInt8 &>(*mutable_column).getData();
/// Check if there are no such variant in read range.
if (variant_limit && *variant_limit == 0)
{
data.resize_fill(data.size() + limit, 1);
}
/// Check if there is only our variant in read range.
else if (variant_limit && *variant_limit == limit)
{
data.resize_fill(data.size() + limit, 0);
}
/// Iterate through new discriminators to calculate the null map of our variant.
else
{
const auto & discriminators_data
= assert_cast<const ColumnVariant::ColumnDiscriminators &>(*variant_element_null_map_state->discriminators).getData();
size_t discriminators_offset = variant_element_null_map_state->discriminators->size() - limit;
for (size_t i = discriminators_offset; i != discriminators_data.size(); ++i)
data.push_back(discriminators_data[i] != variant_discriminator);
}
}
SerializationVariantElementNullMap::VariantNullMapSubcolumnCreator::VariantNullMapSubcolumnCreator(
const ColumnPtr & local_discriminators_,
const String & variant_element_name_,
ColumnVariant::Discriminator global_variant_discriminator_,
ColumnVariant::Discriminator local_variant_discriminator_)
: local_discriminators(local_discriminators_)
, variant_element_name(variant_element_name_)
, global_variant_discriminator(global_variant_discriminator_)
, local_variant_discriminator(local_variant_discriminator_)
{
}
DataTypePtr SerializationVariantElementNullMap::VariantNullMapSubcolumnCreator::create(const DB::DataTypePtr &) const
{
return std::make_shared<DataTypeUInt8>();
}
SerializationPtr SerializationVariantElementNullMap::VariantNullMapSubcolumnCreator::create(const DB::SerializationPtr &) const
{
return std::make_shared<SerializationVariantElementNullMap>(variant_element_name, global_variant_discriminator);
}
ColumnPtr SerializationVariantElementNullMap::VariantNullMapSubcolumnCreator::create(const DB::ColumnPtr &) const
{
/// Iterate through discriminators and create null-map for our variant.
auto null_map_col = ColumnUInt8::create();
auto & null_map_data = null_map_col->getData();
null_map_data.reserve(local_discriminators->size());
const auto & local_discriminators_data = assert_cast<const ColumnVariant::ColumnDiscriminators &>(*local_discriminators).getData();
for (auto local_discr : local_discriminators_data)
null_map_data.push_back(local_discr != local_variant_discriminator);
return null_map_col;
}
}

View File

@ -0,0 +1,107 @@
#pragma once
#include <DataTypes/Serializations/SimpleTextSerialization.h>
#include <DataTypes/DataTypeNullable.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnVariant.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
class SerializationVariant;
class SerializationVariantElement;
/// Serialization for Variant element null map when we read it as a subcolumn.
/// For example, variant.UInt64.null.
/// It requires separate serialization because there is no actual Nullable column
/// and we should construct null map from variant discriminators.
/// The implementation of deserializeBinaryBulk* methods is similar to SerializationVariantElement,
/// but differs in that there is no need to read the actual data of the variant, only discriminators.
class SerializationVariantElementNullMap final : public SimpleTextSerialization
{
public:
SerializationVariantElementNullMap(const String & variant_element_name_, ColumnVariant::Discriminator variant_discriminator_)
: variant_element_name(variant_element_name_), variant_discriminator(variant_discriminator_)
{
}
void enumerateStreams(
EnumerateStreamsSettings & settings,
const StreamCallback & callback,
const SubstreamData & data) const override;
void serializeBinaryBulkStatePrefix(
const IColumn & column,
SerializeBinaryBulkSettings & settings,
SerializeBinaryBulkStatePtr & state) const override;
void serializeBinaryBulkStateSuffix(
SerializeBinaryBulkSettings & settings,
SerializeBinaryBulkStatePtr & state) const override;
void deserializeBinaryBulkStatePrefix(
DeserializeBinaryBulkSettings & settings,
DeserializeBinaryBulkStatePtr & state,
SubstreamsDeserializeStatesCache * cache) const override;
void serializeBinaryBulkWithMultipleStreams(
const IColumn & column,
size_t offset,
size_t limit,
SerializeBinaryBulkSettings & settings,
SerializeBinaryBulkStatePtr & state) const override;
void deserializeBinaryBulkWithMultipleStreams(
ColumnPtr & column,
size_t limit,
DeserializeBinaryBulkSettings & settings,
DeserializeBinaryBulkStatePtr & state,
SubstreamsCache * cache) const override;
void serializeBinary(const Field &, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void deserializeBinary(Field &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void serializeBinary(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void deserializeBinary(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void deserializeText(IColumn &, ReadBuffer &, const FormatSettings &, bool) const override { throwNoSerialization(); }
bool tryDeserializeText(IColumn &, ReadBuffer &, const FormatSettings &, bool) const override { throwNoSerialization(); }
struct VariantNullMapSubcolumnCreator : public ISubcolumnCreator
{
const ColumnPtr local_discriminators;
const String variant_element_name;
const ColumnVariant::Discriminator global_variant_discriminator;
const ColumnVariant::Discriminator local_variant_discriminator;
VariantNullMapSubcolumnCreator(
const ColumnPtr & local_discriminators_,
const String & variant_element_name_,
ColumnVariant::Discriminator global_variant_discriminator_,
ColumnVariant::Discriminator local_variant_discriminator_);
DataTypePtr create(const DataTypePtr & prev) const override;
ColumnPtr create(const ColumnPtr & prev) const override;
SerializationPtr create(const SerializationPtr & prev) const override;
};
private:
[[noreturn]] static void throwNoSerialization()
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Text/binary serialization is not implemented for variant element null map subcolumn");
}
friend SerializationVariant;
friend SerializationVariantElement;
/// To be able to deserialize Variant element null map as a subcolumn
/// we need variant element type name and global discriminator.
String variant_element_name;
ColumnVariant::Discriminator variant_discriminator;
};
}

File diff suppressed because it is too large Load Diff

View File

@ -6,9 +6,6 @@ test4 select
100000
100000
100000
100000
100000
100000
MergeTree compact
test4 insert
test4 select
@ -17,18 +14,12 @@ test4 select
100000
100000
100000
100000
100000
100000
test4 select
500000
100000
100000
100000
100000
100000
100000
100000
MergeTree wide
test4 insert
test4 select
@ -37,15 +28,9 @@ test4 select
100000
100000
100000
100000
100000
100000
test4 select
500000
100000
100000
100000
100000
100000
100000
100000

View File

@ -33,13 +33,10 @@ select v.\`LowCardinality(String)\` from test format Null;
select count() from test where isNotNull(v.\`LowCardinality(String)\`);
select v.\`Tuple(a UInt32, b UInt32)\` from test format Null;
select v.\`Tuple(a UInt32, b UInt32)\`.a from test format Null;
select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.a);
select v.\`Tuple(a UInt32, b UInt32)\`.b from test format Null;
select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.b);
select v.\`Array(UInt64)\` from test format Null;
select count() from test where not empty(v.\`Array(UInt64)\`);
select v.\`Array(UInt64)\`.size0 from test format Null;
select count() from test where isNotNull(v.\`Array(UInt64)\`.size0);"
select v.\`Array(UInt64)\`.size0 from test format Null;"
}
function run()

View File

@ -6,9 +6,6 @@ test5 select
100000
100000
100000
100000
100000
100000
MergeTree compact
test5 insert
test5 select
@ -17,18 +14,12 @@ test5 select
100000
100000
100000
100000
100000
100000
test5 select
500000
100000
100000
100000
100000
100000
100000
100000
MergeTree wide
test5 insert
test5 select
@ -37,15 +28,9 @@ test5 select
100000
100000
100000
100000
100000
100000
test5 select
500000
100000
100000
100000
100000
100000
100000
100000

View File

@ -35,13 +35,10 @@ select v.\`LowCardinality(String)\` from test format Null;
select count() from test where isNotNull(v.\`LowCardinality(String)\`);
select v.\`Tuple(a UInt32, b UInt32)\` from test format Null;
select v.\`Tuple(a UInt32, b UInt32)\`.a from test format Null;
select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.a);
select v.\`Tuple(a UInt32, b UInt32)\`.b from test format Null;
select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.b);
select v.\`Array(UInt64)\` from test format Null;
select count() from test where not empty(v.\`Array(UInt64)\`);
select v.\`Array(UInt64)\`.size0 from test format Null;
select count() from test where isNotNull(v.\`Array(UInt64)\`.size0);"
select v.\`Array(UInt64)\`.size0 from test format Null;"
}
function run()

View File

@ -6,9 +6,6 @@ test6 select
200000
200000
200000
200000
200000
200000
-----------------------------------------------------------------------------------------------------------
MergeTree compact
test6 insert
@ -18,9 +15,6 @@ test6 select
200000
200000
200000
200000
200000
200000
-----------------------------------------------------------------------------------------------------------
test6 select
1000000
@ -28,9 +22,6 @@ test6 select
200000
200000
200000
200000
200000
200000
-----------------------------------------------------------------------------------------------------------
MergeTree wide
test6 insert
@ -40,9 +31,6 @@ test6 select
200000
200000
200000
200000
200000
200000
-----------------------------------------------------------------------------------------------------------
test6 select
1000000
@ -50,7 +38,4 @@ test6 select
200000
200000
200000
200000
200000
200000
-----------------------------------------------------------------------------------------------------------

View File

@ -29,13 +29,10 @@ function test6_select()
select count() from test where isNotNull(v.\`LowCardinality(String)\`);
select v.\`Tuple(a UInt32, b UInt32)\` from test format Null;
select v.\`Tuple(a UInt32, b UInt32)\`.a from test format Null;
select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.a);
select v.\`Tuple(a UInt32, b UInt32)\`.b from test format Null;
select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.b);
select v.\`Array(UInt64)\` from test format Null;
select count() from test where not empty(v.\`Array(UInt64)\`);
select v.\`Array(UInt64)\`.size0 from test format Null;
select count() from test where isNotNull(v.\`Array(UInt64)\`.size0);"
select v.\`Array(UInt64)\`.size0 from test format Null;"
echo "-----------------------------------------------------------------------------------------------------------"
}

View File

@ -2,525 +2,525 @@ Memory
initial insert
alter add column 1
3 None
0 0 \N \N \N \N
1 1 \N \N \N \N
2 2 \N \N \N \N
0 0 \N \N \N 0
1 1 \N \N \N 0
2 2 \N \N \N 0
insert after alter add column 1
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 \N 3 \N 0
4 4 4 \N 4 \N 0
5 5 5 \N 5 \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 \N 12 \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
alter modify column 1
7 None
8 String
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 3 \N \N \N
4 4 4 4 \N \N \N
5 5 5 5 \N \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 12 \N \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 3 \N \N 0
4 4 4 4 \N \N 0
5 5 5 5 \N \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 12 \N \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
insert after alter modify column 1
8 None
11 String
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 3 \N \N \N
4 4 4 4 \N \N \N
5 5 5 5 \N \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 12 \N \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
15 15 \N \N \N \N \N
16 16 16 16 \N \N \N
17 17 str_17 str_17 \N \N \N
18 18 1970-01-19 1970-01-19 \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 3 \N \N 0
4 4 4 4 \N \N 0
5 5 5 5 \N \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 12 \N \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
15 15 \N \N \N \N 0
16 16 16 16 \N \N 0
17 17 str_17 str_17 \N \N 0
18 18 1970-01-19 1970-01-19 \N \N 0
alter modify column 2
4 UInt64
7 String
8 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
15 15 \N \N \N \N \N
16 16 16 16 \N \N \N
17 17 str_17 str_17 \N \N \N
18 18 1970-01-19 1970-01-19 \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 \N 3 \N 0
4 4 4 \N 4 \N 0
5 5 5 \N 5 \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 \N 12 \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
15 15 \N \N \N \N 0
16 16 16 16 \N \N 0
17 17 str_17 str_17 \N \N 0
18 18 1970-01-19 1970-01-19 \N \N 0
insert after alter modify column 2
1 Date
5 UInt64
8 String
9 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
15 15 \N \N \N \N \N
16 16 16 16 \N \N \N
17 17 str_17 str_17 \N \N \N
18 18 1970-01-19 1970-01-19 \N \N \N
19 19 \N \N \N \N \N
20 20 20 \N 20 \N \N
21 21 str_21 str_21 \N \N \N
22 22 1970-01-23 \N \N 1970-01-23 \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 \N 3 \N 0
4 4 4 \N 4 \N 0
5 5 5 \N 5 \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 \N 12 \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
15 15 \N \N \N \N 0
16 16 16 16 \N \N 0
17 17 str_17 str_17 \N \N 0
18 18 1970-01-19 1970-01-19 \N \N 0
19 19 \N \N \N \N 0
20 20 20 \N 20 \N 0
21 21 str_21 str_21 \N \N 0
22 22 1970-01-23 \N \N 1970-01-23 0
alter modify column 3
1 Date
5 UInt64
8 String
9 None
0 0 0 \N \N \N \N \N \N
1 1 1 \N \N \N \N \N \N
2 2 2 \N \N \N \N \N \N
3 3 3 \N \N \N 3 \N \N
4 4 4 \N \N \N 4 \N \N
5 5 5 \N \N \N 5 \N \N
6 6 6 \N \N str_6 \N \N \N
7 7 7 \N \N str_7 \N \N \N
8 8 8 \N \N str_8 \N \N \N
9 9 9 \N \N \N \N \N \N
10 10 10 \N \N \N \N \N \N
11 11 11 \N \N \N \N \N \N
12 12 12 \N \N \N 12 \N \N
13 13 13 \N \N str_13 \N \N \N
14 14 14 \N \N \N \N \N \N
15 15 15 \N \N \N \N \N \N
16 16 16 \N \N 16 \N \N \N
17 17 17 \N \N str_17 \N \N \N
18 18 18 \N \N 1970-01-19 \N \N \N
19 19 19 \N \N \N \N \N \N
20 20 20 \N \N \N 20 \N \N
21 21 21 \N \N str_21 \N \N \N
22 22 22 \N \N \N \N 1970-01-23 \N
0 0 0 \N 0 \N \N \N 0
1 1 1 \N 0 \N \N \N 0
2 2 2 \N 0 \N \N \N 0
3 3 3 \N 0 \N 3 \N 0
4 4 4 \N 0 \N 4 \N 0
5 5 5 \N 0 \N 5 \N 0
6 6 6 \N 0 str_6 \N \N 0
7 7 7 \N 0 str_7 \N \N 0
8 8 8 \N 0 str_8 \N \N 0
9 9 9 \N 0 \N \N \N 0
10 10 10 \N 0 \N \N \N 0
11 11 11 \N 0 \N \N \N 0
12 12 12 \N 0 \N 12 \N 0
13 13 13 \N 0 str_13 \N \N 0
14 14 14 \N 0 \N \N \N 0
15 15 15 \N 0 \N \N \N 0
16 16 16 \N 0 16 \N \N 0
17 17 17 \N 0 str_17 \N \N 0
18 18 18 \N 0 1970-01-19 \N \N 0
19 19 19 \N 0 \N \N \N 0
20 20 20 \N 0 \N 20 \N 0
21 21 21 \N 0 str_21 \N \N 0
22 22 22 \N 0 \N \N 1970-01-23 0
insert after alter modify column 3
1 Date
5 UInt64
8 String
12 None
0 0 0 \N \N \N \N \N \N
1 1 1 \N \N \N \N \N \N
2 2 2 \N \N \N \N \N \N
3 3 3 \N \N \N 3 \N \N
4 4 4 \N \N \N 4 \N \N
5 5 5 \N \N \N 5 \N \N
6 6 6 \N \N str_6 \N \N \N
7 7 7 \N \N str_7 \N \N \N
8 8 8 \N \N str_8 \N \N \N
9 9 9 \N \N \N \N \N \N
10 10 10 \N \N \N \N \N \N
11 11 11 \N \N \N \N \N \N
12 12 12 \N \N \N 12 \N \N
13 13 13 \N \N str_13 \N \N \N
14 14 14 \N \N \N \N \N \N
15 15 15 \N \N \N \N \N \N
16 16 16 \N \N 16 \N \N \N
17 17 17 \N \N str_17 \N \N \N
18 18 18 \N \N 1970-01-19 \N \N \N
19 19 19 \N \N \N \N \N \N
20 20 20 \N \N \N 20 \N \N
21 21 21 \N \N str_21 \N \N \N
22 22 22 \N \N \N \N 1970-01-23 \N
23 \N \N \N \N \N \N \N \N
24 24 24 \N \N \N \N \N \N
25 str_25 \N str_25 \N \N \N \N \N
0 0 0 \N 0 \N \N \N 0
1 1 1 \N 0 \N \N \N 0
2 2 2 \N 0 \N \N \N 0
3 3 3 \N 0 \N 3 \N 0
4 4 4 \N 0 \N 4 \N 0
5 5 5 \N 0 \N 5 \N 0
6 6 6 \N 0 str_6 \N \N 0
7 7 7 \N 0 str_7 \N \N 0
8 8 8 \N 0 str_8 \N \N 0
9 9 9 \N 0 \N \N \N 0
10 10 10 \N 0 \N \N \N 0
11 11 11 \N 0 \N \N \N 0
12 12 12 \N 0 \N 12 \N 0
13 13 13 \N 0 str_13 \N \N 0
14 14 14 \N 0 \N \N \N 0
15 15 15 \N 0 \N \N \N 0
16 16 16 \N 0 16 \N \N 0
17 17 17 \N 0 str_17 \N \N 0
18 18 18 \N 0 1970-01-19 \N \N 0
19 19 19 \N 0 \N \N \N 0
20 20 20 \N 0 \N 20 \N 0
21 21 21 \N 0 str_21 \N \N 0
22 22 22 \N 0 \N \N 1970-01-23 0
23 \N \N \N 0 \N \N \N 0
24 24 24 \N 0 \N \N \N 0
25 str_25 \N str_25 0 \N \N \N 0
MergeTree compact
initial insert
alter add column 1
3 None
0 0 \N \N \N \N
1 1 \N \N \N \N
2 2 \N \N \N \N
0 0 \N \N \N 0
1 1 \N \N \N 0
2 2 \N \N \N 0
insert after alter add column 1
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 \N 3 \N 0
4 4 4 \N 4 \N 0
5 5 5 \N 5 \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 \N 12 \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
alter modify column 1
7 None
8 String
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 3 \N \N \N
4 4 4 4 \N \N \N
5 5 5 5 \N \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 12 \N \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 3 \N \N 0
4 4 4 4 \N \N 0
5 5 5 5 \N \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 12 \N \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
insert after alter modify column 1
8 None
11 String
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 3 \N \N \N
4 4 4 4 \N \N \N
5 5 5 5 \N \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 12 \N \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
15 15 \N \N \N \N \N
16 16 16 16 \N \N \N
17 17 str_17 str_17 \N \N \N
18 18 1970-01-19 1970-01-19 \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 3 \N \N 0
4 4 4 4 \N \N 0
5 5 5 5 \N \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 12 \N \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
15 15 \N \N \N \N 0
16 16 16 16 \N \N 0
17 17 str_17 str_17 \N \N 0
18 18 1970-01-19 1970-01-19 \N \N 0
alter modify column 2
8 None
11 String
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 3 \N \N \N
4 4 4 4 \N \N \N
5 5 5 5 \N \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 12 \N \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
15 15 \N \N \N \N \N
16 16 16 16 \N \N \N
17 17 str_17 str_17 \N \N \N
18 18 1970-01-19 1970-01-19 \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 3 \N \N 0
4 4 4 4 \N \N 0
5 5 5 5 \N \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 12 \N \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
15 15 \N \N \N \N 0
16 16 16 16 \N \N 0
17 17 str_17 str_17 \N \N 0
18 18 1970-01-19 1970-01-19 \N \N 0
insert after alter modify column 2
1 Date
1 UInt64
9 None
12 String
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 3 \N \N \N
4 4 4 4 \N \N \N
5 5 5 5 \N \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 12 \N \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
15 15 \N \N \N \N \N
16 16 16 16 \N \N \N
17 17 str_17 str_17 \N \N \N
18 18 1970-01-19 1970-01-19 \N \N \N
19 19 \N \N \N \N \N
20 20 20 \N 20 \N \N
21 21 str_21 str_21 \N \N \N
22 22 1970-01-23 \N \N 1970-01-23 \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 3 \N \N 0
4 4 4 4 \N \N 0
5 5 5 5 \N \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 12 \N \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
15 15 \N \N \N \N 0
16 16 16 16 \N \N 0
17 17 str_17 str_17 \N \N 0
18 18 1970-01-19 1970-01-19 \N \N 0
19 19 \N \N \N \N 0
20 20 20 \N 20 \N 0
21 21 str_21 str_21 \N \N 0
22 22 1970-01-23 \N \N 1970-01-23 0
alter modify column 3
1 Date
1 UInt64
9 None
12 String
0 0 0 \N \N \N \N \N \N
1 1 1 \N \N \N \N \N \N
2 2 2 \N \N \N \N \N \N
3 3 3 \N \N 3 \N \N \N
4 4 4 \N \N 4 \N \N \N
5 5 5 \N \N 5 \N \N \N
6 6 6 \N \N str_6 \N \N \N
7 7 7 \N \N str_7 \N \N \N
8 8 8 \N \N str_8 \N \N \N
9 9 9 \N \N \N \N \N \N
10 10 10 \N \N \N \N \N \N
11 11 11 \N \N \N \N \N \N
12 12 12 \N \N 12 \N \N \N
13 13 13 \N \N str_13 \N \N \N
14 14 14 \N \N \N \N \N \N
15 15 15 \N \N \N \N \N \N
16 16 16 \N \N 16 \N \N \N
17 17 17 \N \N str_17 \N \N \N
18 18 18 \N \N 1970-01-19 \N \N \N
19 19 19 \N \N \N \N \N \N
20 20 20 \N \N \N 20 \N \N
21 21 21 \N \N str_21 \N \N \N
22 22 22 \N \N \N \N 1970-01-23 \N
0 0 0 \N 0 \N \N \N 0
1 1 1 \N 0 \N \N \N 0
2 2 2 \N 0 \N \N \N 0
3 3 3 \N 0 3 \N \N 0
4 4 4 \N 0 4 \N \N 0
5 5 5 \N 0 5 \N \N 0
6 6 6 \N 0 str_6 \N \N 0
7 7 7 \N 0 str_7 \N \N 0
8 8 8 \N 0 str_8 \N \N 0
9 9 9 \N 0 \N \N \N 0
10 10 10 \N 0 \N \N \N 0
11 11 11 \N 0 \N \N \N 0
12 12 12 \N 0 12 \N \N 0
13 13 13 \N 0 str_13 \N \N 0
14 14 14 \N 0 \N \N \N 0
15 15 15 \N 0 \N \N \N 0
16 16 16 \N 0 16 \N \N 0
17 17 17 \N 0 str_17 \N \N 0
18 18 18 \N 0 1970-01-19 \N \N 0
19 19 19 \N 0 \N \N \N 0
20 20 20 \N 0 \N 20 \N 0
21 21 21 \N 0 str_21 \N \N 0
22 22 22 \N 0 \N \N 1970-01-23 0
insert after alter modify column 3
1 Date
1 UInt64
12 None
12 String
0 0 0 \N \N \N \N \N \N
1 1 1 \N \N \N \N \N \N
2 2 2 \N \N \N \N \N \N
3 3 3 \N \N 3 \N \N \N
4 4 4 \N \N 4 \N \N \N
5 5 5 \N \N 5 \N \N \N
6 6 6 \N \N str_6 \N \N \N
7 7 7 \N \N str_7 \N \N \N
8 8 8 \N \N str_8 \N \N \N
9 9 9 \N \N \N \N \N \N
10 10 10 \N \N \N \N \N \N
11 11 11 \N \N \N \N \N \N
12 12 12 \N \N 12 \N \N \N
13 13 13 \N \N str_13 \N \N \N
14 14 14 \N \N \N \N \N \N
15 15 15 \N \N \N \N \N \N
16 16 16 \N \N 16 \N \N \N
17 17 17 \N \N str_17 \N \N \N
18 18 18 \N \N 1970-01-19 \N \N \N
19 19 19 \N \N \N \N \N \N
20 20 20 \N \N \N 20 \N \N
21 21 21 \N \N str_21 \N \N \N
22 22 22 \N \N \N \N 1970-01-23 \N
23 \N \N \N \N \N \N \N \N
24 24 24 \N \N \N \N \N \N
25 str_25 \N str_25 \N \N \N \N \N
0 0 0 \N 0 \N \N \N 0
1 1 1 \N 0 \N \N \N 0
2 2 2 \N 0 \N \N \N 0
3 3 3 \N 0 3 \N \N 0
4 4 4 \N 0 4 \N \N 0
5 5 5 \N 0 5 \N \N 0
6 6 6 \N 0 str_6 \N \N 0
7 7 7 \N 0 str_7 \N \N 0
8 8 8 \N 0 str_8 \N \N 0
9 9 9 \N 0 \N \N \N 0
10 10 10 \N 0 \N \N \N 0
11 11 11 \N 0 \N \N \N 0
12 12 12 \N 0 12 \N \N 0
13 13 13 \N 0 str_13 \N \N 0
14 14 14 \N 0 \N \N \N 0
15 15 15 \N 0 \N \N \N 0
16 16 16 \N 0 16 \N \N 0
17 17 17 \N 0 str_17 \N \N 0
18 18 18 \N 0 1970-01-19 \N \N 0
19 19 19 \N 0 \N \N \N 0
20 20 20 \N 0 \N 20 \N 0
21 21 21 \N 0 str_21 \N \N 0
22 22 22 \N 0 \N \N 1970-01-23 0
23 \N \N \N 0 \N \N \N 0
24 24 24 \N 0 \N \N \N 0
25 str_25 \N str_25 0 \N \N \N 0
MergeTree wide
initial insert
alter add column 1
3 None
0 0 \N \N \N \N
1 1 \N \N \N \N
2 2 \N \N \N \N
0 0 \N \N \N 0
1 1 \N \N \N 0
2 2 \N \N \N 0
insert after alter add column 1
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 \N 3 \N 0
4 4 4 \N 4 \N 0
5 5 5 \N 5 \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 \N 12 \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
alter modify column 1
7 None
8 String
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 3 \N \N \N
4 4 4 4 \N \N \N
5 5 5 5 \N \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 12 \N \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 3 \N \N 0
4 4 4 4 \N \N 0
5 5 5 5 \N \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 12 \N \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
insert after alter modify column 1
8 None
11 String
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 3 \N \N \N
4 4 4 4 \N \N \N
5 5 5 5 \N \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 12 \N \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
15 15 \N \N \N \N \N
16 16 16 16 \N \N \N
17 17 str_17 str_17 \N \N \N
18 18 1970-01-19 1970-01-19 \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 3 \N \N 0
4 4 4 4 \N \N 0
5 5 5 5 \N \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 12 \N \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
15 15 \N \N \N \N 0
16 16 16 16 \N \N 0
17 17 str_17 str_17 \N \N 0
18 18 1970-01-19 1970-01-19 \N \N 0
alter modify column 2
8 None
11 String
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 3 \N \N \N
4 4 4 4 \N \N \N
5 5 5 5 \N \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 12 \N \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
15 15 \N \N \N \N \N
16 16 16 16 \N \N \N
17 17 str_17 str_17 \N \N \N
18 18 1970-01-19 1970-01-19 \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 3 \N \N 0
4 4 4 4 \N \N 0
5 5 5 5 \N \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 12 \N \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
15 15 \N \N \N \N 0
16 16 16 16 \N \N 0
17 17 str_17 str_17 \N \N 0
18 18 1970-01-19 1970-01-19 \N \N 0
insert after alter modify column 2
1 Date
1 UInt64
9 None
12 String
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 3 \N \N \N
4 4 4 4 \N \N \N
5 5 5 5 \N \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 12 \N \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
15 15 \N \N \N \N \N
16 16 16 16 \N \N \N
17 17 str_17 str_17 \N \N \N
18 18 1970-01-19 1970-01-19 \N \N \N
19 19 \N \N \N \N \N
20 20 20 \N 20 \N \N
21 21 str_21 str_21 \N \N \N
22 22 1970-01-23 \N \N 1970-01-23 \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 3 \N \N 0
4 4 4 4 \N \N 0
5 5 5 5 \N \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 12 \N \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
15 15 \N \N \N \N 0
16 16 16 16 \N \N 0
17 17 str_17 str_17 \N \N 0
18 18 1970-01-19 1970-01-19 \N \N 0
19 19 \N \N \N \N 0
20 20 20 \N 20 \N 0
21 21 str_21 str_21 \N \N 0
22 22 1970-01-23 \N \N 1970-01-23 0
alter modify column 3
1 Date
1 UInt64
9 None
12 String
0 0 0 \N \N \N \N \N \N
1 1 1 \N \N \N \N \N \N
2 2 2 \N \N \N \N \N \N
3 3 3 \N \N 3 \N \N \N
4 4 4 \N \N 4 \N \N \N
5 5 5 \N \N 5 \N \N \N
6 6 6 \N \N str_6 \N \N \N
7 7 7 \N \N str_7 \N \N \N
8 8 8 \N \N str_8 \N \N \N
9 9 9 \N \N \N \N \N \N
10 10 10 \N \N \N \N \N \N
11 11 11 \N \N \N \N \N \N
12 12 12 \N \N 12 \N \N \N
13 13 13 \N \N str_13 \N \N \N
14 14 14 \N \N \N \N \N \N
15 15 15 \N \N \N \N \N \N
16 16 16 \N \N 16 \N \N \N
17 17 17 \N \N str_17 \N \N \N
18 18 18 \N \N 1970-01-19 \N \N \N
19 19 19 \N \N \N \N \N \N
20 20 20 \N \N \N 20 \N \N
21 21 21 \N \N str_21 \N \N \N
22 22 22 \N \N \N \N 1970-01-23 \N
0 0 0 \N 0 \N \N \N 0
1 1 1 \N 0 \N \N \N 0
2 2 2 \N 0 \N \N \N 0
3 3 3 \N 0 3 \N \N 0
4 4 4 \N 0 4 \N \N 0
5 5 5 \N 0 5 \N \N 0
6 6 6 \N 0 str_6 \N \N 0
7 7 7 \N 0 str_7 \N \N 0
8 8 8 \N 0 str_8 \N \N 0
9 9 9 \N 0 \N \N \N 0
10 10 10 \N 0 \N \N \N 0
11 11 11 \N 0 \N \N \N 0
12 12 12 \N 0 12 \N \N 0
13 13 13 \N 0 str_13 \N \N 0
14 14 14 \N 0 \N \N \N 0
15 15 15 \N 0 \N \N \N 0
16 16 16 \N 0 16 \N \N 0
17 17 17 \N 0 str_17 \N \N 0
18 18 18 \N 0 1970-01-19 \N \N 0
19 19 19 \N 0 \N \N \N 0
20 20 20 \N 0 \N 20 \N 0
21 21 21 \N 0 str_21 \N \N 0
22 22 22 \N 0 \N \N 1970-01-23 0
insert after alter modify column 3
1 Date
1 UInt64
12 None
12 String
0 0 0 \N \N \N \N \N \N
1 1 1 \N \N \N \N \N \N
2 2 2 \N \N \N \N \N \N
3 3 3 \N \N 3 \N \N \N
4 4 4 \N \N 4 \N \N \N
5 5 5 \N \N 5 \N \N \N
6 6 6 \N \N str_6 \N \N \N
7 7 7 \N \N str_7 \N \N \N
8 8 8 \N \N str_8 \N \N \N
9 9 9 \N \N \N \N \N \N
10 10 10 \N \N \N \N \N \N
11 11 11 \N \N \N \N \N \N
12 12 12 \N \N 12 \N \N \N
13 13 13 \N \N str_13 \N \N \N
14 14 14 \N \N \N \N \N \N
15 15 15 \N \N \N \N \N \N
16 16 16 \N \N 16 \N \N \N
17 17 17 \N \N str_17 \N \N \N
18 18 18 \N \N 1970-01-19 \N \N \N
19 19 19 \N \N \N \N \N \N
20 20 20 \N \N \N 20 \N \N
21 21 21 \N \N str_21 \N \N \N
22 22 22 \N \N \N \N 1970-01-23 \N
23 \N \N \N \N \N \N \N \N
24 24 24 \N \N \N \N \N \N
25 str_25 \N str_25 \N \N \N \N \N
0 0 0 \N 0 \N \N \N 0
1 1 1 \N 0 \N \N \N 0
2 2 2 \N 0 \N \N \N 0
3 3 3 \N 0 3 \N \N 0
4 4 4 \N 0 4 \N \N 0
5 5 5 \N 0 5 \N \N 0
6 6 6 \N 0 str_6 \N \N 0
7 7 7 \N 0 str_7 \N \N 0
8 8 8 \N 0 str_8 \N \N 0
9 9 9 \N 0 \N \N \N 0
10 10 10 \N 0 \N \N \N 0
11 11 11 \N 0 \N \N \N 0
12 12 12 \N 0 12 \N \N 0
13 13 13 \N 0 str_13 \N \N 0
14 14 14 \N 0 \N \N \N 0
15 15 15 \N 0 \N \N \N 0
16 16 16 \N 0 16 \N \N 0
17 17 17 \N 0 str_17 \N \N 0
18 18 18 \N 0 1970-01-19 \N \N 0
19 19 19 \N 0 \N \N \N 0
20 20 20 \N 0 \N 20 \N 0
21 21 21 \N 0 str_21 \N \N 0
22 22 22 \N 0 \N \N 1970-01-23 0
23 \N \N \N 0 \N \N \N 0
24 24 24 \N 0 \N \N \N 0
25 str_25 \N str_25 0 \N \N \N 0

View File

@ -2,181 +2,181 @@ MergeTree compact
initial insert
alter add column
3 None
0 0 \N \N \N \N
1 1 \N \N \N \N
2 2 \N \N \N \N
0 0 \N \N \N 0
1 1 \N \N \N 0
2 2 \N \N \N 0
insert after alter add column 1
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 \N 3 \N 0
4 4 4 \N 4 \N 0
5 5 5 \N 5 \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 \N 12 \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
alter rename column 1
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 \N 3 \N 0
4 4 4 \N 4 \N 0
5 5 5 \N 5 \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 \N 12 \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
insert nested dynamic
3 Array(Dynamic)
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N [] [] []
1 1 \N \N \N \N \N [] [] []
2 2 \N \N \N \N \N [] [] []
3 3 3 \N 3 \N \N [] [] []
4 4 4 \N 4 \N \N [] [] []
5 5 5 \N 5 \N \N [] [] []
6 6 str_6 str_6 \N \N \N [] [] []
7 7 str_7 str_7 \N \N \N [] [] []
8 8 str_8 str_8 \N \N \N [] [] []
9 9 \N \N \N \N \N [] [] []
10 10 \N \N \N \N \N [] [] []
11 11 \N \N \N \N \N [] [] []
12 12 12 \N 12 \N \N [] [] []
13 13 str_13 str_13 \N \N \N [] [] []
14 14 \N \N \N \N \N [] [] []
15 15 [15] \N \N \N \N [15] [NULL] [NULL]
16 16 ['str_16'] \N \N \N \N [NULL] ['str_16'] [NULL]
17 17 [17] \N \N \N \N [17] [NULL] [NULL]
0 0 \N \N \N \N 0 [] [] []
1 1 \N \N \N \N 0 [] [] []
2 2 \N \N \N \N 0 [] [] []
3 3 3 \N 3 \N 0 [] [] []
4 4 4 \N 4 \N 0 [] [] []
5 5 5 \N 5 \N 0 [] [] []
6 6 str_6 str_6 \N \N 0 [] [] []
7 7 str_7 str_7 \N \N 0 [] [] []
8 8 str_8 str_8 \N \N 0 [] [] []
9 9 \N \N \N \N 0 [] [] []
10 10 \N \N \N \N 0 [] [] []
11 11 \N \N \N \N 0 [] [] []
12 12 12 \N 12 \N 0 [] [] []
13 13 str_13 str_13 \N \N 0 [] [] []
14 14 \N \N \N \N 0 [] [] []
15 15 [15] \N \N \N 0 [15] [NULL] [NULL]
16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL]
17 17 [17] \N \N \N 0 [17] [NULL] [NULL]
alter rename column 2
3 Array(Dynamic)
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N [] [] []
1 1 \N \N \N \N \N [] [] []
2 2 \N \N \N \N \N [] [] []
3 3 3 \N 3 \N \N [] [] []
4 4 4 \N 4 \N \N [] [] []
5 5 5 \N 5 \N \N [] [] []
6 6 str_6 str_6 \N \N \N [] [] []
7 7 str_7 str_7 \N \N \N [] [] []
8 8 str_8 str_8 \N \N \N [] [] []
9 9 \N \N \N \N \N [] [] []
10 10 \N \N \N \N \N [] [] []
11 11 \N \N \N \N \N [] [] []
12 12 12 \N 12 \N \N [] [] []
13 13 str_13 str_13 \N \N \N [] [] []
14 14 \N \N \N \N \N [] [] []
15 15 [15] \N \N \N \N [15] [NULL] [NULL]
16 16 ['str_16'] \N \N \N \N [NULL] ['str_16'] [NULL]
17 17 [17] \N \N \N \N [17] [NULL] [NULL]
0 0 \N \N \N \N 0 [] [] []
1 1 \N \N \N \N 0 [] [] []
2 2 \N \N \N \N 0 [] [] []
3 3 3 \N 3 \N 0 [] [] []
4 4 4 \N 4 \N 0 [] [] []
5 5 5 \N 5 \N 0 [] [] []
6 6 str_6 str_6 \N \N 0 [] [] []
7 7 str_7 str_7 \N \N 0 [] [] []
8 8 str_8 str_8 \N \N 0 [] [] []
9 9 \N \N \N \N 0 [] [] []
10 10 \N \N \N \N 0 [] [] []
11 11 \N \N \N \N 0 [] [] []
12 12 12 \N 12 \N 0 [] [] []
13 13 str_13 str_13 \N \N 0 [] [] []
14 14 \N \N \N \N 0 [] [] []
15 15 [15] \N \N \N 0 [15] [NULL] [NULL]
16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL]
17 17 [17] \N \N \N 0 [17] [NULL] [NULL]
MergeTree wide
initial insert
alter add column
3 None
0 0 \N \N \N \N
1 1 \N \N \N \N
2 2 \N \N \N \N
0 0 \N \N \N 0
1 1 \N \N \N 0
2 2 \N \N \N 0
insert after alter add column 1
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 \N 3 \N 0
4 4 4 \N 4 \N 0
5 5 5 \N 5 \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 \N 12 \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
alter rename column 1
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 \N 3 \N 0
4 4 4 \N 4 \N 0
5 5 5 \N 5 \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 \N 12 \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
insert nested dynamic
3 Array(Dynamic)
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N [] [] []
1 1 \N \N \N \N \N [] [] []
2 2 \N \N \N \N \N [] [] []
3 3 3 \N 3 \N \N [] [] []
4 4 4 \N 4 \N \N [] [] []
5 5 5 \N 5 \N \N [] [] []
6 6 str_6 str_6 \N \N \N [] [] []
7 7 str_7 str_7 \N \N \N [] [] []
8 8 str_8 str_8 \N \N \N [] [] []
9 9 \N \N \N \N \N [] [] []
10 10 \N \N \N \N \N [] [] []
11 11 \N \N \N \N \N [] [] []
12 12 12 \N 12 \N \N [] [] []
13 13 str_13 str_13 \N \N \N [] [] []
14 14 \N \N \N \N \N [] [] []
15 15 [15] \N \N \N \N [15] [NULL] [NULL]
16 16 ['str_16'] \N \N \N \N [NULL] ['str_16'] [NULL]
17 17 [17] \N \N \N \N [17] [NULL] [NULL]
0 0 \N \N \N \N 0 [] [] []
1 1 \N \N \N \N 0 [] [] []
2 2 \N \N \N \N 0 [] [] []
3 3 3 \N 3 \N 0 [] [] []
4 4 4 \N 4 \N 0 [] [] []
5 5 5 \N 5 \N 0 [] [] []
6 6 str_6 str_6 \N \N 0 [] [] []
7 7 str_7 str_7 \N \N 0 [] [] []
8 8 str_8 str_8 \N \N 0 [] [] []
9 9 \N \N \N \N 0 [] [] []
10 10 \N \N \N \N 0 [] [] []
11 11 \N \N \N \N 0 [] [] []
12 12 12 \N 12 \N 0 [] [] []
13 13 str_13 str_13 \N \N 0 [] [] []
14 14 \N \N \N \N 0 [] [] []
15 15 [15] \N \N \N 0 [15] [NULL] [NULL]
16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL]
17 17 [17] \N \N \N 0 [17] [NULL] [NULL]
alter rename column 2
3 Array(Dynamic)
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N [] [] []
1 1 \N \N \N \N \N [] [] []
2 2 \N \N \N \N \N [] [] []
3 3 3 \N 3 \N \N [] [] []
4 4 4 \N 4 \N \N [] [] []
5 5 5 \N 5 \N \N [] [] []
6 6 str_6 str_6 \N \N \N [] [] []
7 7 str_7 str_7 \N \N \N [] [] []
8 8 str_8 str_8 \N \N \N [] [] []
9 9 \N \N \N \N \N [] [] []
10 10 \N \N \N \N \N [] [] []
11 11 \N \N \N \N \N [] [] []
12 12 12 \N 12 \N \N [] [] []
13 13 str_13 str_13 \N \N \N [] [] []
14 14 \N \N \N \N \N [] [] []
15 15 [15] \N \N \N \N [15] [NULL] [NULL]
16 16 ['str_16'] \N \N \N \N [NULL] ['str_16'] [NULL]
17 17 [17] \N \N \N \N [17] [NULL] [NULL]
0 0 \N \N \N \N 0 [] [] []
1 1 \N \N \N \N 0 [] [] []
2 2 \N \N \N \N 0 [] [] []
3 3 3 \N 3 \N 0 [] [] []
4 4 4 \N 4 \N 0 [] [] []
5 5 5 \N 5 \N 0 [] [] []
6 6 str_6 str_6 \N \N 0 [] [] []
7 7 str_7 str_7 \N \N 0 [] [] []
8 8 str_8 str_8 \N \N 0 [] [] []
9 9 \N \N \N \N 0 [] [] []
10 10 \N \N \N \N 0 [] [] []
11 11 \N \N \N \N 0 [] [] []
12 12 12 \N 12 \N 0 [] [] []
13 13 str_13 str_13 \N \N 0 [] [] []
14 14 \N \N \N \N 0 [] [] []
15 15 [15] \N \N \N 0 [15] [NULL] [NULL]
16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL]
17 17 [17] \N \N \N 0 [17] [NULL] [NULL]

View File

@ -2,55 +2,55 @@ MergeTree compact
initial insert
alter add column
3 None
0 0 \N \N \N \N
1 1 \N \N \N \N
2 2 \N \N \N \N
0 0 \N \N \N 0
1 1 \N \N \N 0
2 2 \N \N \N 0
insert after alter add column
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 \N 3 \N 0
4 4 4 \N 4 \N 0
5 5 5 \N 5 \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 \N 12 \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
check table
1
MergeTree wide
initial insert
alter add column
3 None
0 0 \N \N \N \N
1 1 \N \N \N \N
2 2 \N \N \N \N
0 0 \N \N \N 0
1 1 \N \N \N 0
2 2 \N \N \N 0
insert after alter add column
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
0 0 \N \N \N \N 0
1 1 \N \N \N \N 0
2 2 \N \N \N \N 0
3 3 3 \N 3 \N 0
4 4 4 \N 4 \N 0
5 5 5 \N 5 \N 0
6 6 str_6 str_6 \N \N 0
7 7 str_7 str_7 \N \N 0
8 8 str_8 str_8 \N \N 0
9 9 \N \N \N \N 0
10 10 \N \N \N \N 0
11 11 \N \N \N \N 0
12 12 12 \N 12 \N 0
13 13 str_13 str_13 \N \N 0
14 14 \N \N \N \N 0
check table
1

View File

@ -0,0 +1,402 @@
Memory
test
[] 1 0 0 []
1 0 1 0 []
\N 1 1 0 []
['str_3','str_3','str_3'] 1 0 3 [1,1,1]
4 0 1 0 []
\N 1 1 0 []
[6,6,6,6,6,6] 1 0 6 [0,0,0,0,0,0]
7 0 1 0 []
\N 1 1 0 []
[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 9 [1,1,1,1,1,1,1,1,1]
10 0 1 0 []
\N 1 1 0 []
['str_12','str_12'] 1 0 2 [1,1]
13 0 1 0 []
\N 1 1 0 []
[15,15,15,15,15] 1 0 5 [0,0,0,0,0]
16 0 1 0 []
\N 1 1 0 []
[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 8 [1,1,1,1,1,1,1,1]
19 0 1 0 []
\N 1 1 0 []
['str_21'] 1 0 1 [1]
22 0 1 0 []
\N 1 1 0 []
[24,24,24,24] 1 0 4 [0,0,0,0]
25 0 1 0 []
\N 1 1 0 []
[NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 7 [1,1,1,1,1,1,1]
28 0 1 0 []
\N 1 1 0 []
[] 1 0 0 []
31 0 1 0 []
\N 1 1 0 []
[33,33,33] 1 0 3 [0,0,0]
34 0 1 0 []
\N 1 1 0 []
1 0 0 []
0 1 0 []
1 1 0 []
1 0 3 [1,1,1]
0 1 0 []
1 1 0 []
1 0 6 [0,0,0,0,0,0]
0 1 0 []
1 1 0 []
1 0 9 [1,1,1,1,1,1,1,1,1]
0 1 0 []
1 1 0 []
1 0 2 [1,1]
0 1 0 []
1 1 0 []
1 0 5 [0,0,0,0,0]
0 1 0 []
1 1 0 []
1 0 8 [1,1,1,1,1,1,1,1]
0 1 0 []
1 1 0 []
1 0 1 [1]
0 1 0 []
1 1 0 []
1 0 4 [0,0,0,0]
0 1 0 []
1 1 0 []
1 0 7 [1,1,1,1,1,1,1]
0 1 0 []
1 1 0 []
1 0 0 []
0 1 0 []
1 1 0 []
1 0 3 [0,0,0]
0 1 0 []
1 1 0 []
0 0 [] []
1 0 [] []
1 0 [] []
0 3 [1,1,1] [0,0,0]
1 0 [] []
1 0 [] []
0 6 [0,0,0,0,0,0] [1,1,1,1,1,1]
1 0 [] []
1 0 [] []
0 9 [1,1,1,1,1,1,1,1,1] [1,1,1,1,1,1,1,1,1]
1 0 [] []
1 0 [] []
0 2 [1,1] [0,0]
1 0 [] []
1 0 [] []
0 5 [0,0,0,0,0] [1,1,1,1,1]
1 0 [] []
1 0 [] []
0 8 [1,1,1,1,1,1,1,1] [1,1,1,1,1,1,1,1]
1 0 [] []
1 0 [] []
0 1 [1] [0]
1 0 [] []
1 0 [] []
0 4 [0,0,0,0] [1,1,1,1]
1 0 [] []
1 0 [] []
0 7 [1,1,1,1,1,1,1] [1,1,1,1,1,1,1]
1 0 [] []
1 0 [] []
0 0 [] []
1 0 [] []
1 0 [] []
0 3 [0,0,0] [1,1,1]
1 0 [] []
1 0 [] []
0
2
3
5
6
8
9
11
12
14
15
17
18
20
21
23
24
26
27
29
30
32
33
35
MergeTree compact
test
[] 1 0 0 []
1 0 1 0 []
\N 1 1 0 []
['str_3','str_3','str_3'] 1 0 3 [1,1,1]
4 0 1 0 []
\N 1 1 0 []
[6,6,6,6,6,6] 1 0 6 [0,0,0,0,0,0]
7 0 1 0 []
\N 1 1 0 []
[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 9 [1,1,1,1,1,1,1,1,1]
10 0 1 0 []
\N 1 1 0 []
['str_12','str_12'] 1 0 2 [1,1]
13 0 1 0 []
\N 1 1 0 []
[15,15,15,15,15] 1 0 5 [0,0,0,0,0]
16 0 1 0 []
\N 1 1 0 []
[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 8 [1,1,1,1,1,1,1,1]
19 0 1 0 []
\N 1 1 0 []
['str_21'] 1 0 1 [1]
22 0 1 0 []
\N 1 1 0 []
[24,24,24,24] 1 0 4 [0,0,0,0]
25 0 1 0 []
\N 1 1 0 []
[NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 7 [1,1,1,1,1,1,1]
28 0 1 0 []
\N 1 1 0 []
[] 1 0 0 []
31 0 1 0 []
\N 1 1 0 []
[33,33,33] 1 0 3 [0,0,0]
34 0 1 0 []
\N 1 1 0 []
1 0 0 []
0 1 0 []
1 1 0 []
1 0 3 [1,1,1]
0 1 0 []
1 1 0 []
1 0 6 [0,0,0,0,0,0]
0 1 0 []
1 1 0 []
1 0 9 [1,1,1,1,1,1,1,1,1]
0 1 0 []
1 1 0 []
1 0 2 [1,1]
0 1 0 []
1 1 0 []
1 0 5 [0,0,0,0,0]
0 1 0 []
1 1 0 []
1 0 8 [1,1,1,1,1,1,1,1]
0 1 0 []
1 1 0 []
1 0 1 [1]
0 1 0 []
1 1 0 []
1 0 4 [0,0,0,0]
0 1 0 []
1 1 0 []
1 0 7 [1,1,1,1,1,1,1]
0 1 0 []
1 1 0 []
1 0 0 []
0 1 0 []
1 1 0 []
1 0 3 [0,0,0]
0 1 0 []
1 1 0 []
0 0 [] []
1 0 [] []
1 0 [] []
0 3 [1,1,1] [0,0,0]
1 0 [] []
1 0 [] []
0 6 [0,0,0,0,0,0] [1,1,1,1,1,1]
1 0 [] []
1 0 [] []
0 9 [1,1,1,1,1,1,1,1,1] [1,1,1,1,1,1,1,1,1]
1 0 [] []
1 0 [] []
0 2 [1,1] [0,0]
1 0 [] []
1 0 [] []
0 5 [0,0,0,0,0] [1,1,1,1,1]
1 0 [] []
1 0 [] []
0 8 [1,1,1,1,1,1,1,1] [1,1,1,1,1,1,1,1]
1 0 [] []
1 0 [] []
0 1 [1] [0]
1 0 [] []
1 0 [] []
0 4 [0,0,0,0] [1,1,1,1]
1 0 [] []
1 0 [] []
0 7 [1,1,1,1,1,1,1] [1,1,1,1,1,1,1]
1 0 [] []
1 0 [] []
0 0 [] []
1 0 [] []
1 0 [] []
0 3 [0,0,0] [1,1,1]
1 0 [] []
1 0 [] []
0
2
3
5
6
8
9
11
12
14
15
17
18
20
21
23
24
26
27
29
30
32
33
35
MergeTree wide
test
[] 1 0 0 []
1 0 1 0 []
\N 1 1 0 []
['str_3','str_3','str_3'] 1 0 3 [1,1,1]
4 0 1 0 []
\N 1 1 0 []
[6,6,6,6,6,6] 1 0 6 [0,0,0,0,0,0]
7 0 1 0 []
\N 1 1 0 []
[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 9 [1,1,1,1,1,1,1,1,1]
10 0 1 0 []
\N 1 1 0 []
['str_12','str_12'] 1 0 2 [1,1]
13 0 1 0 []
\N 1 1 0 []
[15,15,15,15,15] 1 0 5 [0,0,0,0,0]
16 0 1 0 []
\N 1 1 0 []
[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 8 [1,1,1,1,1,1,1,1]
19 0 1 0 []
\N 1 1 0 []
['str_21'] 1 0 1 [1]
22 0 1 0 []
\N 1 1 0 []
[24,24,24,24] 1 0 4 [0,0,0,0]
25 0 1 0 []
\N 1 1 0 []
[NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 7 [1,1,1,1,1,1,1]
28 0 1 0 []
\N 1 1 0 []
[] 1 0 0 []
31 0 1 0 []
\N 1 1 0 []
[33,33,33] 1 0 3 [0,0,0]
34 0 1 0 []
\N 1 1 0 []
1 0 0 []
0 1 0 []
1 1 0 []
1 0 3 [1,1,1]
0 1 0 []
1 1 0 []
1 0 6 [0,0,0,0,0,0]
0 1 0 []
1 1 0 []
1 0 9 [1,1,1,1,1,1,1,1,1]
0 1 0 []
1 1 0 []
1 0 2 [1,1]
0 1 0 []
1 1 0 []
1 0 5 [0,0,0,0,0]
0 1 0 []
1 1 0 []
1 0 8 [1,1,1,1,1,1,1,1]
0 1 0 []
1 1 0 []
1 0 1 [1]
0 1 0 []
1 1 0 []
1 0 4 [0,0,0,0]
0 1 0 []
1 1 0 []
1 0 7 [1,1,1,1,1,1,1]
0 1 0 []
1 1 0 []
1 0 0 []
0 1 0 []
1 1 0 []
1 0 3 [0,0,0]
0 1 0 []
1 1 0 []
0 0 [] []
1 0 [] []
1 0 [] []
0 3 [1,1,1] [0,0,0]
1 0 [] []
1 0 [] []
0 6 [0,0,0,0,0,0] [1,1,1,1,1,1]
1 0 [] []
1 0 [] []
0 9 [1,1,1,1,1,1,1,1,1] [1,1,1,1,1,1,1,1,1]
1 0 [] []
1 0 [] []
0 2 [1,1] [0,0]
1 0 [] []
1 0 [] []
0 5 [0,0,0,0,0] [1,1,1,1,1]
1 0 [] []
1 0 [] []
0 8 [1,1,1,1,1,1,1,1] [1,1,1,1,1,1,1,1]
1 0 [] []
1 0 [] []
0 1 [1] [0]
1 0 [] []
1 0 [] []
0 4 [0,0,0,0] [1,1,1,1]
1 0 [] []
1 0 [] []
0 7 [1,1,1,1,1,1,1] [1,1,1,1,1,1,1]
1 0 [] []
1 0 [] []
0 0 [] []
1 0 [] []
1 0 [] []
0 3 [0,0,0] [1,1,1]
1 0 [] []
1 0 [] []
0
2
3
5
6
8
9
11
12
14
15
17
18
20
21
23
24
26
27
29
30
32
33
35

View File

@ -0,0 +1,44 @@
#!/usr/bin/env bash
# Tags: long
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# reset --log_comment
CLICKHOUSE_LOG_COMMENT=
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_suspicious_variant_types=1"
function test()
{
echo "test"
$CH_CLIENT -q "insert into test select number, multiIf(number % 3 == 2, NULL, number % 3 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10))) from numbers(36)"
$CH_CLIENT -q "select v, v.UInt64.null, v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null from test order by id"
$CH_CLIENT -q "select v.UInt64.null, v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null from test order by id"
$CH_CLIENT -q "select v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null, v.\`Array(Variant(String, UInt64))\`.String.null from test order by id"
$CH_CLIENT -q "select id from test where v.UInt64 is null order by id"
$CH_CLIENT -q "insert into test select number, multiIf(number % 3 == 2, NULL, number % 3 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10))) from numbers(1000000) settings min_insert_block_size_rows=100000"
$CH_CLIENT -q "select v, v.UInt64.null, v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null from test order by id format Null"
$CH_CLIENT -q "select v.UInt64.null, v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null from test order by id format Null"
$CH_CLIENT -q "select v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null, v.\`Array(Variant(String, UInt64))\`.String.null from test order by id format Null"
$CH_CLIENT -q "select id from test where v.UInt64 is null order by id format Null"
}
$CH_CLIENT -q "drop table if exists test;"
echo "Memory"
$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=Memory"
test
$CH_CLIENT -q "drop table test;"
echo "MergeTree compact"
$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;"
test
$CH_CLIENT -q "drop table test;"
echo "MergeTree wide"
$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
test
$CH_CLIENT -q "drop table test;"

View File

@ -0,0 +1,57 @@
Memory
test
Array(Array(Dynamic))
Array(Variant(String, UInt64))
None
String
UInt64
20
20
20
20
0
0
20
20
10
10
20
0
MergeTree compact
test
Array(Array(Dynamic))
Array(Variant(String, UInt64))
None
String
UInt64
20
20
20
20
0
0
20
20
10
10
20
0
MergeTree wide
test
Array(Array(Dynamic))
Array(Variant(String, UInt64))
None
String
UInt64
20
20
20
20
0
0
20
20
10
10
20
0

View File

@ -0,0 +1,62 @@
#!/usr/bin/env bash
# Tags: long
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# reset --log_comment
CLICKHOUSE_LOG_COMMENT=
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1"
function test()
{
echo "test"
$CH_CLIENT -q "insert into test select number, number from numbers(10) settings min_insert_block_size_rows=50000"
$CH_CLIENT -q "insert into test select number, 'str_' || toString(number) from numbers(10, 10) settings min_insert_block_size_rows=50000"
$CH_CLIENT -q "insert into test select number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1)) from numbers(20, 10) settings min_insert_block_size_rows=50000"
$CH_CLIENT -q "insert into test select number, NULL from numbers(30, 10) settings min_insert_block_size_rows=50000"
$CH_CLIENT -q "insert into test select number, multiIf(number % 4 == 3, 'str_' || toString(number), number % 4 == 2, NULL, number % 4 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1))) from numbers(40, 40) settings min_insert_block_size_rows=50000"
$CH_CLIENT -q "insert into test select number, [range((number % 10 + 1)::UInt64)]::Array(Array(Dynamic)) from numbers(10, 10) settings min_insert_block_size_rows=50000"
$CH_CLIENT -q "select distinct dynamicType(d) as type from test order by type"
$CH_CLIENT -q "select count() from test where dynamicType(d) == 'UInt64'"
$CH_CLIENT -q "select count() from test where d.UInt64 is not NULL"
$CH_CLIENT -q "select count() from test where dynamicType(d) == 'String'"
$CH_CLIENT -q "select count() from test where d.String is not NULL"
$CH_CLIENT -q "select count() from test where dynamicType(d) == 'Date'"
$CH_CLIENT -q "select count() from test where d.Date is not NULL"
$CH_CLIENT -q "select count() from test where dynamicType(d) == 'Array(Variant(String, UInt64))'"
$CH_CLIENT -q "select count() from test where not empty(d.\`Array(Variant(String, UInt64))\`)"
$CH_CLIENT -q "select count() from test where dynamicType(d) == 'Array(Array(Dynamic))'"
$CH_CLIENT -q "select count() from test where not empty(d.\`Array(Array(Dynamic))\`)"
$CH_CLIENT -q "select count() from test where d is NULL"
$CH_CLIENT -q "select count() from test where not empty(d.\`Tuple(a Array(Dynamic))\`.a.String)"
$CH_CLIENT -q "select d, d.UInt64.null, d.String.null, d.\`Array(Variant(String, UInt64))\`.null from test format Null"
$CH_CLIENT -q "select d.UInt64.null, d.String.null, d.\`Array(Variant(String, UInt64))\`.null from test format Null"
$CH_CLIENT -q "select d.Int8.null, d.Date.null, d.\`Array(String)\`.null from test format Null"
$CH_CLIENT -q "select d, d.UInt64.null, d.Date.null, d.\`Array(Variant(String, UInt64))\`.null, d.\`Array(Variant(String, UInt64))\`.size0, d.\`Array(Variant(String, UInt64))\`.UInt64.null from test format Null"
$CH_CLIENT -q "select d.UInt64.null, d.Date.null, d.\`Array(Variant(String, UInt64))\`.null, d.\`Array(Variant(String, UInt64))\`.size0, d.\`Array(Variant(String, UInt64))\`.UInt64.null, d.\`Array(Variant(String, UInt64))\`.String.null from test format Null"
$CH_CLIENT -q "select d, d.\`Tuple(a UInt64, b String)\`.a, d.\`Array(Dynamic)\`.\`Variant(String, UInt64)\`.UInt64.null, d.\`Array(Variant(String, UInt64))\`.UInt64.null from test format Null"
$CH_CLIENT -q "select d.\`Array(Dynamic)\`.\`Variant(String, UInt64)\`.UInt64.null, d.\`Array(Dynamic)\`.size0, d.\`Array(Variant(String, UInt64))\`.UInt64.null from test format Null"
$CH_CLIENT -q "select d.\`Array(Array(Dynamic))\`.size1, d.\`Array(Array(Dynamic))\`.UInt64.null, d.\`Array(Array(Dynamic))\`.\`Map(String, Tuple(a UInt64))\`.values.a from test format Null"
}
$CH_CLIENT -q "drop table if exists test;"
echo "Memory"
$CH_CLIENT -q "create table test (id UInt64, d Dynamic) engine=Memory"
test
$CH_CLIENT -q "drop table test;"
echo "MergeTree compact"
$CH_CLIENT -q "create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;"
test
$CH_CLIENT -q "drop table test;"
echo "MergeTree wide"
$CH_CLIENT -q "create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
test
$CH_CLIENT -q "drop table test;"