Merge branch 'master' of github.com:ClickHouse/ClickHouse into remove_integration_test_datetime

This commit is contained in:
Nikita Mikhaylov 2024-08-25 14:32:26 +02:00
commit 21eeda9d0b
52 changed files with 390 additions and 334 deletions

View File

@ -54,6 +54,7 @@ Other upcoming meetups
* [Sydney Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302862966/) - September 5
* [Zurich Meetup](https://www.meetup.com/clickhouse-switzerland-meetup-group/events/302267429/) - September 5
* [Toronto Meetup (Shopify)](https://www.meetup.com/clickhouse-toronto-user-group/events/301490855/) - September 10
* [Austin Meetup](https://www.meetup.com/clickhouse-austin-user-group/events/302558689/) - September 17
* [London Meetup](https://www.meetup.com/clickhouse-london-user-group/events/302977267) - September 17
## Recent Recordings

View File

@ -80,7 +80,7 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da
`PRIMARY KEY` — The primary key if it [differs from the sorting key](#choosing-a-primary-key-that-differs-from-the-sorting-key). Optional.
Specifying a sorting key (using `ORDER BY` clause) implicitly specifies a primary key.
It is usually not necessary to specify the primary key in addition to the primary key.
It is usually not necessary to specify the primary key in addition to the sorting key.
#### SAMPLE BY

View File

@ -692,7 +692,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromStorage(
result_column_node = it->second;
}
/// Check if it's a dynamic subcolumn
else
else if (table_expression_data.supports_subcolumns)
{
auto [column_name, dynamic_subcolumn_name] = Nested::splitName(identifier_full_name);
auto jt = table_expression_data.column_name_to_column_node.find(column_name);

View File

@ -4379,7 +4379,10 @@ void QueryAnalyzer::initializeTableExpressionData(const QueryTreeNodePtr & table
auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals();
if (storage_snapshot->storage.supportsSubcolumns())
{
get_column_options.withSubcolumns();
table_expression_data.supports_subcolumns = true;
}
auto column_names_and_types = storage_snapshot->getColumns(get_column_options);
table_expression_data.column_names_and_types = NamesAndTypes(column_names_and_types.begin(), column_names_and_types.end());

View File

@ -36,6 +36,7 @@ struct AnalysisTableExpressionData
std::string database_name;
std::string table_name;
bool should_qualify_columns = true;
bool supports_subcolumns = false;
NamesAndTypes column_names_and_types;
ColumnNameToColumnNodeMap column_name_to_column_node;
std::unordered_set<std::string> subcolumn_names; /// Subset columns that are subcolumns of other columns

View File

@ -1181,13 +1181,14 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source
/// Check if the number of all dynamic types exceeds the limit.
if (!canAddNewVariants(0, all_variants.size()))
{
/// Create list of variants with their sizes and sort it.
std::vector<std::pair<size_t, DataTypePtr>> variants_with_sizes;
/// Create a list of variants with their sizes and names and then sort it.
std::vector<std::tuple<size_t, String, DataTypePtr>> variants_with_sizes;
variants_with_sizes.reserve(all_variants.size());
for (const auto & variant : all_variants)
{
if (variant->getName() != getSharedVariantTypeName())
variants_with_sizes.emplace_back(total_sizes[variant->getName()], variant);
auto variant_name = variant->getName();
if (variant_name != getSharedVariantTypeName())
variants_with_sizes.emplace_back(total_sizes[variant_name], variant_name, variant);
}
std::sort(variants_with_sizes.begin(), variants_with_sizes.end(), std::greater());
@ -1196,14 +1197,14 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source
result_variants.reserve(max_dynamic_types + 1); /// +1 for shared variant.
/// Add shared variant.
result_variants.push_back(getSharedVariantDataType());
for (const auto & [size, variant] : variants_with_sizes)
for (const auto & [size, variant_name, variant_type] : variants_with_sizes)
{
/// Add variant to the resulting variants list until we reach max_dynamic_types.
if (canAddNewVariant(result_variants.size()))
result_variants.push_back(variant);
result_variants.push_back(variant_type);
/// Add all remaining variants into shared_variants_statistics until we reach its max size.
else if (new_statistics.shared_variants_statistics.size() < Statistics::MAX_SHARED_VARIANT_STATISTICS_SIZE)
new_statistics.shared_variants_statistics[variant->getName()] = size;
new_statistics.shared_variants_statistics[variant_name] = size;
else
break;
}

View File

@ -127,7 +127,7 @@ std::string ColumnObject::getName() const
{
WriteBufferFromOwnString ss;
ss << "Object(";
ss << "max_dynamic_paths=" << max_dynamic_paths;
ss << "max_dynamic_paths=" << global_max_dynamic_paths;
ss << ", max_dynamic_types=" << max_dynamic_types;
std::vector<String> sorted_typed_paths;
sorted_typed_paths.reserve(typed_paths.size());
@ -1045,9 +1045,9 @@ void ColumnObject::forEachSubcolumnRecursively(DB::IColumn::RecursiveMutableColu
bool ColumnObject::structureEquals(const IColumn & rhs) const
{
/// 2 Object columns have equal structure if they have the same typed paths and max_dynamic_paths/max_dynamic_types.
/// 2 Object columns have equal structure if they have the same typed paths and global_max_dynamic_paths/max_dynamic_types.
const auto * rhs_object = typeid_cast<const ColumnObject *>(&rhs);
if (!rhs_object || typed_paths.size() != rhs_object->typed_paths.size() || max_dynamic_paths != rhs_object->max_dynamic_paths || max_dynamic_types != rhs_object->max_dynamic_types)
if (!rhs_object || typed_paths.size() != rhs_object->typed_paths.size() || global_max_dynamic_paths != rhs_object->global_max_dynamic_paths || max_dynamic_types != rhs_object->max_dynamic_types)
return false;
for (const auto & [path, column] : typed_paths)

View File

@ -953,7 +953,7 @@ ColumnPtr ColumnVariant::index(const IColumn & indexes, size_t limit) const
{
/// If we have only NULLs, index will take no effect, just return resized column.
if (hasOnlyNulls())
return cloneResized(limit);
return cloneResized(limit == 0 ? indexes.size(): limit);
/// Optimization when we have only one non empty variant and no NULLs.
/// In this case local_discriminators column is filled with identical values and offsets column
@ -1009,8 +1009,16 @@ ColumnPtr ColumnVariant::indexImpl(const PaddedPODArray<Type> & indexes, size_t
new_variants.reserve(num_variants);
for (size_t i = 0; i != num_variants; ++i)
{
size_t nested_limit = nested_perms[i].size() == variants[i]->size() ? 0 : nested_perms[i].size();
new_variants.emplace_back(variants[i]->permute(nested_perms[i], nested_limit));
/// Check if no values from this variant were selected.
if (nested_perms[i].empty())
{
new_variants.emplace_back(variants[i]->cloneEmpty());
}
else
{
size_t nested_limit = nested_perms[i].size() == variants[i]->size() ? 0 : nested_perms[i].size();
new_variants.emplace_back(variants[i]->permute(nested_perms[i], nested_limit));
}
}
/// We cannot use new_offsets column as an offset column, because it became invalid after variants permutation.

View File

@ -701,7 +701,6 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_deflate_qpl_codec, enable_zstd_qat_codec);
}
column.statistics.column_name = column.name; /// We assign column name here for better exception error message.
if (col_decl.statistics_desc)
{
if (!skip_checks && !context_->getSettingsRef().allow_experimental_statistics)

View File

@ -706,9 +706,9 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
}
auto stats_vec = ColumnStatisticsDescription::fromAST(statistics_decl, metadata.columns);
for (const auto & stats : stats_vec)
for (const auto & [stats_column_name, stats] : stats_vec)
{
metadata.columns.modify(stats.column_name,
metadata.columns.modify(stats_column_name,
[&](ColumnDescription & column) { column.statistics.merge(stats, column.name, column.type, if_not_exists); });
}
}
@ -735,14 +735,14 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
{
if (!metadata.columns.has(statistics_column_name))
{
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Cannot add statistics for column {}: this column is not found", statistics_column_name);
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Cannot modify statistics for column {}: this column is not found", statistics_column_name);
}
}
auto stats_vec = ColumnStatisticsDescription::fromAST(statistics_decl, metadata.columns);
for (const auto & stats : stats_vec)
for (const auto & [stats_column_name, stats] : stats_vec)
{
metadata.columns.modify(stats.column_name,
metadata.columns.modify(stats_column_name,
[&](ColumnDescription & column) { column.statistics.assign(stats); });
}
}
@ -867,8 +867,6 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
rename_visitor.visit(column_to_modify.default_desc.expression);
if (column_to_modify.ttl)
rename_visitor.visit(column_to_modify.ttl);
if (column_to_modify.name == column_name && !column_to_modify.statistics.empty())
column_to_modify.statistics.column_name = rename_to;
});
}
if (metadata.table_ttl.definition_ast)

View File

@ -218,11 +218,7 @@ void ColumnDescription::readText(ReadBuffer & buf)
settings = col_ast->settings->as<ASTSetQuery &>().changes;
if (col_ast->statistics_desc)
{
statistics = ColumnStatisticsDescription::fromColumnDeclaration(*col_ast, type);
/// every column has name `x` here, so we have to set the name manually.
statistics.column_name = name;
}
}
else
throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description");

View File

@ -444,8 +444,8 @@ StorageHive::StorageHive(
storage_metadata.setComment(comment_);
storage_metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, storage_metadata.columns, getContext());
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.columns, getContext()));
setInMemoryMetadata(storage_metadata);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), getContext()));
}
void StorageHive::lazyInitialize()

View File

@ -75,7 +75,7 @@ static ColumnsStatistics getStatisticsForColumns(
const auto * desc = all_columns.tryGet(column.name);
if (desc && !desc->statistics.empty())
{
auto statistics = MergeTreeStatisticsFactory::instance().get(desc->statistics);
auto statistics = MergeTreeStatisticsFactory::instance().get(*desc);
all_statistics.push_back(std::move(statistics));
}
}

View File

@ -552,7 +552,7 @@ static std::set<ColumnStatisticsPtr> getStatisticsToRecalculate(const StorageMet
{
if (!col_desc.statistics.empty() && materialized_stats.contains(col_desc.name))
{
stats_to_recalc.insert(stats_factory.get(col_desc.statistics));
stats_to_recalc.insert(stats_factory.get(col_desc));
}
}
return stats_to_recalc;
@ -1557,7 +1557,7 @@ private:
if (ctx->materialized_statistics.contains(col.name))
{
stats_to_rewrite.push_back(MergeTreeStatisticsFactory::instance().get(col.statistics));
stats_to_rewrite.push_back(MergeTreeStatisticsFactory::instance().get(col));
}
else
{

View File

@ -94,7 +94,7 @@ StorageObjectStorage::StorageObjectStorage(
if (sample_path.empty() && context->getSettingsRef().use_hive_partitioning)
sample_path = getPathSample(metadata, context);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context, sample_path, format_settings));
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.columns, context, sample_path, format_settings));
setInMemoryMetadata(metadata);
}

View File

@ -68,7 +68,7 @@ StorageObjectStorageCluster::StorageObjectStorageCluster(
if (sample_path.empty() && context_->getSettingsRef().use_hive_partitioning)
sample_path = getPathSample(metadata, context_);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context_, sample_path));
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.columns, context_, sample_path));
setInMemoryMetadata(metadata);
}

View File

@ -208,7 +208,7 @@ Chunk StorageObjectStorageSource::generate()
.filename = &filename,
.last_modified = object_info->metadata->last_modified,
.etag = &(object_info->metadata->etag)
}, getContext(), read_from_format_info.columns_description);
}, getContext());
const auto & partition_columns = configuration->getPartitionColumns();
if (!partition_columns.empty() && chunk_size && chunk.hasColumns())
@ -280,7 +280,7 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade
const std::shared_ptr<IIterator> & file_iterator,
const ConfigurationPtr & configuration,
const ObjectStoragePtr & object_storage,
const ReadFromFormatInfo & read_from_format_info,
ReadFromFormatInfo & read_from_format_info,
const std::optional<FormatSettings> & format_settings,
const std::shared_ptr<const KeyCondition> & key_condition_,
const ContextPtr & context_,

View File

@ -74,7 +74,7 @@ protected:
const UInt64 max_block_size;
const bool need_only_count;
const size_t max_parsing_threads;
const ReadFromFormatInfo read_from_format_info;
ReadFromFormatInfo read_from_format_info;
const std::shared_ptr<ThreadPool> create_reader_pool;
std::shared_ptr<IIterator> file_iterator;
@ -122,7 +122,7 @@ protected:
const std::shared_ptr<IIterator> & file_iterator,
const ConfigurationPtr & configuration,
const ObjectStoragePtr & object_storage,
const ReadFromFormatInfo & read_from_format_info,
ReadFromFormatInfo & read_from_format_info,
const std::optional<FormatSettings> & format_settings,
const std::shared_ptr<const KeyCondition> & key_condition_,
const ContextPtr & context_,

View File

@ -524,7 +524,7 @@ Chunk ObjectStorageQueueSource::generateImpl()
{
.path = path,
.size = reader.getObjectInfo()->metadata->size_bytes
}, getContext(), read_from_format_info.columns_description);
}, getContext());
return chunk;
}

View File

@ -128,7 +128,7 @@ private:
const std::shared_ptr<FileIterator> file_iterator;
const ConfigurationPtr configuration;
const ObjectStoragePtr object_storage;
const ReadFromFormatInfo read_from_format_info;
ReadFromFormatInfo read_from_format_info;
const std::optional<FormatSettings> format_settings;
const ObjectStorageQueueSettings queue_settings;
const std::shared_ptr<ObjectStorageQueueMetadata> files_metadata;

View File

@ -169,7 +169,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue(
storage_metadata.setColumns(columns);
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_));
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.columns, context_));
setInMemoryMetadata(storage_metadata);
LOG_INFO(log, "Using zookeeper path: {}", zk_path.string());

View File

@ -58,8 +58,8 @@ IStatistics::IStatistics(const SingleStatisticsDescription & stat_)
{
}
ColumnStatistics::ColumnStatistics(const ColumnStatisticsDescription & stats_desc_)
: stats_desc(stats_desc_)
ColumnStatistics::ColumnStatistics(const ColumnStatisticsDescription & stats_desc_, const String & column_name_)
: stats_desc(stats_desc_), column_name(column_name_)
{
}
@ -176,7 +176,7 @@ String ColumnStatistics::getFileName() const
const String & ColumnStatistics::columnName() const
{
return stats_desc.column_name;
return column_name;
}
UInt64 ColumnStatistics::rowCount() const
@ -227,15 +227,15 @@ void MergeTreeStatisticsFactory::validate(const ColumnStatisticsDescription & st
}
}
ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnStatisticsDescription & stats) const
ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnDescription & column_desc) const
{
ColumnStatisticsPtr column_stat = std::make_shared<ColumnStatistics>(stats);
for (const auto & [type, desc] : stats.types_to_desc)
ColumnStatisticsPtr column_stat = std::make_shared<ColumnStatistics>(column_desc.statistics, column_desc.name);
for (const auto & [type, desc] : column_desc.statistics.types_to_desc)
{
auto it = creators.find(type);
if (it == creators.end())
throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'tdigest' 'uniq' and 'count_min'", type);
auto stat_ptr = (it->second)(desc, stats.data_type);
auto stat_ptr = (it->second)(desc, column_desc.type);
column_stat->stats[type] = stat_ptr;
}
return column_stat;
@ -246,7 +246,7 @@ ColumnsStatistics MergeTreeStatisticsFactory::getMany(const ColumnsDescription &
ColumnsStatistics result;
for (const auto & col : columns)
if (!col.statistics.empty())
result.push_back(get(col.statistics));
result.push_back(get(col));
return result;
}

View File

@ -54,7 +54,7 @@ using StatisticsPtr = std::shared_ptr<IStatistics>;
class ColumnStatistics
{
public:
explicit ColumnStatistics(const ColumnStatisticsDescription & stats_desc_);
explicit ColumnStatistics(const ColumnStatisticsDescription & stats_desc_, const String & column_name_);
void serialize(WriteBuffer & buf);
void deserialize(ReadBuffer & buf);
@ -73,10 +73,12 @@ public:
private:
friend class MergeTreeStatisticsFactory;
ColumnStatisticsDescription stats_desc;
String column_name;
std::map<StatisticsType, StatisticsPtr> stats;
UInt64 rows = 0; /// the number of rows in the column
};
struct ColumnDescription;
class ColumnsDescription;
using ColumnStatisticsPtr = std::shared_ptr<ColumnStatistics>;
using ColumnsStatistics = std::vector<ColumnStatisticsPtr>;
@ -91,7 +93,7 @@ public:
using Validator = std::function<void(const SingleStatisticsDescription & stats, const DataTypePtr & data_type)>;
using Creator = std::function<StatisticsPtr(const SingleStatisticsDescription & stats, const DataTypePtr & data_type)>;
ColumnStatisticsPtr get(const ColumnStatisticsDescription & stats) const;
ColumnStatisticsPtr get(const ColumnDescription & column_desc) const;
ColumnsStatistics getMany(const ColumnsDescription & columns) const;
void registerValidator(StatisticsType type, Validator validator);

View File

@ -6,7 +6,6 @@
#include <Parsers/ASTStatisticsDeclaration.h>
#include <Parsers/queryToString.h>
#include <Parsers/ParserCreateQuery.h>
#include <Poco/Logger.h>
#include <Storages/ColumnsDescription.h>
@ -97,16 +96,13 @@ void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & othe
{
chassert(merging_column_type);
if (column_name.empty())
column_name = merging_column_name;
data_type = merging_column_type;
for (const auto & [stats_type, stats_desc]: other.types_to_desc)
{
if (!if_not_exists && types_to_desc.contains(stats_type))
{
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics type name {} has existed in column {}", stats_type, column_name);
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics type name {} has existed in column {}", stats_type, merging_column_name);
}
else if (!types_to_desc.contains(stats_type))
types_to_desc.emplace(stats_type, stats_desc);
@ -115,9 +111,6 @@ void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & othe
void ColumnStatisticsDescription::assign(const ColumnStatisticsDescription & other)
{
if (other.column_name != column_name)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot assign statistics from column {} to {}", column_name, other.column_name);
types_to_desc = other.types_to_desc;
data_type = other.data_type;
}
@ -127,7 +120,7 @@ void ColumnStatisticsDescription::clear()
types_to_desc.clear();
}
std::vector<ColumnStatisticsDescription> ColumnStatisticsDescription::fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns)
std::vector<std::pair<String, ColumnStatisticsDescription>> ColumnStatisticsDescription::fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns)
{
const auto * stat_definition_ast = definition_ast->as<ASTStatisticsDeclaration>();
if (!stat_definition_ast)
@ -145,7 +138,7 @@ std::vector<ColumnStatisticsDescription> ColumnStatisticsDescription::fromAST(co
statistics_types.emplace(stat.type, stat);
}
std::vector<ColumnStatisticsDescription> result;
std::vector<std::pair<String, ColumnStatisticsDescription>> result;
result.reserve(stat_definition_ast->columns->children.size());
for (const auto & column_ast : stat_definition_ast->columns->children)
@ -157,10 +150,9 @@ std::vector<ColumnStatisticsDescription> ColumnStatisticsDescription::fromAST(co
throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column name {}", physical_column_name);
const auto & column = columns.getPhysical(physical_column_name);
stats.column_name = column.name;
stats.data_type = column.type;
stats.types_to_desc = statistics_types;
result.push_back(stats);
result.emplace_back(physical_column_name, stats);
}
if (result.empty())
@ -175,14 +167,13 @@ ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(c
if (stat_type_list_ast->children.empty())
throw Exception(ErrorCodes::INCORRECT_QUERY, "We expect at least one statistics type for column {}", queryToString(column));
ColumnStatisticsDescription stats;
stats.column_name = column.name;
for (const auto & ast : stat_type_list_ast->children)
{
const auto & stat_type = ast->as<const ASTFunction &>().name;
SingleStatisticsDescription stat(stringToStatisticsType(Poco::toLower(stat_type)), ast->clone());
if (stats.types_to_desc.contains(stat.type))
throw Exception(ErrorCodes::INCORRECT_QUERY, "Column {} already contains statistics type {}", stats.column_name, stat_type);
throw Exception(ErrorCodes::INCORRECT_QUERY, "Column {} already contains statistics type {}", column.name, stat_type);
stats.types_to_desc.emplace(stat.type, std::move(stat));
}
stats.data_type = data_type;

View File

@ -55,12 +55,12 @@ struct ColumnStatisticsDescription
ASTPtr getAST() const;
static std::vector<ColumnStatisticsDescription> fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns);
/// get a vector of <column name, statistics desc> pair
static std::vector<std::pair<String, ColumnStatisticsDescription>> fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns);
static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type);
using StatisticsTypeDescMap = std::map<StatisticsType, SingleStatisticsDescription>;
StatisticsTypeDescMap types_to_desc;
String column_name;
DataTypePtr data_type;
};

View File

@ -1112,9 +1112,9 @@ void StorageFile::setStorageMetadata(CommonArguments args)
storage_metadata.setConstraints(args.constraints);
storage_metadata.setComment(args.comment);
setInMemoryMetadata(storage_metadata);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), args.getContext(), paths.empty() ? "" : paths[0], format_settings));
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.columns, args.getContext(), paths.empty() ? "" : paths[0], format_settings));
setInMemoryMetadata(storage_metadata);
}
@ -1468,7 +1468,7 @@ Chunk StorageFileSource::generate()
.size = current_file_size,
.filename = (filename_override.has_value() ? &filename_override.value() : nullptr),
.last_modified = current_file_last_modified
}, getContext(), columns_description);
}, getContext());
return chunk;
}

View File

@ -60,8 +60,8 @@ StorageFileCluster::StorageFileCluster(
}
storage_metadata.setConstraints(constraints_);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.columns, context, paths.empty() ? "" : paths[0]));
setInMemoryMetadata(storage_metadata);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context, paths.empty() ? "" : paths[0]));
}
void StorageFileCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context)

View File

@ -6340,7 +6340,7 @@ void StorageReplicatedMergeTree::alter(
"Metadata on replica is not up to date with common metadata in Zookeeper. "
"It means that this replica still not applied some of previous alters."
" Probably too many alters executing concurrently (highly not recommended). "
"You can retry the query");
"You can retry this error");
/// Cannot retry automatically, because some zookeeper ops were lost on the first attempt. Will retry on DDLWorker-level.
if (query_context->getZooKeeperMetadataTransaction())

View File

@ -165,9 +165,9 @@ IStorageURLBase::IStorageURLBase(
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_, getSampleURI(uri, context_), format_settings));
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.columns, context_, getSampleURI(uri, context_), format_settings));
setInMemoryMetadata(storage_metadata);
}
@ -435,7 +435,7 @@ Chunk StorageURLSource::generate()
{
.path = curr_uri.getPath(),
.size = current_file_size,
}, getContext(), columns_description);
}, getContext());
return chunk;
}

View File

@ -75,8 +75,8 @@ StorageURLCluster::StorageURLCluster(
}
storage_metadata.setConstraints(constraints_);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.columns, context, getSampleURI(uri, context)));
setInMemoryMetadata(storage_metadata);
setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context, getSampleURI(uri, context)));
}
void StorageURLCluster::updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context)

View File

@ -129,36 +129,45 @@ NameSet getVirtualNamesForFileLikeStorage()
return {"_path", "_file", "_size", "_time", "_etag"};
}
std::unordered_map<std::string, std::string> parseHivePartitioningKeysAndValues(const String & path, const ColumnsDescription & storage_columns)
std::unordered_map<std::string, std::string> parseHivePartitioningKeysAndValues(const String & path)
{
std::string pattern = "([^/]+)=([^/]+)/";
re2::StringPiece input_piece(path);
std::unordered_map<std::string, std::string> key_values;
std::string key, value;
std::unordered_set<String> used_keys;
std::unordered_map<std::string, std::string> used_keys;
while (RE2::FindAndConsume(&input_piece, pattern, &key, &value))
{
if (used_keys.contains(key))
throw Exception(ErrorCodes::INCORRECT_DATA, "Path '{}' to file with enabled hive-style partitioning contains duplicated partition key {}, only unique keys are allowed", path, key);
used_keys.insert(key);
auto it = used_keys.find(key);
if (it != used_keys.end() && it->second != value)
throw Exception(ErrorCodes::INCORRECT_DATA, "Path '{}' to file with enabled hive-style partitioning contains duplicated partition key {} with different values, only unique keys are allowed", path, key);
used_keys.insert({key, value});
auto col_name = "_" + key;
while (storage_columns.has(col_name))
col_name = "_" + col_name;
auto col_name = key;
key_values[col_name] = value;
}
return key_values;
}
VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, const ContextPtr & context, const std::string & path, std::optional<FormatSettings> format_settings_)
VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & storage_columns, const ContextPtr & context, const std::string & path, std::optional<FormatSettings> format_settings_)
{
VirtualColumnsDescription desc;
auto add_virtual = [&](const auto & name, const auto & type)
{
if (storage_columns.has(name))
{
if (!context->getSettingsRef().use_hive_partitioning)
return;
if (storage_columns.size() == 1)
throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot use hive partitioning for file {}: it contains only partition columns. Disable use_hive_partitioning setting to read this file", path);
auto local_type = storage_columns.get(name).type;
storage_columns.remove(name);
desc.addEphemeral(name, local_type, "");
return;
}
desc.addEphemeral(name, type, "");
};
@ -171,7 +180,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription
if (context->getSettingsRef().use_hive_partitioning)
{
auto map = parseHivePartitioningKeysAndValues(path, storage_columns);
auto map = parseHivePartitioningKeysAndValues(path);
auto format_settings = format_settings_ ? *format_settings_ : getFormatSettings(context);
for (auto & item : map)
{
@ -244,11 +253,11 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector<String> & paths, const
void addRequestedFileLikeStorageVirtualsToChunk(
Chunk & chunk, const NamesAndTypesList & requested_virtual_columns,
VirtualsForFileLikeStorage virtual_values, ContextPtr context, const ColumnsDescription & columns)
VirtualsForFileLikeStorage virtual_values, ContextPtr context)
{
std::unordered_map<std::string, std::string> hive_map;
if (context->getSettingsRef().use_hive_partitioning)
hive_map = parseHivePartitioningKeysAndValues(virtual_values.path, columns);
hive_map = parseHivePartitioningKeysAndValues(virtual_values.path);
for (const auto & virtual_column : requested_virtual_columns)
{

View File

@ -70,7 +70,7 @@ auto extractSingleValueFromBlock(const Block & block, const String & name)
NameSet getVirtualNamesForFileLikeStorage();
VirtualColumnsDescription getVirtualsForFileLikeStorage(
const ColumnsDescription & storage_columns,
ColumnsDescription & storage_columns,
const ContextPtr & context,
const std::string & sample_path = "",
std::optional<FormatSettings> format_settings_ = std::nullopt);
@ -105,7 +105,7 @@ struct VirtualsForFileLikeStorage
void addRequestedFileLikeStorageVirtualsToChunk(
Chunk & chunk, const NamesAndTypesList & requested_virtual_columns,
VirtualsForFileLikeStorage virtual_values, ContextPtr context, const ColumnsDescription & columns);
VirtualsForFileLikeStorage virtual_values, ContextPtr context);
}
}

View File

@ -60,7 +60,6 @@ MESSAGES_TO_RETRY = [
"is already started to be removing by another replica right now",
# This is from LSan, and it indicates its own internal problem:
"Unable to get registers from thread",
"You can retry",
]
MAX_RETRIES = 3

View File

@ -6,11 +6,17 @@ from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance(
"node1", user_configs=["config/config.xml"], with_zookeeper=True
"node1",
user_configs=["config/config.xml"],
with_zookeeper=True,
macros={"replica": "a", "shard": "shard1"},
)
node2 = cluster.add_instance(
"node2", user_configs=["config/config.xml"], with_zookeeper=True
"node2",
user_configs=["config/config.xml"],
with_zookeeper=True,
macros={"replica": "b", "shard": "shard1"},
)
@ -129,8 +135,8 @@ def test_single_node_normal(started_cluster):
def test_replicated_table_ddl(started_cluster):
node1.query("DROP TABLE IF EXISTS test_stat")
node2.query("DROP TABLE IF EXISTS test_stat")
node1.query("DROP TABLE IF EXISTS test_stat SYNC")
node2.query("DROP TABLE IF EXISTS test_stat SYNC")
node1.query(
"""
@ -183,3 +189,19 @@ def test_replicated_table_ddl(started_cluster):
)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "a", True)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "b", True)
def test_replicated_db(started_cluster):
node1.query("DROP DATABASE IF EXISTS test SYNC")
node2.query("DROP DATABASE IF EXISTS test SYNC")
node1.query(
"CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')"
)
node2.query(
"CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')"
)
node1.query(
"CREATE TABLE test.test_stats (a Int64, b Int64) ENGINE = ReplicatedMergeTree() ORDER BY()"
)
node2.query("ALTER TABLE test.test_stats MODIFY COLUMN b Float64")
node2.query("ALTER TABLE test.test_stats MODIFY STATISTICS b TYPE tdigest")

View File

@ -1513,19 +1513,19 @@ def test_hive_partitioning_with_one_parameter(cluster):
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}',"
f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}",
f" container='cont', blob_path='{path}', format='CSVWithNames', compression='auto', structure='{table_format}') VALUES {values}",
settings={"azure_truncate_on_insert": 1},
)
query = (
f"SELECT column1, column2, _file, _path, _column1 FROM azureBlobStorage(azure_conf2, "
f"SELECT column2, _file, _path, column1 FROM azureBlobStorage(azure_conf2, "
f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', "
f"blob_path='{path}', format='CSV', structure='{table_format}')"
f"blob_path='{path}', format='CSVWithNames', structure='{table_format}')"
)
assert azure_query(
node, query, settings={"use_hive_partitioning": 1}
).splitlines() == [
"Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format(
"Gordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format(
bucket="cont", max_path=path
)
]
@ -1533,14 +1533,14 @@ def test_hive_partitioning_with_one_parameter(cluster):
query = (
f"SELECT column2 FROM azureBlobStorage(azure_conf2, "
f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', "
f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;"
f"blob_path='{path}', format='CSVWithNames', structure='{table_format}');"
)
assert azure_query(
node, query, settings={"use_hive_partitioning": 1}
).splitlines() == ["Gordon"]
def test_hive_partitioning_with_two_parameters(cluster):
def test_hive_partitioning_with_all_parameters(cluster):
# type: (ClickHouseCluster) -> None
node = cluster.instances["node"] # type: ClickHouseInstance
table_format = "column1 String, column2 String"
@ -1551,40 +1551,19 @@ def test_hive_partitioning_with_two_parameters(cluster):
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}',"
f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}",
f" container='cont', blob_path='{path}', format='CSVWithNames', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}",
settings={"azure_truncate_on_insert": 1},
)
query = (
f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, "
f"SELECT column1, column2, _file, _path FROM azureBlobStorage(azure_conf2, "
f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', "
f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;"
f"blob_path='{path}', format='CSVWithNames', structure='{table_format}');"
)
assert azure_query(
node, query, settings={"use_hive_partitioning": 1}
).splitlines() == [
"Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth\tGordon".format(
bucket="cont", max_path=path
)
]
pattern = r"DB::Exception: Cannot use hive partitioning for file"
query = (
f"SELECT column1 FROM azureBlobStorage(azure_conf2, "
f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', "
f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2;"
)
assert azure_query(
node, query, settings={"use_hive_partitioning": 1}
).splitlines() == ["Elizabeth"]
query = (
f"SELECT column1 FROM azureBlobStorage(azure_conf2, "
f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', "
f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2 AND column1=_column1;"
)
assert azure_query(
node, query, settings={"use_hive_partitioning": 1}
).splitlines() == ["Elizabeth"]
with pytest.raises(Exception, match=pattern):
azure_query(node, query, settings={"use_hive_partitioning": 1})
def test_hive_partitioning_without_setting(cluster):
@ -1593,19 +1572,19 @@ def test_hive_partitioning_without_setting(cluster):
table_format = "column1 String, column2 String"
values_1 = f"('Elizabeth', 'Gordon')"
values_2 = f"('Emilia', 'Gregor')"
path = "a/column1=Elizabeth/column2=Gordon/sample.csv"
path = "a/column1=Elizabeth/column2=Gordon/column3=Gordon/sample.csv"
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}',"
f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}",
f" container='cont', blob_path='{path}', format='CSVWithNames', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}",
settings={"azure_truncate_on_insert": 1},
)
query = (
f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, "
f"SELECT column1, column2, _file, _path, column3 FROM azureBlobStorage(azure_conf2, "
f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', "
f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;"
f"blob_path='{path}', format='CSVWithNames', structure='{table_format}');"
)
pattern = re.compile(
r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL

View File

@ -1259,33 +1259,21 @@ def test_respect_object_existence_on_partitioned_write(started_cluster):
def test_hive_partitioning_with_one_parameter(started_cluster):
hdfs_api = started_cluster.hdfs_api
hdfs_api.write_data(f"/column0=Elizabeth/parquet_1", f"Elizabeth\tGordon\n")
assert hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") == f"Elizabeth\tGordon\n"
hdfs_api.write_data(
f"/column0=Elizabeth/file_1", f"column0,column1\nElizabeth,Gordon\n"
)
assert (
hdfs_api.read_data(f"/column0=Elizabeth/file_1")
== f"column0,column1\nElizabeth,Gordon\n"
)
r = node1.query(
"SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')",
"SELECT column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/file_1', 'CSVWithNames')",
settings={"use_hive_partitioning": 1},
)
assert r == f"Elizabeth\n"
def test_hive_partitioning_with_two_parameters(started_cluster):
hdfs_api = started_cluster.hdfs_api
hdfs_api.write_data(
f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n"
)
assert (
hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2")
== f"Elizabeth\tGordon\n"
)
r = node1.query(
"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');",
settings={"use_hive_partitioning": 1},
)
assert r == f"Gordon\n"
def test_hive_partitioning_without_setting(started_cluster):
hdfs_api = started_cluster.hdfs_api
hdfs_api.write_data(
@ -1301,7 +1289,7 @@ def test_hive_partitioning_without_setting(started_cluster):
with pytest.raises(QueryRuntimeException, match=pattern):
node1.query(
f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');",
f"SELECT column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');",
settings={"use_hive_partitioning": 0},
)

View File

@ -26,6 +26,10 @@ while [[ $($CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE mutation_id='00000000
sleep 1
done
while [[ $($CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue WHERE type='ALTER_METADATA' AND database = '$CLICKHOUSE_DATABASE'" 2>&1) ]]; do
sleep 1
done
$CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE table_for_bad_alters;" # Type changed, but we can revert back
$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_bad_alters VALUES(2, 2, 7)"

View File

@ -1635,21 +1635,21 @@ QUERY id: 0
JOIN TREE
TABLE id: 10, alias: __table1, table_name: default.test_table
SELECT sum(float64 + 2) From test_table;
26.5
26.875
SELECT sum(2 + float64) From test_table;
26.5
26.875
SELECT sum(float64 - 2) From test_table;
6.5
6.875
SELECT sum(2 - float64) From test_table;
-6.5
-6.875
SELECT sum(float64) + 2 * count(float64) From test_table;
26.5
26.875
SELECT 2 * count(float64) + sum(float64) From test_table;
26.5
26.875
SELECT sum(float64) - 2 * count(float64) From test_table;
6.5
6.875
SELECT 2 * count(float64) - sum(float64) From test_table;
-6.5
-6.875
EXPLAIN QUERY TREE (SELECT sum(float64 + 2) From test_table);
QUERY id: 0
PROJECTION COLUMNS
@ -2463,25 +2463,25 @@ QUERY id: 0
JOIN TREE
TABLE id: 12, alias: __table1, table_name: default.test_table
SELECT sum(float64 + 2) + sum(float64 + 3) From test_table;
58
58.75
SELECT sum(float64 + 2) - sum(float64 + 3) From test_table;
-5
SELECT sum(float64 - 2) + sum(float64 - 3) From test_table;
8
8.75
SELECT sum(float64 - 2) - sum(float64 - 3) From test_table;
5
SELECT sum(2 - float64) - sum(3 - float64) From test_table;
-5
SELECT (sum(float64) + 2 * count(float64)) + (sum(float64) + 3 * count(float64)) From test_table;
58
58.75
SELECT (sum(float64) + 2 * count(float64)) - (sum(float64) + 3 * count(float64)) From test_table;
-5
SELECT (sum(float64) - 2 * count(float64)) + (sum(float64) - 3 * count(float64)) From test_table;
8
8.75
SELECT (sum(float64) - 2 * count(float64)) - (sum(float64) - 3 * count(float64)) From test_table;
5
SELECT (2 * count(float64) - sum(float64)) + (3 * count(float64) - sum(float64)) From test_table;
-8
-8.75
EXPLAIN QUERY TREE (SELECT sum(float64 + 2) + sum(float64 + 3) From test_table);
QUERY id: 0
PROJECTION COLUMNS

View File

@ -25,11 +25,12 @@ CREATE TABLE test_table
decimal32 Decimal32(5),
) ENGINE=MergeTree ORDER BY uint64;
INSERT INTO test_table VALUES (1, 1.1, 1.11);
INSERT INTO test_table VALUES (2, 2.2, 2.22);
INSERT INTO test_table VALUES (3, 3.3, 3.33);
INSERT INTO test_table VALUES (4, 4.4, 4.44);
INSERT INTO test_table VALUES (5, 5.5, 5.55);
-- Use Float64 numbers divisible by 1/16 (or some other small power of two), so that their sum doesn't depend on summation order.
INSERT INTO test_table VALUES (1, 1.125, 1.11);
INSERT INTO test_table VALUES (2, 2.250, 2.22);
INSERT INTO test_table VALUES (3, 3.375, 3.33);
INSERT INTO test_table VALUES (4, 4.500, 4.44);
INSERT INTO test_table VALUES (5, 5.625, 5.55);
-- { echoOn }
SELECT sum(uint64 + 1 AS i) from test_table where i > 0;

View File

@ -1,4 +1,14 @@
TESTING THE FILE HIVE PARTITIONING
last Elizabeth
Frank Elizabeth
Moreno Elizabeth
Guzman Elizabeth
Stephens Elizabeth
Franklin Elizabeth
Gibson Elizabeth
Greer Elizabeth
Delgado Elizabeth
Cross Elizabeth
first last Elizabeth
Jorge Frank Elizabeth
Hunter Moreno Elizabeth
@ -9,56 +19,36 @@ Stanley Gibson Elizabeth
Eugenia Greer Elizabeth
Jeffery Delgado Elizabeth
Clara Cross Elizabeth
Elizabeth Gordon Elizabeth
Eva Schmidt Elizabeth Schmidt
Samuel Schmidt Elizabeth Schmidt
Eva Schmidt Elizabeth
Samuel Schmidt Elizabeth
Elizabeth Gordon Elizabeth Gordon
Elizabeth Gordon Elizabeth
Elizabeth Gordon Elizabeth Gordon
Elizabeth Gordon Elizabeth
first last Elizabeth
Jorge Frank Elizabeth
Hunter Moreno Elizabeth
Esther Guzman Elizabeth
Dennis Stephens Elizabeth
Nettie Franklin Elizabeth
Stanley Gibson Elizabeth
Eugenia Greer Elizabeth
Jeffery Delgado Elizabeth
Clara Cross Elizabeth
Elizabeth Gordon Elizabeth
last Elizabeth
Frank Elizabeth
Moreno Elizabeth
Guzman Elizabeth
Stephens Elizabeth
Franklin Elizabeth
Gibson Elizabeth
Greer Elizabeth
Delgado Elizabeth
Cross Elizabeth
42 2020-01-01
[1,2,3] 42.42
Array(Int64) LowCardinality(Float64)
101
2070
4081
2070
2070
b
1
1
TESTING THE URL PARTITIONING
first last Elizabeth
Jorge Frank Elizabeth
Hunter Moreno Elizabeth
Esther Guzman Elizabeth
Dennis Stephens Elizabeth
Nettie Franklin Elizabeth
Stanley Gibson Elizabeth
Eugenia Greer Elizabeth
Jeffery Delgado Elizabeth
Clara Cross Elizabeth
Elizabeth Gordon Elizabeth
Eva Schmidt Elizabeth Schmidt
Samuel Schmidt Elizabeth Schmidt
Eva Schmidt Elizabeth
Samuel Schmidt Elizabeth
Elizabeth Gordon Elizabeth Gordon
Elizabeth Gordon Elizabeth
Elizabeth Gordon Elizabeth Gordon
Elizabeth Gordon Elizabeth
last Elizabeth
Frank Elizabeth
Moreno Elizabeth
Guzman Elizabeth
Stephens Elizabeth
Franklin Elizabeth
Gibson Elizabeth
Greer Elizabeth
Delgado Elizabeth
Cross Elizabeth
first last Elizabeth
Jorge Frank Elizabeth
Hunter Moreno Elizabeth
@ -71,6 +61,16 @@ Jeffery Delgado Elizabeth
Clara Cross Elizabeth
1
TESTING THE S3 PARTITIONING
last Elizabeth
Frank Elizabeth
Moreno Elizabeth
Guzman Elizabeth
Stephens Elizabeth
Franklin Elizabeth
Gibson Elizabeth
Greer Elizabeth
Delgado Elizabeth
Cross Elizabeth
first last Elizabeth
Jorge Frank Elizabeth
Hunter Moreno Elizabeth
@ -81,40 +81,35 @@ Stanley Gibson Elizabeth
Eugenia Greer Elizabeth
Jeffery Delgado Elizabeth
Clara Cross Elizabeth
Elizabeth Gordon Elizabeth
Eva Schmidt Elizabeth Schmidt
Samuel Schmidt Elizabeth Schmidt
Eva Schmidt Elizabeth
Samuel Schmidt Elizabeth
Elizabeth Gordon Elizabeth Gordon
Elizabeth Gordon Elizabeth
Elizabeth Gordon Elizabeth Gordon
Elizabeth Gordon Elizabeth
first last Elizabeth
Jorge Frank Elizabeth
Hunter Moreno Elizabeth
Esther Guzman Elizabeth
Dennis Stephens Elizabeth
Nettie Franklin Elizabeth
Stanley Gibson Elizabeth
Eugenia Greer Elizabeth
Jeffery Delgado Elizabeth
Clara Cross Elizabeth
Elizabeth Gordon Elizabeth
last Elizabeth
Frank Elizabeth
Moreno Elizabeth
Guzman Elizabeth
Stephens Elizabeth
Franklin Elizabeth
Gibson Elizabeth
Greer Elizabeth
Delgado Elizabeth
Cross Elizabeth
OK
TESTING THE S3CLUSTER PARTITIONING
first last Elizabeth
Jorge Frank Elizabeth
Hunter Moreno Elizabeth
Esther Guzman Elizabeth
Dennis Stephens Elizabeth
Nettie Franklin Elizabeth
Stanley Gibson Elizabeth
Eugenia Greer Elizabeth
Jeffery Delgado Elizabeth
Clara Cross Elizabeth
Elizabeth Gordon Elizabeth
Eva Schmidt Elizabeth Schmidt
Samuel Schmidt Elizabeth Schmidt
Eva Schmidt Elizabeth
Samuel Schmidt Elizabeth
last Elizabeth
Frank Elizabeth
Moreno Elizabeth
Guzman Elizabeth
Stephens Elizabeth
Franklin Elizabeth
Gibson Elizabeth
Greer Elizabeth
Delgado Elizabeth
Cross Elizabeth
last Elizabeth
Frank Elizabeth
Moreno Elizabeth
Guzman Elizabeth
Stephens Elizabeth
Franklin Elizabeth
Gibson Elizabeth
Greer Elizabeth
Delgado Elizabeth
Cross Elizabeth

View File

@ -11,48 +11,34 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'"
$CLICKHOUSE_LOCAL -n -q """
set use_hive_partitioning = 1;
SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;
SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;
SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0;
SELECT *, non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;
SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' LIMIT 10;
SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1;
SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1;
SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1;
SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1;
SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1;
SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1;
SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1;
SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1;
SELECT *, _non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;
SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;
SELECT _number, _date FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') LIMIT 1;
SELECT _array, _float FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1;
SELECT toTypeName(_array), toTypeName(_float) FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1;
SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') WHERE _number = 42;
SELECT number, date FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') LIMIT 1;
SELECT array, float FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1;
SELECT toTypeName(array), toTypeName(float) FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1;
SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') WHERE number = 42;
"""
$CLICKHOUSE_LOCAL -n -q """
set use_hive_partitioning = 1;
SELECT _identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') LIMIT 2;
SELECT __identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') LIMIT 2;
SELECT identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') LIMIT 2;
SELECT a FROM file('$CURDIR/data_hive/partitioning/a=b/a=b/sample.parquet') LIMIT 1;
"""
$CLICKHOUSE_LOCAL -n -q """
set use_hive_partitioning = 1;
SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth/sample.parquet') LIMIT 10;
SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth1/sample.parquet') LIMIT 10;
""" 2>&1 | grep -c "INCORRECT_DATA"
$CLICKHOUSE_LOCAL -n -q """
set use_hive_partitioning = 0;
SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;
SELECT *, non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;
""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER"
@ -62,23 +48,9 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'"
$CLICKHOUSE_LOCAL -n -q """
set use_hive_partitioning = 1;
SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;
SELECT *, column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;
SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0;
SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1;
SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1;
SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1;
SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1;
SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1;
SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1;
SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1;
SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1;
SELECT *, _non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;"""
SELECT *, non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;"""
$CLICKHOUSE_LOCAL -n -q """
set use_hive_partitioning = 0;
@ -93,24 +65,10 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'"
$CLICKHOUSE_CLIENT -n -q """
set use_hive_partitioning = 1;
SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;
SELECT *, column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;
SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0;
SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1;
SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1;
SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1;
SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1;
SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1;
SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1;
SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1;
SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1;
SELECT *, _non_existing_column FROM s3('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;
SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = _column0;
SELECT *, non_existing_column FROM s3('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;
SELECT *, column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' LIMIT 10;
"""
$CLICKHOUSE_CLIENT -n -q """
@ -124,13 +82,7 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3CLUSTER PARTITIONING'"
$CLICKHOUSE_CLIENT -n -q """
set use_hive_partitioning = 1;
SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;
SELECT *, column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;
SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0;
SELECT *, _column0, _column1 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1;
SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1;
SELECT *, _column0, _column1 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1;
SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1;
SELECT *, column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = 'Elizabeth' LIMIT 10;
"""

View File

@ -0,0 +1,46 @@
#!/usr/bin/env bash
# Tags: no-random-settings, no-random-merge-tree-settings
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --query "
DROP TABLE IF EXISTS t_mutate_skip_part;
CREATE TABLE t_mutate_skip_part (key UInt64, id UInt64, v1 UInt64, v2 UInt64)
ENGINE = MergeTree ORDER BY id PARTITION BY key
SETTINGS min_bytes_for_wide_part = 0;
INSERT INTO t_mutate_skip_part SELECT 1, number, number, number FROM numbers(10000);
INSERT INTO t_mutate_skip_part SELECT 2, number, number, number FROM numbers(10000);
SET mutations_sync = 2;
ALTER TABLE t_mutate_skip_part UPDATE v1 = 1000 WHERE key = 1;
ALTER TABLE t_mutate_skip_part DELETE WHERE key = 2 AND v2 % 10 = 0;
"
# Mutation query may return before the entry is added to part log.
# So, we may have to retry the flush of logs until all entries are actually flushed.
for _ in {1..10}; do
${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS"
res=$(${CLICKHOUSE_CLIENT} --query "SELECT count() FROM system.part_log WHERE database = currentDatabase() AND table = 't_mutate_skip_part' AND event_type = 'MutatePart'")
if [[ $res -eq 4 ]]; then
break
fi
sleep 2.0
done
${CLICKHOUSE_CLIENT} --query "
SYSTEM FLUSH LOGS;
-- If part is skipped in mutation and hardlinked then read_rows must be 0.
SELECT part_name, read_rows
FROM system.part_log
WHERE database = currentDatabase() AND table = 't_mutate_skip_part' AND event_type = 'MutatePart'
ORDER BY part_name;
DROP TABLE IF EXISTS t_mutate_skip_part;
"

View File

@ -1,21 +0,0 @@
DROP TABLE IF EXISTS t_mutate_skip_part;
CREATE TABLE t_mutate_skip_part (key UInt64, id UInt64, v1 UInt64, v2 UInt64) ENGINE = MergeTree ORDER BY id PARTITION BY key;
INSERT INTO t_mutate_skip_part SELECT 1, number, number, number FROM numbers(10000);
INSERT INTO t_mutate_skip_part SELECT 2, number, number, number FROM numbers(10000);
SET mutations_sync = 2;
ALTER TABLE t_mutate_skip_part UPDATE v1 = 1000 WHERE key = 1;
ALTER TABLE t_mutate_skip_part DELETE WHERE key = 2 AND v2 % 10 = 0;
SYSTEM FLUSH LOGS;
-- If part is skipped in mutation and hardlinked then read_rows must be 0.
SELECT part_name, read_rows
FROM system.part_log
WHERE database = currentDatabase() AND table = 't_mutate_skip_part' AND event_type = 'MutatePart'
ORDER BY part_name;
DROP TABLE IF EXISTS t_mutate_skip_part;

View File

@ -0,0 +1,4 @@
str
42
42
42

View File

@ -0,0 +1,9 @@
set allow_experimental_dynamic_type=1;
set allow_experimental_json_type=1;
set allow_experimental_analyzer=1;
select d.String from (select 'str'::Dynamic as d);
select json.a from (select '{"a" : 42}'::JSON as json);
select json.a from (select '{"a" : 42}'::JSON(a UInt32) as json);
select json.a.:Int64 from (select materialize('{"a" : 42}')::JSON as json);

View File

@ -0,0 +1,8 @@
2 {"foo2":"bar"} 1
3 {"foo2":"bar"} 1
2 {"foo2":"baz"} 2
3 {"foo2":"bar"} 1
2 {"foo2":"bar"} 1
3 {"foo2":"bar"} 1
2 {"foo2":"baz"} 2
3 {"foo2":"bar"} 1

View File

@ -0,0 +1,33 @@
SET allow_experimental_json_type = 1;
DROP TABLE IF EXISTS test_new_json_type;
CREATE TABLE test_new_json_type(id UInt32, data JSON, version UInt64) ENGINE=ReplacingMergeTree(version) ORDER BY id;
INSERT INTO test_new_json_type format JSONEachRow
{"id":1,"data":{"foo1":"bar"},"version":1}
{"id":2,"data":{"foo2":"bar"},"version":1}
{"id":3,"data":{"foo2":"bar"},"version":1}
;
SELECT * FROM test_new_json_type FINAL WHERE data.foo2 is not null ORDER BY id;
INSERT INTO test_new_json_type SELECT id, '{"foo2":"baz"}' AS _data, version+1 AS _version FROM test_new_json_type where id=2;
SELECT * FROM test_new_json_type FINAL WHERE data.foo2 is not null ORDER BY id;
DROP TABLE test_new_json_type;
CREATE TABLE test_new_json_type(id Nullable(UInt32), data JSON, version UInt64) ENGINE=ReplacingMergeTree(version) ORDER BY id settings allow_nullable_key=1;
INSERT INTO test_new_json_type format JSONEachRow
{"id":1,"data":{"foo1":"bar"},"version":1}
{"id":2,"data":{"foo2":"bar"},"version":1}
{"id":3,"data":{"foo2":"bar"},"version":1}
;
SELECT * FROM test_new_json_type FINAL WHERE data.foo2 is not null ORDER BY id;
INSERT INTO test_new_json_type SELECT id, '{"foo2":"baz"}' AS _data, version+1 AS _version FROM test_new_json_type where id=2;
SELECT * FROM test_new_json_type FINAL PREWHERE data.foo2 IS NOT NULL WHERE data.foo2 IS NOT NULL ORDER BY id ASC NULLS FIRST;
DROP TABLE test_new_json_type;

View File

@ -1,3 +1,6 @@
-- There is a bug in old analyzer with currentDatabase() and distributed query.
SET enable_analyzer = 1;
DROP TABLE IF EXISTS t_local_1;
DROP TABLE IF EXISTS t_local_2;
DROP TABLE IF EXISTS t_merge;
@ -10,7 +13,7 @@ INSERT INTO t_local_1 VALUES (1);
INSERT INTO t_local_2 VALUES (2);
CREATE TABLE t_merge AS t_local_1 ENGINE = Merge(currentDatabase(), '^(t_local_1|t_local_2)$');
CREATE TABLE t_distr AS t_local_1 engine=Distributed('test_shard_localhost', currentDatabase(), t_merge, rand());
CREATE TABLE t_distr AS t_local_1 ENGINE = Distributed('test_shard_localhost', currentDatabase(), t_merge, rand());
SELECT a, _table FROM t_merge ORDER BY a;
SELECT a, _table FROM t_distr ORDER BY a;

View File

@ -0,0 +1,3 @@
{"foo1":"bar"} {"foo1":"bar"}
{"foo2":"bar"} {"foo2":"bar"}
{"foo2":"bar"} {"foo2":"bar"}

View File

@ -0,0 +1,22 @@
SET allow_experimental_json_type=1;
DROP TABLE IF EXISTS test_new_json_type;
CREATE TABLE test_new_json_type(id UInt32, data JSON, version UInt64) ENGINE=ReplacingMergeTree(version) ORDER BY id;
INSERT INTO test_new_json_type format JSONEachRow
{"id":1,"data":{"foo1":"bar"},"version":1}
{"id":2,"data":{"foo2":"bar"},"version":1}
{"id":3,"data":{"foo2":"bar"},"version":1}
;
SELECT
a.data
, b.data
FROM
test_new_json_type a
JOIN test_new_json_type b
ON a.id = b.id;
DROP TABLE test_new_json_type;