mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Better [#METR-22173].
This commit is contained in:
parent
fb5981871a
commit
61b9996d9e
@ -749,7 +749,7 @@ public:
|
||||
replicated_column.name = name;
|
||||
replicated_column.column = typeid_cast<ColumnArray &>(*replicated_column.column).getDataPtr();
|
||||
replicated_column.type = typeid_cast<const DataTypeArray &>(*replicated_column.type).getNestedType(),
|
||||
temp_block.insert(replicated_column);
|
||||
temp_block.insert(std::move(replicated_column));
|
||||
|
||||
++prerequisite_index;
|
||||
}
|
||||
|
@ -203,7 +203,7 @@ protected:
|
||||
{
|
||||
const auto minimum_size_column_name = owned_data_part->getMinimumSizeColumnName();
|
||||
columns.push_back(minimum_size_column_name);
|
||||
injected_columns.insert(minimum_size_column_name);
|
||||
injected_columns.insert(std::move(minimum_size_column_name));
|
||||
}
|
||||
|
||||
return injected_columns;
|
||||
|
@ -501,7 +501,7 @@ private:
|
||||
res.rows(), column_to_add.type->getDefault())).convertToFullColumn();
|
||||
}
|
||||
|
||||
res.insert(column_to_add);
|
||||
res.insert(std::move(column_to_add));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -80,7 +80,7 @@ void Block::addDefaults(const NamesAndTypesList & required_columns)
|
||||
rowsInFirstColumn(), column_to_add.type->getDefault())).convertToFullColumn();
|
||||
}
|
||||
|
||||
insert(column_to_add);
|
||||
insert(std::move(column_to_add));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -93,7 +93,7 @@ try
|
||||
elem.name = name_type.name;
|
||||
elem.type = name_type.type;
|
||||
elem.column = elem.type->createColumn();
|
||||
sample.insert(elem);
|
||||
sample.insert(std::move(elem));
|
||||
}
|
||||
|
||||
/// читаем данные из строчного tsv файла и одновременно пишем в блочный tsv файл
|
||||
|
@ -46,7 +46,7 @@ try
|
||||
elem.name = name_type.name;
|
||||
elem.type = name_type.type;
|
||||
elem.column = elem.type->createColumn();
|
||||
sample.insert(elem);
|
||||
sample.insert(std::move(elem));
|
||||
}
|
||||
|
||||
{
|
||||
|
@ -140,7 +140,7 @@ void ExpressionAction::prepare(Block & sample_block)
|
||||
ColumnWithTypeAndName new_column;
|
||||
new_column.name = result_name;
|
||||
new_column.type = result_type;
|
||||
sample_block.insert(new_column);
|
||||
sample_block.insert(std::move(new_column));
|
||||
|
||||
function->execute(sample_block, arguments, prerequisites, result_position);
|
||||
|
||||
@ -151,7 +151,7 @@ void ExpressionAction::prepare(Block & sample_block)
|
||||
}
|
||||
else
|
||||
{
|
||||
sample_block.insert(ColumnWithTypeAndName(nullptr, result_type, result_name));
|
||||
sample_block.insert({nullptr, result_type, result_name});
|
||||
}
|
||||
|
||||
break;
|
||||
@ -258,13 +258,11 @@ void ExpressionAction::execute(Block & block) const
|
||||
prerequisites[i] = block.getPositionByName(prerequisite_names[i]);
|
||||
}
|
||||
|
||||
ColumnWithTypeAndName new_column;
|
||||
new_column.name = result_name;
|
||||
new_column.type = result_type;
|
||||
block.insert(new_column);
|
||||
size_t num_columns_without_result = block.columns();
|
||||
block.insert({ nullptr, result_type, result_name});
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::FunctionExecute);
|
||||
function->execute(block, arguments, prerequisites, block.getPositionByName(result_name));
|
||||
function->execute(block, arguments, prerequisites, num_columns_without_result);
|
||||
|
||||
break;
|
||||
}
|
||||
@ -361,11 +359,11 @@ void ExpressionAction::execute(Block & block) const
|
||||
break;
|
||||
|
||||
case ADD_COLUMN:
|
||||
block.insert(ColumnWithTypeAndName(added_column->cloneResized(block.rowsInFirstColumn()), result_type, result_name));
|
||||
block.insert({ added_column->cloneResized(block.rowsInFirstColumn()), result_type, result_name });
|
||||
break;
|
||||
|
||||
case COPY_COLUMN:
|
||||
block.insert(ColumnWithTypeAndName(block.getByName(source_name).column, result_type, result_name));
|
||||
block.insert({ block.getByName(source_name).column, result_type, result_name });
|
||||
break;
|
||||
|
||||
default:
|
||||
@ -612,7 +610,7 @@ void ExpressionActions::executeOnTotals(Block & block) const
|
||||
{
|
||||
ColumnWithTypeAndName elem(name_and_type.type->createColumn(), name_and_type.type, name_and_type.name);
|
||||
elem.column->insertDefault();
|
||||
block.insert(elem);
|
||||
block.insert(std::move(elem));
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -650,7 +648,7 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
NameSet final_columns;
|
||||
for (size_t i = 0; i < output_columns.size(); ++i)
|
||||
{
|
||||
const std::string name = output_columns[i];
|
||||
const std::string & name = output_columns[i];
|
||||
if (!sample_block.has(name))
|
||||
throw Exception("Unknown column: " + name + ", there are only columns "
|
||||
+ sample_block.dumpNames(), ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
@ -997,7 +995,7 @@ BlockInputStreamPtr ExpressionActions::createStreamWithNonJoinedDataIfFullOrRigh
|
||||
{
|
||||
Block left_sample_block;
|
||||
for (const auto & input_elem : input_columns)
|
||||
left_sample_block.insert(ColumnWithTypeAndName(nullptr, input_elem.type, input_elem.name));
|
||||
left_sample_block.insert({ nullptr, input_elem.type, input_elem.name });
|
||||
|
||||
return action.join->createStreamWithNonJoinedRows(left_sample_block, max_block_size);
|
||||
}
|
||||
|
@ -641,9 +641,9 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
|
||||
{
|
||||
const ColumnWithTypeAndName & src_column = sample_block_with_columns_to_add.getByPosition(i);
|
||||
ColumnWithTypeAndName new_column = src_column.cloneEmpty();
|
||||
block.insert(new_column);
|
||||
added_columns[i] = new_column.column.get();
|
||||
added_columns[i]->reserve(src_column.column->size());
|
||||
block.insert(std::move(new_column));
|
||||
}
|
||||
|
||||
size_t rows = block.rowsInFirstColumn();
|
||||
@ -776,8 +776,8 @@ void Join::joinBlockImplCross(Block & block) const
|
||||
{
|
||||
const ColumnWithTypeAndName & src_column = sample_block_with_columns_to_add.unsafeGetByPosition(i);
|
||||
ColumnWithTypeAndName new_column = src_column.cloneEmpty();
|
||||
res.insert(new_column);
|
||||
dst_right_columns[i] = new_column.column.get();
|
||||
res.insert(std::move(new_column));
|
||||
}
|
||||
|
||||
size_t rows_left = block.rowsInFirstColumn();
|
||||
@ -939,7 +939,7 @@ public:
|
||||
{
|
||||
const ColumnWithTypeAndName & src_column = parent.sample_block_with_columns_to_add.getByPosition(i);
|
||||
ColumnWithTypeAndName new_column = src_column.cloneEmpty();
|
||||
result_sample_block.insert(new_column);
|
||||
result_sample_block.insert(std::move(new_column));
|
||||
}
|
||||
|
||||
column_numbers_left.reserve(num_columns_left);
|
||||
|
@ -172,16 +172,16 @@ void ITableDeclaration::check(const Names & column_names) const
|
||||
UniqueStrings unique_names;
|
||||
unique_names.set_empty_key(StringRef());
|
||||
|
||||
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
|
||||
for (const auto & name : column_names)
|
||||
{
|
||||
if (columns_map.end() == columns_map.find(*it))
|
||||
throw Exception("There is no column with name " + *it + " in table. There are columns: " + listOfColumns(available_columns),
|
||||
if (columns_map.end() == columns_map.find(name))
|
||||
throw Exception("There is no column with name " + name + " in table. There are columns: " + listOfColumns(available_columns),
|
||||
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
|
||||
if (unique_names.end() != unique_names.find(*it))
|
||||
throw Exception("Column " + *it + " queried more than once",
|
||||
if (unique_names.end() != unique_names.find(name))
|
||||
throw Exception("Column " + name + " queried more than once",
|
||||
ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE);
|
||||
unique_names.insert(*it);
|
||||
unique_names.insert(name);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -47,7 +47,7 @@ public:
|
||||
LogBlockInputStream(
|
||||
size_t block_size_, const Names & column_names_, StorageLog & storage_,
|
||||
size_t mark_number_, size_t rows_limit_, size_t max_read_buffer_size_)
|
||||
: block_size(block_size_), column_names(column_names_), storage(storage_),
|
||||
: block_size(block_size_), column_names(column_names_), column_types(column_names.size()), storage(storage_),
|
||||
mark_number(mark_number_), rows_limit(rows_limit_), current_mark(mark_number_), max_read_buffer_size(max_read_buffer_size_) {}
|
||||
|
||||
String getName() const { return "Log"; }
|
||||
@ -69,6 +69,7 @@ protected:
|
||||
private:
|
||||
size_t block_size;
|
||||
Names column_names;
|
||||
DataTypes column_types;
|
||||
StorageLog & storage;
|
||||
size_t mark_number; /// С какой засечки читать данные
|
||||
size_t rows_limit; /// Максимальное количество строк, которых можно прочитать
|
||||
@ -177,14 +178,18 @@ Block LogBlockInputStream::readImpl()
|
||||
{
|
||||
Poco::ScopedReadRWLock lock(storage.rwlock);
|
||||
|
||||
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
|
||||
if (*it != storage._table_column_name) /// Для виртуального столбца не надо ничего открывать
|
||||
addStream(*it, *storage.getDataTypeByName(*it));
|
||||
for (size_t i = 0, size = column_names.size(); i < size; ++i)
|
||||
{
|
||||
const auto & name = column_names[i];
|
||||
column_types[i] = storage.getDataTypeByName(name);
|
||||
if (name != storage._table_column_name) /// Для виртуального столбца не надо ничего открывать
|
||||
addStream(name, *column_types[i]);
|
||||
}
|
||||
}
|
||||
|
||||
bool has_virtual_column_table = false;
|
||||
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
|
||||
if (*it == storage._table_column_name)
|
||||
for (const auto & name : column_names)
|
||||
if (name == storage._table_column_name)
|
||||
has_virtual_column_table = true;
|
||||
|
||||
/// Сколько строк читать для следующего блока.
|
||||
@ -210,15 +215,17 @@ Block LogBlockInputStream::readImpl()
|
||||
using OffsetColumns = std::map<std::string, ColumnPtr>;
|
||||
OffsetColumns offset_columns;
|
||||
|
||||
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
|
||||
for (size_t i = 0, size = column_names.size(); i < size; ++i)
|
||||
{
|
||||
const auto & name = column_names[i];
|
||||
|
||||
/// Виртуальный столбец не надо считывать с жесткого диска
|
||||
if (*it == storage._table_column_name)
|
||||
if (name == storage._table_column_name)
|
||||
continue;
|
||||
|
||||
ColumnWithTypeAndName column;
|
||||
column.name = *it;
|
||||
column.type = storage.getDataTypeByName(*it);
|
||||
column.name = name;
|
||||
column.type = column_types[i];
|
||||
|
||||
bool read_offsets = true;
|
||||
|
||||
@ -239,11 +246,11 @@ Block LogBlockInputStream::readImpl()
|
||||
|
||||
try
|
||||
{
|
||||
readData(*it, *column.type, *column.column, max_rows_to_read, 0, read_offsets);
|
||||
readData(name, *column.type, *column.column, max_rows_to_read, 0, read_offsets);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("while reading column " + *it + " at " + storage.path + escapeForFileName(storage.name));
|
||||
e.addMessage("while reading column " + name + " at " + storage.path + escapeForFileName(storage.name));
|
||||
throw;
|
||||
}
|
||||
|
||||
|
@ -44,7 +44,8 @@ class TinyLogBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
TinyLogBlockInputStream(size_t block_size_, const Names & column_names_, StorageTinyLog & storage_, size_t max_read_buffer_size_)
|
||||
: block_size(block_size_), column_names(column_names_), storage(storage_), max_read_buffer_size(max_read_buffer_size_) {}
|
||||
: block_size(block_size_), column_names(column_names_), column_types(column_names.size()),
|
||||
storage(storage_), max_read_buffer_size(max_read_buffer_size_) {}
|
||||
|
||||
String getName() const { return "TinyLog"; }
|
||||
|
||||
@ -55,6 +56,7 @@ protected:
|
||||
private:
|
||||
size_t block_size;
|
||||
Names column_names;
|
||||
DataTypes column_types;
|
||||
StorageTinyLog & storage;
|
||||
bool finished = false;
|
||||
size_t max_read_buffer_size;
|
||||
@ -172,43 +174,51 @@ Block TinyLogBlockInputStream::readImpl()
|
||||
|
||||
/// Если файлы не открыты, то открываем их.
|
||||
if (streams.empty())
|
||||
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
|
||||
addStream(*it, *storage.getDataTypeByName(*it));
|
||||
{
|
||||
for (size_t i = 0, size = column_names.size(); i < size; ++i)
|
||||
{
|
||||
const auto & name = column_names[i];
|
||||
column_types[i] = storage.getDataTypeByName(name);
|
||||
addStream(name, *column_types[i]);
|
||||
}
|
||||
}
|
||||
|
||||
/// Указатели на столбцы смещений, общие для столбцов из вложенных структур данных
|
||||
using OffsetColumns = std::map<std::string, ColumnPtr>;
|
||||
OffsetColumns offset_columns;
|
||||
|
||||
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
|
||||
for (size_t i = 0, size = column_names.size(); i < size; ++i)
|
||||
{
|
||||
const auto & name = column_names[i];
|
||||
|
||||
ColumnWithTypeAndName column;
|
||||
column.name = *it;
|
||||
column.type = storage.getDataTypeByName(*it);
|
||||
column.name = name;
|
||||
column.type = column_types[i];
|
||||
|
||||
bool read_offsets = true;
|
||||
|
||||
/// Для вложенных структур запоминаем указатели на столбцы со смещениями
|
||||
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&*column.type))
|
||||
{
|
||||
String name = DataTypeNested::extractNestedTableName(column.name);
|
||||
String nested_name = DataTypeNested::extractNestedTableName(column.name);
|
||||
|
||||
if (offset_columns.count(name) == 0)
|
||||
offset_columns[name] = std::make_shared<ColumnArray::ColumnOffsets_t>();
|
||||
if (offset_columns.count(nested_name) == 0)
|
||||
offset_columns[nested_name] = std::make_shared<ColumnArray::ColumnOffsets_t>();
|
||||
else
|
||||
read_offsets = false; /// на предыдущих итерациях смещения уже считали вызовом readData
|
||||
|
||||
column.column = std::make_shared<ColumnArray>(type_arr->getNestedType()->createColumn(), offset_columns[name]);
|
||||
column.column = std::make_shared<ColumnArray>(type_arr->getNestedType()->createColumn(), offset_columns[nested_name]);
|
||||
}
|
||||
else
|
||||
column.column = column.type->createColumn();
|
||||
|
||||
try
|
||||
{
|
||||
readData(*it, *column.type, *column.column, block_size, 0, read_offsets);
|
||||
readData(name, *column.type, *column.column, block_size, 0, read_offsets);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("while reading column " + *it + " at " + storage.full_path());
|
||||
e.addMessage("while reading column " + name + " at " + storage.full_path());
|
||||
throw;
|
||||
}
|
||||
|
||||
|
@ -109,7 +109,7 @@ try
|
||||
elem.name = name_type.name;
|
||||
elem.type = name_type.type;
|
||||
elem.column = elem.type->createColumn();
|
||||
sample.insert(elem);
|
||||
sample.insert(std::move(elem));
|
||||
}
|
||||
|
||||
/// читаем данные из tsv файла и одновременно пишем в таблицу
|
||||
|
Loading…
Reference in New Issue
Block a user