提交 0ef74983 编写于 作者: A artpaul

issue #390: insert values by their positions instead of name

上级 633ce17c
#pragma once
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <experimental/optional>
namespace DB
{
......@@ -17,7 +18,7 @@ class NullableAdapterBlockInputStream : public IProfilingBlockInputStream
{
public:
NullableAdapterBlockInputStream(BlockInputStreamPtr input_, const Block & in_sample_,
const Block & out_sample_, const NamesAndTypesListPtr & required_columns_);
const Block & out_sample_);
String getName() const override { return "NullableAdapterBlockInputStream"; }
......@@ -43,19 +44,17 @@ private:
using Actions = std::vector<Action>;
private:
/// Return true if we must transform the blocks we read.
bool mustTransform() const;
/// Determine the actions to be taken using the source sample block,
/// which describes the columns from which we fetch data inside an INSERT
/// query, and the target sample block which contains the columns
/// we insert data into.
Actions getActions(const Block & in_sample, const Block & out_sample) const;
void buildActions(const Block & in_sample, const Block & out_sample);
private:
NamesAndTypesListPtr required_columns;
const Actions actions;
const bool must_transform;
Actions actions;
std::vector<std::experimental::optional<String>> rename;
bool must_transform;
};
}
......@@ -9,18 +9,22 @@ namespace ErrorCodes
{
extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN;
extern const int NO_SUCH_COLUMN_IN_TABLE;
extern const int TYPE_MISMATCH;
}
static DataTypePtr removeNullable(DataTypePtr type)
{
while (type->isNullable())
type = typeid_cast<DataTypeNullable *>(type.get())->getNestedType();
return type;
}
NullableAdapterBlockInputStream::NullableAdapterBlockInputStream(
BlockInputStreamPtr input_,
const Block & in_sample_, const Block & out_sample_,
const NamesAndTypesListPtr & required_columns_)
: required_columns{required_columns_},
actions{getActions(in_sample_, out_sample_)},
must_transform{mustTransform()}
const Block & in_sample_, const Block & out_sample_)
{
buildActions(in_sample_, out_sample_);
children.push_back(input_);
}
......@@ -35,7 +39,7 @@ Block NullableAdapterBlockInputStream::readImpl()
{
Block block = children.back()->read();
if (!block || !must_transform)
if (!block && !must_transform)
return block;
Block res;
......@@ -44,7 +48,6 @@ Block NullableAdapterBlockInputStream::readImpl()
for (size_t i = 0; i < s; ++i)
{
const auto & elem = block.getByPosition(i);
ColumnWithTypeAndName new_elem;
switch (actions[i])
{
......@@ -63,7 +66,7 @@ Block NullableAdapterBlockInputStream::readImpl()
res.insert({
nullable_col.getNestedColumn(),
nullable_type.getNestedType(),
elem.name
rename[i].value_or(elem.name)
});
break;
}
......@@ -74,85 +77,65 @@ Block NullableAdapterBlockInputStream::readImpl()
res.insert({
std::make_shared<ColumnNullable>(elem.column, null_map),
std::make_shared<DataTypeNullable>(elem.type),
elem.name
rename[i].value_or(elem.name)
});
break;
}
case NONE:
res.insert(elem);
{
if (rename[i])
res.insert({elem.column, elem.type, rename[i].value()});
else
res.insert(elem);
break;
}
}
}
return res;
}
bool NullableAdapterBlockInputStream::mustTransform() const
{
return !std::all_of(actions.begin(), actions.end(), [](Action action) { return action == NONE; });
}
NullableAdapterBlockInputStream::Actions NullableAdapterBlockInputStream::getActions(
const Block & in_sample, const Block & out_sample) const
void NullableAdapterBlockInputStream::buildActions(
const Block & in_sample,
const Block & out_sample)
{
size_t in_size = in_sample.columns();
size_t out_size = out_sample.columns();
Actions actions;
actions.reserve(in_size);
rename.reserve(in_size);
size_t j = 0;
for (size_t i = 0; i < in_size; ++i)
{
const auto & in_elem = in_sample.getByPosition(i);
while (j < out_size)
for (size_t i = 0; i < in_size; ++i) {
const auto & in_elem = in_sample.getByPosition(i);
const auto & out_elem = out_sample.getByPosition(i);
if (removeNullable(in_elem.type)->getName() == removeNullable(out_elem.type)->getName())
{
const auto & out_elem = out_sample.getByPosition(j);
if (in_elem.name == out_elem.name)
{
bool is_in_nullable = in_elem.type->isNullable();
bool is_out_nullable = out_elem.type->isNullable();
bool is_in_nullable = in_elem.type->isNullable();
bool is_out_nullable = out_elem.type->isNullable();
if (is_in_nullable && !is_out_nullable)
actions.push_back(TO_ORDINARY);
else if (!is_in_nullable && is_out_nullable)
actions.push_back(TO_NULLABLE);
else
actions.push_back(NONE);
if (is_in_nullable && !is_out_nullable)
actions.push_back(TO_ORDINARY);
else if (!is_in_nullable && is_out_nullable)
actions.push_back(TO_NULLABLE);
else
actions.push_back(NONE);
++j;
break;
}
if (in_elem.name != out_elem.name)
rename.push_back(std::experimental::make_optional(out_elem.name));
else
{
++j;
if (j == out_size)
{
auto print_columns = [](const NamesAndTypesList & columns)
{
bool is_first = true;
std::ostringstream ostr;
for (const auto & it : columns)
{
if (is_first)
is_first = false;
else
ostr << ", ";
ostr << it.name;
}
return ostr.str();
};
throw Exception{"There is no column with name " + in_elem.name
+ ". There are columns: "
+ print_columns(*required_columns),
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE};
}
}
rename.push_back(std::experimental::nullopt);
if (actions.back() != NONE || rename.back())
must_transform = true;
}
else
{
throw Exception{String("Types must be the same for columns at same position. ")
+ "Column " + in_elem.name + " has type " + in_elem.type->getName()
+ ", but column " + out_elem.name + " has type " + out_elem.type->getName(),
ErrorCodes::TYPE_MISMATCH};
}
}
return actions;
}
}
......@@ -120,8 +120,8 @@ BlockIO InterpreterInsertQuery::execute()
InterpreterSelectQuery interpreter_select{query.select, context};
res.in_sample = interpreter_select.getSampleBlock();
BlockInputStreamPtr in = interpreter_select.execute().in;
res.in = std::make_shared<NullableAdapterBlockInputStream>(in, res.in_sample, res.out_sample, required_columns);
res.in = interpreter_select.execute().in;
res.in = std::make_shared<NullableAdapterBlockInputStream>(res.in, res.in_sample, res.out_sample);
res.in = std::make_shared<NullAndDoCopyBlockInputStream>(res.in, out);
}
......
......@@ -2,4 +2,13 @@ DROP TABLE IF EXISTS test.insert_fewer_columns;
CREATE TABLE test.insert_fewer_columns (a UInt8, b UInt8) ENGINE = Memory;
INSERT INTO test.insert_fewer_columns (a) VALUES (1), (2);
SELECT * FROM test.insert_fewer_columns;
-- Test position arguments in insert.
DROP TABLE IF EXISTS test.insert_fewer_columns_2;
CREATE TABLE test.insert_fewer_columns_2 (b UInt8, a UInt8) ENGINE = Memory;
INSERT INTO test.insert_fewer_columns_2 SELECT * FROM test.insert_fewer_columns;
SELECT a, b FROM test.insert_fewer_columns;
SELECT a, b FROM test.insert_fewer_columns_2;
DROP TABLE IF EXISTS test.insert_fewer_columns_2;
DROP TABLE test.insert_fewer_columns;
......@@ -6,7 +6,7 @@ CREATE TABLE test.merge (d Date, x UInt64) ENGINE = Merge(test, '^mt$');
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
SET max_block_size = 1000000;
INSERT INTO test.mt SELECT number AS x FROM system.numbers LIMIT 100000;
INSERT INTO test.mt (x) SELECT number AS x FROM system.numbers LIMIT 100000;
SELECT *, b FROM test.mt WHERE x IN (12345, 67890) AND NOT ignore(blockSize() < 10 AS b) ORDER BY x;
SELECT *, b FROM test.merge WHERE x IN (12345, 67890) AND NOT ignore(blockSize() < 10 AS b) ORDER BY x;
......
DROP TABLE IF EXISTS test.aggregating;
CREATE TABLE test.aggregating (d Date DEFAULT '2000-01-01', k UInt64, u AggregateFunction(uniq, UInt64)) ENGINE = AggregatingMergeTree(d, k, 8192);
INSERT INTO test.aggregating SELECT intDiv(number, 100) AS k, uniqState(toUInt64(number % 100)) AS u FROM (SELECT * FROM system.numbers LIMIT 1000) GROUP BY k;
INSERT INTO test.aggregating SELECT intDiv(number, 100) AS k, uniqState(toUInt64(number % 100) + 50) AS u FROM (SELECT * FROM system.numbers LIMIT 500, 1000) GROUP BY k;
INSERT INTO test.aggregating (k, u) SELECT intDiv(number, 100) AS k, uniqState(toUInt64(number % 100)) AS u FROM (SELECT * FROM system.numbers LIMIT 1000) GROUP BY k;
INSERT INTO test.aggregating (k, u) SELECT intDiv(number, 100) AS k, uniqState(toUInt64(number % 100) + 50) AS u FROM (SELECT * FROM system.numbers LIMIT 500, 1000) GROUP BY k;
SELECT k, finalizeAggregation(u) FROM test.aggregating FINAL;
......
......@@ -2,7 +2,7 @@ DROP TABLE IF EXISTS test.agg_func_col;
CREATE TABLE test.agg_func_col (p Date, k UInt8, d AggregateFunction(sum, UInt64) DEFAULT arrayReduce('sumState', [toUInt64(200)])) ENGINE = AggregatingMergeTree(p, k, 1);
INSERT INTO test.agg_func_col (k) VALUES (0);
INSERT INTO test.agg_func_col SELECT 1 AS k, arrayReduce('sumState', [toUInt64(100)]) AS d;
INSERT INTO test.agg_func_col (k, d) SELECT 1 AS k, arrayReduce('sumState', [toUInt64(100)]) AS d;
SELECT k, sumMerge(d) FROM test.agg_func_col GROUP BY k ORDER BY k;
SELECT '';
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册