Skip to content

Commit

Permalink
Avoid copying ColumnWithTypeAndName [#METR-22173].
Browse files Browse the repository at this point in the history
  • Loading branch information
alexey-milovidov committed Aug 4, 2016
1 parent 093353d commit fb59818
Show file tree
Hide file tree
Showing 23 changed files with 39 additions and 66 deletions.
2 changes: 1 addition & 1 deletion dbms/include/DB/Common/ExternalTable.h
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ class BaseExternalTable
column.name = structure[i].first;
column.type = data_type_factory.get(structure[i].second);
column.column = column.type->createColumn();
sample_block.insert(column);
sample_block.insert(std::move(column));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,8 +39,7 @@ class AddingConstColumnBlockInputStream : public IProfilingBlockInputStream
if (!res)
return res;
ColumnPtr column_ptr = ColumnConst<ColumnType>(res.rows(), value, data_type).convertToFullColumn();
ColumnWithTypeAndName column(column_ptr, data_type, column_name);
res.insert(column);
res.insert({column_ptr, data_type, column_name});
return res;
}

Expand Down
24 changes: 5 additions & 19 deletions dbms/include/DB/Interpreters/InterpreterExistsQuery.h
Original file line number Diff line number Diff line change
Expand Up @@ -37,32 +37,18 @@ class InterpreterExistsQuery : public IInterpreter

Block getSampleBlock()
{
ColumnWithTypeAndName col;
col.name = "result";
col.type = std::make_shared<DataTypeUInt8>();
col.column = col.type->createColumn();

Block block;
block.insert(col);

return block;
return {{ std::make_shared<ColumnConstUInt8>(1, 0), std::make_shared<DataTypeUInt8>(), "result" }};
}

BlockInputStreamPtr executeImpl()
{
const ASTExistsQuery & ast = typeid_cast<const ASTExistsQuery &>(*query_ptr);

bool res = context.isTableExist(ast.database, ast.table);

ColumnWithTypeAndName col;
col.name = "result";
col.type = std::make_shared<DataTypeUInt8>();
col.column = std::make_shared<ColumnConstUInt8>(1, res);

Block block;
block.insert(col);

return std::make_shared<OneBlockInputStream>(block);
return std::make_shared<OneBlockInputStream>(Block{{
std::make_shared<ColumnConstUInt8>(1, res),
std::make_shared<DataTypeUInt8>(),
"result" }});
}
};

Expand Down
24 changes: 6 additions & 18 deletions dbms/include/DB/Interpreters/InterpreterShowCreateQuery.h
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@
#include <DB/DataStreams/copyData.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/Columns/ColumnString.h>


namespace DB
Expand Down Expand Up @@ -39,15 +40,7 @@ class InterpreterShowCreateQuery : public IInterpreter

Block getSampleBlock()
{
ColumnWithTypeAndName col;
col.name = "statement";
col.type = std::make_shared<DataTypeString>();
col.column = col.type->createColumn();

Block block;
block.insert(col);

return block;
return {{ std::make_shared<ColumnConstString>(1, String()), std::make_shared<DataTypeString>(), "statement" }};
}

BlockInputStreamPtr executeImpl()
Expand All @@ -58,15 +51,10 @@ class InterpreterShowCreateQuery : public IInterpreter
formatAST(*context.getCreateQuery(ast.database, ast.table), stream, 0, false, true);
String res = stream.str();

ColumnWithTypeAndName col;
col.name = "statement";
col.type = std::make_shared<DataTypeString>();
col.column = std::make_shared<ColumnConstString>(1, res);

Block block;
block.insert(col);

return std::make_shared<OneBlockInputStream>(block);
return std::make_shared<OneBlockInputStream>(Block{{
std::make_shared<ColumnConstString>(1, res),
std::make_shared<DataTypeString>(),
"statement"}});
}
};

Expand Down
2 changes: 1 addition & 1 deletion dbms/include/DB/Storages/MergeTree/MergeTreeReader.h
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,7 @@ class MergeTreeReader
}

if (!append && column.column->size())
res.insert(column);
res.insert(std::move(column));
}
}
catch (Exception & e)
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/DataStreams/NativeBlockInputStream.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ Block NativeBlockInputStream::readImpl()
if (rows) /// If no rows, nothing to read.
readData(*column.type, *column.column, istr, rows);

res.insert(column);
res.insert(std::move(column));

if (use_index)
++index_column_it;
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/DataStreams/tests/aggregating_stream.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ int main(int argc, char ** argv)
{
ColumnWithTypeAndName col;
col.type = *it;
sample.insert(col);
sample.insert(std::move(col));
}

Aggregator::Params params(key_column_names, aggregate_descriptions, false);
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/DataStreams/tests/sorting_stream.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ try
{
ColumnWithTypeAndName col;
col.type = type;
sample.insert(col);
sample.insert(std::move(col));
}

SortDescription sort_columns;
Expand Down
4 changes: 2 additions & 2 deletions dbms/src/DataStreams/tests/tab_separated_streams.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,12 @@ try
{
ColumnWithTypeAndName col;
col.type = std::make_shared<DataTypeUInt64>();
sample.insert(col);
sample.insert(std::move(col));
}
{
ColumnWithTypeAndName col;
col.type = std::make_shared<DataTypeString>();
sample.insert(col);
sample.insert(std::move(col));
}

ReadBufferFromFile in_buf("test_in");
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/DataTypes/DataTypeTuple.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -238,7 +238,7 @@ ColumnPtr DataTypeTuple::createColumn() const
ColumnWithTypeAndName col;
col.column = elems[i]->createColumn();
col.type = elems[i]->clone();
tuple_block.insert(col);
tuple_block.insert(std::move(col));
}
return std::make_shared<ColumnTuple>(tuple_block);
}
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Interpreters/Aggregator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -157,7 +157,7 @@ void Aggregator::initialize(const Block & block)
col.type = std::make_shared<DataTypeAggregateFunction>(params.aggregates[i].function, argument_types, params.aggregates[i].parameters);
col.column = col.type->createColumn();

sample.insert(col);
sample.insert(std::move(col));
}
}
}
Expand Down
4 changes: 2 additions & 2 deletions dbms/src/Interpreters/ExpressionActions.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -191,7 +191,7 @@ void ExpressionAction::prepare(Block & sample_block)
ColumnWithTypeAndName column = sample_block.getByName(name);
if (alias != "")
column.name = alias;
new_block.insert(column);
new_block.insert(std::move(column));
}

sample_block.swap(new_block);
Expand Down Expand Up @@ -348,7 +348,7 @@ void ExpressionAction::execute(Block & block) const
ColumnWithTypeAndName column = block.getByName(name);
if (alias != "")
column.name = alias;
new_block.insert(column);
new_block.insert(std::move(column));
}

block.swap(new_block);
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Interpreters/InterpreterInsertQuery.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ Block InterpreterInsertQuery::getSampleBlock()
col.name = current_name;
col.type = table_sample.getByName(current_name).type;
col.column = col.type->createColumn();
res.insert(col);
res.insert(std::move(col));
}

return res;
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Interpreters/Join.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -446,7 +446,7 @@ bool Join::insertFromBlock(const Block & block)
size_t pos = stored_block->getPositionByName(name);
ColumnWithTypeAndName col = stored_block->getByPosition(pos);
stored_block->erase(pos);
stored_block->insert(key_num, col);
stored_block->insert(key_num, std::move(col));
++key_num;
}
}
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Interpreters/Set.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -291,7 +291,7 @@ void Set::createFromAST(DataTypes & types, ASTPtr node, const Context & context,
col.column = data_types[i]->createColumn();
col.name = "_" + toString(i);

block.insert(col);
block.insert(std::move(col));
}

Row tuple_values;
Expand Down
5 changes: 2 additions & 3 deletions dbms/src/Storages/StorageLog.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -248,7 +248,7 @@ Block LogBlockInputStream::readImpl()
}

if (column.column->size())
res.insert(column);
res.insert(std::move(column));
}

/// Отдельно обрабатываем виртуальный столбец
Expand All @@ -260,8 +260,7 @@ Block LogBlockInputStream::readImpl()
if (rows > 0)
{
ColumnPtr column_ptr = ColumnConst<String>(rows, current_table.first, std::make_shared<DataTypeString>()).convertToFullColumn();
ColumnWithTypeAndName column(column_ptr, std::make_shared<DataTypeString>(), storage._table_column_name);
res.insert(column);
res.insert({column_ptr, std::make_shared<DataTypeString>(), storage._table_column_name});
}
}

Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Storages/StorageTinyLog.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -213,7 +213,7 @@ Block TinyLogBlockInputStream::readImpl()
}

if (column.column->size())
res.insert(column);
res.insert(std::move(column));
}

if (!res || streams.begin()->second->compressed.eof())
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Storages/System/StorageSystemNumbers.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ class NumbersBlockInputStream : public IProfilingBlockInputStream
while (pos < end)
*pos++ = curr++;

res.insert(column_with_type_and_name);
res.insert(std::move(column_with_type_and_name));

next += step;
return res;
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Storages/System/StorageSystemOne.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ BlockInputStreams StorageSystemOne::read(
col.name = "dummy";
col.type = std::make_shared<DataTypeUInt8>();
col.column = ColumnConstUInt8(1, 0).convertToFullColumn();
block.insert(col);
block.insert(std::move(col));

return BlockInputStreams(1, std::make_shared<OneBlockInputStream>(block));
}
Expand Down
5 changes: 3 additions & 2 deletions dbms/src/Storages/System/StorageSystemTables.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -39,11 +39,12 @@ static ColumnWithTypeAndName getFilteredDatabases(ASTPtr query, const Context &
column.type = std::make_shared<DataTypeString>();
column.column = std::make_shared<ColumnString>();

Block block;
block.insert(column);
for (const auto & db : context.getDatabases())
column.column->insert(db.first);

Block block;
block.insert(std::move(column));

VirtualColumnUtils::filterBlockWithQuery(query, block, context);

return block.getByPosition(0);
Expand Down
4 changes: 2 additions & 2 deletions dbms/src/Storages/tests/merge_tree.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -103,12 +103,12 @@ try
{
ColumnWithTypeAndName col;
col.type = names_and_types->front().type;
sample.insert(col);
sample.insert(std::move(col));
}
{
ColumnWithTypeAndName col;
col.type = names_and_types->back().type;
sample.insert(col);
sample.insert(std::move(col));
}

WriteBufferFromFileDescriptor out_buf(STDOUT_FILENO);
Expand Down
4 changes: 2 additions & 2 deletions dbms/src/Storages/tests/storage_log.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -72,12 +72,12 @@ try
{
ColumnWithTypeAndName col;
col.type = std::make_shared<DataTypeUInt64>();
sample.insert(col);
sample.insert(std::move(col));
}
{
ColumnWithTypeAndName col;
col.type = std::make_shared<DataTypeUInt8>();
sample.insert(col);
sample.insert(std::move(col));
}

WriteBufferFromOStream out_buf(std::cout);
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Storages/tests/system_numbers.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ try
Block sample;
ColumnWithTypeAndName col;
col.type = std::make_shared<DataTypeUInt64>();
sample.insert(col);
sample.insert(std::move(col));

WriteBufferFromOStream out_buf(std::cout);

Expand Down

0 comments on commit fb59818

Please sign in to comment.