major rework, transform added to the insert pipe

This commit is contained in:
Yakov Olkhovskiy 2022-02-13 17:42:59 +00:00
parent eaf660fd61
commit 579fe6c97a
6 changed files with 94 additions and 15 deletions

View File

@ -24,6 +24,7 @@
#include <Processors/Transforms/MaterializingTransform.h>
#include <Processors/Transforms/SquashingChunksTransform.h>
#include <Processors/Transforms/getSourceFromASTInsertQuery.h>
#include <Processors/Transforms/ReduceToPhysicalTransform.h>
#include <Storages/StorageDistributed.h>
#include <Storages/StorageMaterializedView.h>
#include <TableFunctions/TableFunctionFactory.h>
@ -109,22 +110,31 @@ Block InterpreterInsertQuery::getSampleBlock(
const StoragePtr & table,
const StorageMetadataPtr & metadata_snapshot) const
{
Block table_sample = metadata_snapshot->getSampleBlock();
Block table_sample_non_materialized = metadata_snapshot->getSampleBlockInsertable();
Block table_sample_physical = metadata_snapshot->getSampleBlock();
Block table_sample_insertable = metadata_snapshot->getSampleBlockInsertable();
Block res;
for (const auto & current_name : names)
{
/// The table does not have a column with that name
if (!table_sample.has(current_name))
throw Exception("No such column " + current_name + " in table " + table->getStorageID().getNameForLogs(),
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
if (!allow_materialized && !table_sample_non_materialized.has(current_name))
throw Exception("Cannot insert column " + current_name + ", because it is MATERIALIZED column.", ErrorCodes::ILLEGAL_COLUMN);
if (res.has(current_name))
throw Exception("Column " + current_name + " specified more than once", ErrorCodes::DUPLICATE_COLUMN);
res.insert(ColumnWithTypeAndName(table_sample.getByName(current_name).type, current_name));
/// Column is not ordinary or ephemeral
if (!table_sample_insertable.has(current_name))
{
/// Column is materialized
if (table_sample_physical.has(current_name))
{
if (!allow_materialized)
throw Exception("Cannot insert column " + current_name + ", because it is MATERIALIZED column.",
ErrorCodes::ILLEGAL_COLUMN);
res.insert(ColumnWithTypeAndName(table_sample_physical.getByName(current_name).type, current_name));
}
else /// The table does not have a column with that name
throw Exception("No such column " + current_name + " in table " + table->getStorageID().getNameForLogs(),
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
}
else
res.insert(ColumnWithTypeAndName(table_sample_insertable.getByName(current_name).type, current_name));
}
return res;
}
@ -229,6 +239,12 @@ Chain InterpreterInsertQuery::buildChainImpl(
auto adding_missing_defaults_actions = std::make_shared<ExpressionActions>(adding_missing_defaults_dag);
/// Non-physical columns may be necessary up to this point. Sink requires only physical columns, so
/// add this transform to remove non-physical columns.
out.addSource(std::make_shared<ReduceToPhysicalTransform>(
ExpressionTransform::transformHeader(query_sample_block, *adding_missing_defaults_dag),
metadata_snapshot->getColumns()));
/// Actually we don't know structure of input blocks from query/table,
/// because some clients break insertion protocol (columns != header)
out.addSource(std::make_shared<ConvertingTransform>(query_sample_block, adding_missing_defaults_actions));

View File

@ -0,0 +1,35 @@
#include <Processors/Transforms/ReduceToPhysicalTransform.h>
#include <Common/PODArray.h>
namespace DB
{
Block ReduceToPhysicalTransform::transformToPhysicalBlock(const Block & header, const ColumnsDescription & columns)
{
Block out = header.cloneWithoutColumns();
for(const auto & column : columns.getEphemeral())
if(out.has(column.name))
out.erase(column.name);
return out;
}
ReduceToPhysicalTransform::ReduceToPhysicalTransform(const Block & input_header, const ColumnsDescription & columns) :
ISimpleTransform(input_header, transformToPhysicalBlock(input_header, columns), false)
{
/// Find non-physical columns in input_header
for(const auto & column : columns.getEphemeral())
if(input_header.has(column.name))
index.push_back(input_header.getPositionByName(column.name));
std::sort(index.begin(), index.end(), std::greater());
}
void ReduceToPhysicalTransform::transform(Chunk & chunk)
{
for(size_t i : index)
chunk.erase(i);
}
}

View File

@ -0,0 +1,22 @@
#pragma once
#include <Processors/ISimpleTransform.h>
#include <Storages/ColumnsDescription.h>
namespace DB
{
/// Remove non-physical columns.
class ReduceToPhysicalTransform : public ISimpleTransform
{
public:
ReduceToPhysicalTransform(const Block & input_header, const ColumnsDescription & columns);
String getName() const override { return "ReduceToPhysicalTransform"; }
static Block transformToPhysicalBlock(const Block & header, const ColumnsDescription & columns);
protected:
void transform(Chunk & chunk) override;
private:
std::vector<size_t> index;
};
}

View File

@ -461,7 +461,7 @@ NamesAndTypesList ColumnsDescription::getAllPhysical() const
{
NamesAndTypesList ret;
for (const auto & col : columns)
if (col.default_desc.kind != ColumnDefaultKind::Alias)
if (col.default_desc.kind != ColumnDefaultKind::Alias && col.default_desc.kind != ColumnDefaultKind::Ephemeral)
ret.emplace_back(col.name, col.type);
return ret;
}
@ -470,7 +470,7 @@ Names ColumnsDescription::getNamesOfPhysical() const
{
Names ret;
for (const auto & col : columns)
if (col.default_desc.kind != ColumnDefaultKind::Alias)
if (col.default_desc.kind != ColumnDefaultKind::Alias && col.default_desc.kind != ColumnDefaultKind::Ephemeral)
ret.emplace_back(col.name);
return ret;
}
@ -501,7 +501,8 @@ NameAndTypePair ColumnsDescription::getColumnOrSubcolumn(GetFlags flags, const S
std::optional<NameAndTypePair> ColumnsDescription::tryGetPhysical(const String & column_name) const
{
auto it = columns.get<1>().find(column_name);
if (it == columns.get<1>().end() || it->default_desc.kind == ColumnDefaultKind::Alias)
if (it == columns.get<1>().end() ||
it->default_desc.kind == ColumnDefaultKind::Alias || it->default_desc.kind == ColumnDefaultKind::Ephemeral)
return {};
return NameAndTypePair(it->name, it->type);
@ -520,7 +521,8 @@ NameAndTypePair ColumnsDescription::getPhysical(const String & column_name) cons
bool ColumnsDescription::hasPhysical(const String & column_name) const
{
auto it = columns.get<1>().find(column_name);
return it != columns.get<1>().end() && it->default_desc.kind != ColumnDefaultKind::Alias;
return it != columns.get<1>().end() &&
it->default_desc.kind != ColumnDefaultKind::Alias && it->default_desc.kind != ColumnDefaultKind::Ephemeral;
}
bool ColumnsDescription::hasColumnOrSubcolumn(GetFlags flags, const String & column_name) const

View File

@ -94,7 +94,7 @@ public:
NamesAndTypesList getOrdinary() const;
NamesAndTypesList getMaterialized() const;
NamesAndTypesList getInsertable() const;
NamesAndTypesList getInsertable() const; /// ordinary + ephemeral
NamesAndTypesList getAliases() const;
NamesAndTypesList getEphemeral() const;
NamesAndTypesList getAllPhysical() const; /// ordinary + materialized.

View File

@ -25,6 +25,10 @@ TRUNCATE TABLE t_ephemeral_02205_1;
INSERT INTO t_ephemeral_02205_1 (x, y) VALUES (DEFAULT, 7);
SELECT * FROM t_ephemeral_02205_1;
# Test ALTER TABLE DELETE
ALTER TABLE t_ephemeral_02205_1 DELETE WHERE x = 7;
SELECT * FROM t_ephemeral_02205_1;
TRUNCATE TABLE t_ephemeral_02205_1;
# Test INSERT into column, defaulted to ephemeral, but explicitly provided with value