minor fixes

This commit is contained in:
Anton Popov 2022-03-01 20:20:53 +03:00
parent 2758db5341
commit 04a3a10148
11 changed files with 14 additions and 14 deletions

View File

@ -295,7 +295,7 @@ ColumnPtr ColumnAggregateFunction::filter(const Filter & filter, ssize_t result_
{
size_t size = data.size();
if (size != filter.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of filter ({}) doesn't match size of column ({})", filter.size(), size);
if (size == 0)
return cloneEmpty();

View File

@ -592,7 +592,7 @@ ColumnPtr ColumnArray::filterString(const Filter & filt, ssize_t result_size_hin
{
size_t col_size = getOffsets().size();
if (col_size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of filter ({}) doesn't match size of column ({})", filt.size(), col_size);
if (0 == col_size)
return ColumnArray::create(data);
@ -660,7 +660,7 @@ ColumnPtr ColumnArray::filterGeneric(const Filter & filt, ssize_t result_size_hi
{
size_t size = getOffsets().size();
if (size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of filter ({}) doesn't match size of column ({})", filt.size(), size);
if (size == 0)
return ColumnArray::create(data);

View File

@ -205,7 +205,7 @@ ColumnPtr ColumnDecimal<T>::filter(const IColumn::Filter & filt, ssize_t result_
{
size_t size = data.size();
if (size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of filter ({}) doesn't match size of column ({})", filt.size(), size);
auto res = this->create(0, scale);
Container & res_data = res->getData();

View File

@ -226,7 +226,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result
{
size_t col_size = size();
if (col_size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of filter ({}) doesn't match size of column ({})", filt.size(), col_size);
auto res = ColumnFixedString::create(n);

View File

@ -288,7 +288,7 @@ void ColumnSparse::popBack(size_t n)
ColumnPtr ColumnSparse::filter(const Filter & filt, ssize_t) const
{
if (_size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of filter ({}) doesn't match size of column ({})", filt.size(), _size);
if (offsets->empty())
{

View File

@ -297,7 +297,7 @@ ColumnPtr ColumnVector<T>::filter(const IColumn::Filter & filt, ssize_t result_s
{
size_t size = data.size();
if (size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of filter ({}) doesn't match size of column ({})", filt.size(), size);
auto res = this->create();
Container & res_data = res->getData();
@ -366,7 +366,7 @@ void ColumnVector<T>::applyZeroMap(const IColumn::Filter & filt, bool inverted)
{
size_t size = data.size();
if (size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of filter ({}) doesn't match size of column ({})", filt.size(), size);
const UInt8 * filt_pos = filt.data();
const UInt8 * filt_end = filt_pos + size;

View File

@ -192,7 +192,7 @@ namespace
{
const size_t size = src_offsets.size();
if (size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of filter ({}) doesn't match size of column ({})", filt.size(), size);
ResultOffsetsBuilder result_offsets_builder(res_offsets);

View File

@ -149,7 +149,7 @@ NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & requ
auto subcolumn_type = type->tryGetSubcolumnType(subcolumn_name);
if (!subcolumn_type)
return {required_column.name, required_column.type};
return required_column;
return {String(it->getKey()), subcolumn_name, type, subcolumn_type};
}

View File

@ -6117,7 +6117,7 @@ ColumnsDescription MergeTreeData::getObjectColumns(
{
return DB::getObjectColumns(
parts.begin(), parts.end(),
storage_columns, [](const auto & part) { return part->getColumns(); });
storage_columns, [](const auto & part) -> const auto & { return part->getColumns(); });
}
ColumnsDescription MergeTreeData::getObjectColumns(
@ -6125,7 +6125,7 @@ ColumnsDescription MergeTreeData::getObjectColumns(
{
return DB::getObjectColumns(
range.begin(), range.end(),
storage_columns, [](const auto & part) { return part->getColumns(); });
storage_columns, [](const auto & part) -> const auto & { return part->getColumns(); });
}
void MergeTreeData::resetObjectColumnsFromActiveParts(const DataPartsLock & /*lock*/)

View File

@ -638,7 +638,7 @@ StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery(
snapshot_data->objects_by_shard.begin(),
snapshot_data->objects_by_shard.end(),
metadata_snapshot->getColumns(),
[](const auto & shard_num_and_columns) { return shard_num_and_columns.second; });
[](const auto & shard_num_and_columns) -> const auto & { return shard_num_and_columns.second; });
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, object_columns, std::move(snapshot_data));
}

View File

@ -202,7 +202,7 @@ StorageSnapshotPtr StorageMemory::getStorageSnapshot(const StorageMetadataPtr &
snapshot_data->blocks->begin(),
snapshot_data->blocks->end(),
metadata_snapshot->getColumns(),
[](const auto & block) { return block.getColumnsWithTypeAndName(); });
[](const auto & block) -> const auto & { return block.getColumnsWithTypeAndName(); });
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, object_columns, std::move(snapshot_data));
}