mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
parent
47e2b3a35a
commit
6325d4d9b0
@ -13,7 +13,7 @@ private:
|
||||
DataTypePtr dictionary_type;
|
||||
|
||||
public:
|
||||
DataTypeLowCardinality(DataTypePtr dictionary_type_);
|
||||
explicit DataTypeLowCardinality(DataTypePtr dictionary_type_);
|
||||
|
||||
const DataTypePtr & getDictionaryType() const { return dictionary_type; }
|
||||
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
@ -966,7 +967,9 @@ bool StorageFileLog::updateFileInfos()
|
||||
|
||||
NamesAndTypesList StorageFileLog::getVirtuals() const
|
||||
{
|
||||
return NamesAndTypesList{{"_filename", std::make_shared<DataTypeString>()}, {"_offset", std::make_shared<DataTypeUInt64>()}};
|
||||
return NamesAndTypesList{
|
||||
{"_filename", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
|
||||
{"_offset", std::make_shared<DataTypeUInt64>()}};
|
||||
}
|
||||
|
||||
Names StorageFileLog::getVirtualColumnNames()
|
||||
|
@ -258,9 +258,15 @@ Block HDFSSource::getHeader(const StorageMetadataPtr & metadata_snapshot, bool n
|
||||
auto header = metadata_snapshot->getSampleBlock();
|
||||
/// Note: AddingDefaultsBlockInputStream doesn't change header.
|
||||
if (need_path_column)
|
||||
header.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_path"});
|
||||
header.insert(
|
||||
{DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn(),
|
||||
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
|
||||
"_path"});
|
||||
if (need_file_column)
|
||||
header.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_file"});
|
||||
header.insert(
|
||||
{DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn(),
|
||||
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
|
||||
"_file"});
|
||||
return header;
|
||||
}
|
||||
|
||||
@ -378,7 +384,7 @@ Chunk HDFSSource::generate()
|
||||
/// Enrich with virtual columns.
|
||||
if (need_path_column)
|
||||
{
|
||||
auto column = DataTypeString().createColumnConst(num_rows, current_path);
|
||||
auto column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumnConst(num_rows, current_path);
|
||||
columns.push_back(column->convertToFullColumnIfConst());
|
||||
}
|
||||
|
||||
@ -387,7 +393,7 @@ Chunk HDFSSource::generate()
|
||||
size_t last_slash_pos = current_path.find_last_of('/');
|
||||
auto file_name = current_path.substr(last_slash_pos + 1);
|
||||
|
||||
auto column = DataTypeString().createColumnConst(num_rows, std::move(file_name));
|
||||
auto column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumnConst(num_rows, std::move(file_name));
|
||||
columns.push_back(column->convertToFullColumnIfConst());
|
||||
}
|
||||
|
||||
@ -689,9 +695,8 @@ void registerStorageHDFS(StorageFactory & factory)
|
||||
NamesAndTypesList StorageHDFS::getVirtuals() const
|
||||
{
|
||||
return NamesAndTypesList{
|
||||
{"_path", std::make_shared<DataTypeString>()},
|
||||
{"_file", std::make_shared<DataTypeString>()}
|
||||
};
|
||||
{"_path", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
|
||||
{"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())}};
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -138,9 +138,8 @@ QueryProcessingStage::Enum StorageHDFSCluster::getQueryProcessingStage(
|
||||
NamesAndTypesList StorageHDFSCluster::getVirtuals() const
|
||||
{
|
||||
return NamesAndTypesList{
|
||||
{"_path", std::make_shared<DataTypeString>()},
|
||||
{"_file", std::make_shared<DataTypeString>()}
|
||||
};
|
||||
{"_path", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
|
||||
{"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())}};
|
||||
}
|
||||
|
||||
|
||||
|
@ -76,9 +76,15 @@ public:
|
||||
static Block getHeader(Block header, const SourcesInfoPtr & source_info)
|
||||
{
|
||||
if (source_info->need_path_column)
|
||||
header.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_path"});
|
||||
header.insert(
|
||||
{DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn(),
|
||||
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
|
||||
"_path"});
|
||||
if (source_info->need_file_column)
|
||||
header.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_file"});
|
||||
header.insert(
|
||||
{DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn(),
|
||||
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
|
||||
"_file"});
|
||||
|
||||
return header;
|
||||
}
|
||||
@ -87,9 +93,9 @@ public:
|
||||
{
|
||||
ColumnsDescription columns_description{header.getNamesAndTypesList()};
|
||||
if (source_info->need_path_column)
|
||||
columns_description.add({"_path", std::make_shared<DataTypeString>()});
|
||||
columns_description.add({"_path", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())});
|
||||
if (source_info->need_file_column)
|
||||
columns_description.add({"_file", std::make_shared<DataTypeString>()});
|
||||
columns_description.add({"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())});
|
||||
return columns_description;
|
||||
}
|
||||
|
||||
@ -211,7 +217,7 @@ public:
|
||||
/// Enrich with virtual columns.
|
||||
if (source_info->need_path_column)
|
||||
{
|
||||
auto column = DataTypeString().createColumnConst(num_rows, current_path);
|
||||
auto column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumnConst(num_rows, current_path);
|
||||
columns.push_back(column->convertToFullColumnIfConst());
|
||||
}
|
||||
|
||||
@ -220,7 +226,8 @@ public:
|
||||
size_t last_slash_pos = current_path.find_last_of('/');
|
||||
auto file_name = current_path.substr(last_slash_pos + 1);
|
||||
|
||||
auto column = DataTypeString().createColumnConst(num_rows, std::move(file_name));
|
||||
auto column
|
||||
= DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumnConst(num_rows, std::move(file_name));
|
||||
columns.push_back(column->convertToFullColumnIfConst());
|
||||
}
|
||||
return Chunk(std::move(columns), num_rows);
|
||||
@ -633,7 +640,9 @@ SinkToStoragePtr StorageHive::write(const ASTPtr & /*query*/, const StorageMetad
|
||||
|
||||
NamesAndTypesList StorageHive::getVirtuals() const
|
||||
{
|
||||
return NamesAndTypesList{{"_path", std::make_shared<DataTypeString>()}, {"_file", std::make_shared<DataTypeString>()}};
|
||||
return NamesAndTypesList{
|
||||
{"_path", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
|
||||
{"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())}};
|
||||
}
|
||||
|
||||
void registerStorageHive(StorageFactory & factory)
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
@ -16,13 +17,15 @@
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Processors/Executors/CompletedPipelineExecutor.h>
|
||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||
#include <Storages/Kafka/KafkaBlockOutputStream.h>
|
||||
#include <Storages/Kafka/KafkaSettings.h>
|
||||
#include <Storages/Kafka/KafkaSource.h>
|
||||
#include <Storages/Kafka/WriteBufferToKafkaProducer.h>
|
||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/StorageMaterializedView.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <base/logger_useful.h>
|
||||
#include <boost/algorithm/string/replace.hpp>
|
||||
#include <boost/algorithm/string/split.hpp>
|
||||
#include <boost/algorithm/string/trim.hpp>
|
||||
@ -36,8 +39,6 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <base/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -807,15 +808,14 @@ void registerStorageKafka(StorageFactory & factory)
|
||||
NamesAndTypesList StorageKafka::getVirtuals() const
|
||||
{
|
||||
auto result = NamesAndTypesList{
|
||||
{"_topic", std::make_shared<DataTypeString>()},
|
||||
{"_topic", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
|
||||
{"_key", std::make_shared<DataTypeString>()},
|
||||
{"_offset", std::make_shared<DataTypeUInt64>()},
|
||||
{"_partition", std::make_shared<DataTypeUInt64>()},
|
||||
{"_timestamp", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeDateTime>())},
|
||||
{"_timestamp_ms", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeDateTime64>(3))},
|
||||
{"_headers.name", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"_headers.value", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())}
|
||||
};
|
||||
{"_headers.value", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())}};
|
||||
if (kafka_settings->kafka_handle_error_mode == HandleKafkaErrorMode::STREAM)
|
||||
{
|
||||
result.push_back({"_raw_message", std::make_shared<DataTypeString>()});
|
||||
|
@ -302,13 +302,13 @@ NamesAndTypesList StorageDistributed::getVirtuals() const
|
||||
/// NOTE This is weird. Most of these virtual columns are part of MergeTree
|
||||
/// tables info. But Distributed is general-purpose engine.
|
||||
return NamesAndTypesList{
|
||||
NameAndTypePair("_table", std::make_shared<DataTypeString>()),
|
||||
NameAndTypePair("_part", std::make_shared<DataTypeString>()),
|
||||
NameAndTypePair("_part_index", std::make_shared<DataTypeUInt64>()),
|
||||
NameAndTypePair("_part_uuid", std::make_shared<DataTypeUUID>()),
|
||||
NameAndTypePair("_partition_id", std::make_shared<DataTypeString>()),
|
||||
NameAndTypePair("_sample_factor", std::make_shared<DataTypeFloat64>()),
|
||||
NameAndTypePair("_shard_num", std::make_shared<DataTypeUInt32>()), /// deprecated
|
||||
NameAndTypePair("_table", std::make_shared<DataTypeString>()),
|
||||
NameAndTypePair("_part", std::make_shared<DataTypeString>()),
|
||||
NameAndTypePair("_part_index", std::make_shared<DataTypeUInt64>()),
|
||||
NameAndTypePair("_part_uuid", std::make_shared<DataTypeUUID>()),
|
||||
NameAndTypePair("_partition_id", std::make_shared<DataTypeString>()),
|
||||
NameAndTypePair("_sample_factor", std::make_shared<DataTypeFloat64>()),
|
||||
NameAndTypePair("_shard_num", std::make_shared<DataTypeUInt32>()), /// deprecated
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -209,9 +209,15 @@ String StorageS3Source::KeysIterator::next()
|
||||
Block StorageS3Source::getHeader(Block sample_block, bool with_path_column, bool with_file_column)
|
||||
{
|
||||
if (with_path_column)
|
||||
sample_block.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_path"});
|
||||
sample_block.insert(
|
||||
{DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn(),
|
||||
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
|
||||
"_path"});
|
||||
if (with_file_column)
|
||||
sample_block.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_file"});
|
||||
sample_block.insert(
|
||||
{DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn(),
|
||||
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
|
||||
"_file"});
|
||||
|
||||
return sample_block;
|
||||
}
|
||||
@ -305,12 +311,15 @@ Chunk StorageS3Source::generate()
|
||||
UInt64 num_rows = chunk.getNumRows();
|
||||
|
||||
if (with_path_column)
|
||||
chunk.addColumn(DataTypeString().createColumnConst(num_rows, file_path)->convertToFullColumnIfConst());
|
||||
chunk.addColumn(DataTypeLowCardinality{std::make_shared<DataTypeString>()}
|
||||
.createColumnConst(num_rows, file_path)
|
||||
->convertToFullColumnIfConst());
|
||||
if (with_file_column)
|
||||
{
|
||||
size_t last_slash_pos = file_path.find_last_of('/');
|
||||
chunk.addColumn(DataTypeString().createColumnConst(num_rows, file_path.substr(
|
||||
last_slash_pos + 1))->convertToFullColumnIfConst());
|
||||
chunk.addColumn(DataTypeLowCardinality{std::make_shared<DataTypeString>()}
|
||||
.createColumnConst(num_rows, file_path.substr(last_slash_pos + 1))
|
||||
->convertToFullColumnIfConst());
|
||||
}
|
||||
|
||||
return chunk;
|
||||
@ -961,9 +970,8 @@ void registerStorageCOS(StorageFactory & factory)
|
||||
NamesAndTypesList StorageS3::getVirtuals() const
|
||||
{
|
||||
return NamesAndTypesList{
|
||||
{"_path", std::make_shared<DataTypeString>()},
|
||||
{"_file", std::make_shared<DataTypeString>()}
|
||||
};
|
||||
{"_path", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
|
||||
{"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())}};
|
||||
}
|
||||
|
||||
bool StorageS3::supportsPartitionBy() const
|
||||
|
@ -152,9 +152,8 @@ QueryProcessingStage::Enum StorageS3Cluster::getQueryProcessingStage(
|
||||
NamesAndTypesList StorageS3Cluster::getVirtuals() const
|
||||
{
|
||||
return NamesAndTypesList{
|
||||
{"_path", std::make_shared<DataTypeString>()},
|
||||
{"_file", std::make_shared<DataTypeString>()}
|
||||
};
|
||||
{"_path", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
|
||||
{"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())}};
|
||||
}
|
||||
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user