mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 21:12:28 +00:00
reading optimization when all columns to read are partition keys
This commit is contained in:
parent
788b704f7c
commit
4411fd87c8
@ -151,13 +151,26 @@ public:
|
|||||||
{
|
{
|
||||||
if (!reader)
|
if (!reader)
|
||||||
{
|
{
|
||||||
|
if (current_file_remained_rows) [[unlikely]]
|
||||||
|
{
|
||||||
|
return generateChunkByPartitionKeys();
|
||||||
|
}
|
||||||
|
|
||||||
current_idx = source_info->next_uri_to_read.fetch_add(1);
|
current_idx = source_info->next_uri_to_read.fetch_add(1);
|
||||||
if (current_idx >= source_info->hive_files.size())
|
if (current_idx >= source_info->hive_files.size())
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
const auto & current_file = source_info->hive_files[current_idx];
|
current_file = source_info->hive_files[current_idx];
|
||||||
current_path = current_file->getPath();
|
current_path = current_file->getPath();
|
||||||
|
|
||||||
|
if (!to_read_block.columns() && current_file->getRows())
|
||||||
|
{
|
||||||
|
/// this is the case that all columns to read are partition keys. We can construct const columns
|
||||||
|
/// directly without reading from hive files.
|
||||||
|
current_file_remained_rows = *(current_file->getRows());
|
||||||
|
return generateChunkByPartitionKeys();
|
||||||
|
}
|
||||||
|
|
||||||
String uri_with_path = hdfs_namenode_url + current_path;
|
String uri_with_path = hdfs_namenode_url + current_path;
|
||||||
auto compression = chooseCompressionMethod(current_path, compression_method);
|
auto compression = chooseCompressionMethod(current_path, compression_method);
|
||||||
std::unique_ptr<ReadBuffer> raw_read_buf;
|
std::unique_ptr<ReadBuffer> raw_read_buf;
|
||||||
@ -260,6 +273,45 @@ public:
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Chunk generateChunkByPartitionKeys()
|
||||||
|
{
|
||||||
|
size_t max_rows = getContext()->getSettings().max_block_size;
|
||||||
|
size_t rows = 0;
|
||||||
|
if (max_rows > current_file_remained_rows)
|
||||||
|
{
|
||||||
|
rows = current_file_remained_rows;
|
||||||
|
current_file_remained_rows = 0;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
rows = max_rows;
|
||||||
|
current_file_remained_rows -= max_rows;
|
||||||
|
}
|
||||||
|
|
||||||
|
Columns cols;
|
||||||
|
auto types = source_info->partition_name_types.getTypes();
|
||||||
|
auto names = source_info->partition_name_types.getNames();
|
||||||
|
auto fields = current_file->getPartitionValues();
|
||||||
|
for (size_t i = 0, sz = types.size(); i < sz; ++i)
|
||||||
|
{
|
||||||
|
if (!sample_block.has(names[i]))
|
||||||
|
continue;
|
||||||
|
auto col = types[i]->createColumnConst(rows, fields[i]);
|
||||||
|
auto col_idx = sample_block.getPositionByName(names[i]);
|
||||||
|
cols.insert(cols.begin() + col_idx, col);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (source_info->need_file_column)
|
||||||
|
{
|
||||||
|
size_t last_slash_pos = current_file->getPath().find_last_of('/');
|
||||||
|
auto file_name = current_path.substr(last_slash_pos + 1);
|
||||||
|
|
||||||
|
auto col = DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumnConst(rows, std::move(file_name));
|
||||||
|
cols.push_back(col);
|
||||||
|
}
|
||||||
|
return Chunk(std::move(cols), rows);
|
||||||
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
std::unique_ptr<ReadBuffer> read_buf;
|
std::unique_ptr<ReadBuffer> read_buf;
|
||||||
std::unique_ptr<QueryPipeline> pipeline;
|
std::unique_ptr<QueryPipeline> pipeline;
|
||||||
@ -275,8 +327,10 @@ private:
|
|||||||
const Names & text_input_field_names;
|
const Names & text_input_field_names;
|
||||||
FormatSettings format_settings;
|
FormatSettings format_settings;
|
||||||
|
|
||||||
|
HiveFilePtr current_file;
|
||||||
String current_path;
|
String current_path;
|
||||||
size_t current_idx = 0;
|
size_t current_idx = 0;
|
||||||
|
size_t current_file_remained_rows = 0;
|
||||||
|
|
||||||
Poco::Logger * log = &Poco::Logger::get("StorageHive");
|
Poco::Logger * log = &Poco::Logger::get("StorageHive");
|
||||||
};
|
};
|
||||||
@ -627,30 +681,6 @@ bool StorageHive::isColumnOriented() const
|
|||||||
return format_name == "Parquet" || format_name == "ORC";
|
return format_name == "Parquet" || format_name == "ORC";
|
||||||
}
|
}
|
||||||
|
|
||||||
void StorageHive::getActualColumnsToRead(Block & sample_block, const Block & header_block, const NameSet & partition_columns) const
|
|
||||||
{
|
|
||||||
if (!isColumnOriented())
|
|
||||||
sample_block = header_block;
|
|
||||||
UInt32 erased_columns = 0;
|
|
||||||
for (const auto & column : partition_columns)
|
|
||||||
{
|
|
||||||
if (sample_block.has(column))
|
|
||||||
erased_columns++;
|
|
||||||
}
|
|
||||||
if (erased_columns == sample_block.columns())
|
|
||||||
{
|
|
||||||
for (size_t i = 0; i < header_block.columns(); ++i)
|
|
||||||
{
|
|
||||||
const auto & col = header_block.getByPosition(i);
|
|
||||||
if (!partition_columns.count(col.name))
|
|
||||||
{
|
|
||||||
sample_block.insert(col);
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
Pipe StorageHive::read(
|
Pipe StorageHive::read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
const StorageSnapshotPtr & storage_snapshot,
|
const StorageSnapshotPtr & storage_snapshot,
|
||||||
@ -690,8 +720,6 @@ Pipe StorageHive::read(
|
|||||||
sources_info->need_file_column = true;
|
sources_info->need_file_column = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
getActualColumnsToRead(sample_block, header_block, NameSet{partition_names.begin(), partition_names.end()});
|
|
||||||
|
|
||||||
if (num_streams > sources_info->hive_files.size())
|
if (num_streams > sources_info->hive_files.size())
|
||||||
num_streams = sources_info->hive_files.size();
|
num_streams = sources_info->hive_files.size();
|
||||||
|
|
||||||
|
@ -117,8 +117,6 @@ private:
|
|||||||
const ContextPtr & context_,
|
const ContextPtr & context_,
|
||||||
PruneLevel prune_level = PruneLevel::Max) const;
|
PruneLevel prune_level = PruneLevel::Max) const;
|
||||||
|
|
||||||
void getActualColumnsToRead(Block & sample_block, const Block & header_block, const NameSet & partition_columns) const;
|
|
||||||
|
|
||||||
void lazyInitialize();
|
void lazyInitialize();
|
||||||
|
|
||||||
std::optional<UInt64>
|
std::optional<UInt64>
|
||||||
|
Loading…
Reference in New Issue
Block a user