mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
minor fixes
This commit is contained in:
parent
2758db5341
commit
04a3a10148
@ -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();
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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())
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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};
|
||||
}
|
||||
|
@ -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*/)
|
||||
|
@ -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));
|
||||
}
|
||||
|
@ -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));
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user