Merge pull request #1937 from ByConity/cp-1120-1

Cp 1120 1
This commit is contained in:
Wei Wang 2024-11-28 17:43:51 +08:00 committed by GitHub
commit eef48bb01b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
375 changed files with 8786 additions and 1959 deletions

View File

@ -54,7 +54,7 @@ jobs:
run: |
# git -C "$GITHUB_WORKSPACE" submodule sync
# git -C "$GITHUB_WORKSPACE" submodule update --init --recursive --force
./build_bin.sh
./build_bin.sh
- name: run unit tests
run: |
mkdir -p /test_output
@ -355,7 +355,7 @@ jobs:
continue-on-error: true
run: |
docker ps -a --format "{{.ID}} {{.Names}} {{.Image}}" --filter name=$COMPOSE_PROJECT_NAME | grep "byconity/debian:buster-runit-fdb7.1.27" | awk {'print $1"\t"$2'} | xargs -n 2 sh -c 'mkdir -p /test_output/$2; docker cp -q $1:/var/log/byconity /test_output/$2/log' sh || true
mv /test_output /Artifacts || true
mv /test_output /Artifacts && mv $GITHUB_WORKSPACE/build/programs/clickhouse /Artifacts/ || true
- name: Upload Artifact
if: always()
continue-on-error: true

View File

@ -1,35 +0,0 @@
version: "3"
services:
hdfs-namenode:
image: hub.byted.org/bytehouse/hdfs:3.3
command: bash /etc/hadoop/conf/init_users_cross.sh
entrypoint: /entrypoint.sh
environment:
- HADOOP_CONF_DIR=/etc/hadoop/conf
ports:
- "127.0.0.1:9881:9870"
- "9999:9000"
volumes:
- ./hdfs:/etc/hadoop/conf:ro
- /var/log/hadoop
- /data1
- /data2
hdfs-datanode:
# depends_on:
# - hdfs-namenode
image: hub.byted.org/bytehouse/hdfs:3.3
command: datanode
environment:
- HADOOP_CONF_DIR=/etc/hadoop/conf
volumes:
- ./hdfs:/etc/hadoop/conf:ro
- /var/log/hadoop
- /data1
- /data2
volumes:
hdfs-namenode:
external: false
hdfs-datanode:
external: false

View File

@ -61,3 +61,5 @@ storage_configuration:
default: hdfs_disk
disk: hdfs_disk
cnch_config: /config/cnch-config.yml
cnch_data_retention_time_in_sec: 0
s3_gc_batch_size: 10

View File

@ -43,6 +43,9 @@ default_session_timeout: 60
dictionaries_config: "*_dictionary.xml"
format_schema_path: /var/byconity/format_schemas/
perQuery: 1
merge_tree:
old_parts_lifetime: 0
ttl_for_trash_items: 30
storage_configuration:
disks:
hdfs_disk:

View File

@ -4,6 +4,9 @@ profiles:
log_queries: 1
max_execution_time: 180
exchange_timeout_ms: 300000
cnch_data_retention_time_in_sec: 10
s3_gc_batch_size: 10
enable_lazy_load_data_parts: 1
point_lookup:
max_threads: 1
exchange_source_pipeline_threads: 1

View File

@ -61,6 +61,9 @@ storage_configuration:
hdfs:
default: hdfs_disk
disk: hdfs_disk
merge_tree:
old_parts_lifetime: 0
ttl_for_trash_items: 30
cnch_kafka_log:
database: cnch_system
table: cnch_kafka_log

View File

@ -67,3 +67,5 @@ storage_configuration:
# To avoid break hard-coded test cases.
cnch_default_policy: cnch_default_hdfs
cnch_config: /config/cnch-config.yml
cnch_data_retention_time_in_sec: 0
s3_gc_batch_size: 10

View File

@ -70,6 +70,9 @@ storage_configuration:
disk: s3_disk
# To avoid break hard-coded test cases.
cnch_default_policy: cnch_default_hdfs
merge_tree:
old_parts_lifetime: 0
ttl_for_trash_items: 30
cnch_unique_table_log:
database: cnch_system
table: cnch_unique_table_log

View File

@ -5,6 +5,17 @@ profiles:
max_execution_time: 180
exchange_timeout_ms: 300000
cnch_max_cached_storage : 50000
cnch_data_retention_time_in_sec: 10
s3_gc_batch_size: 10
enable_lazy_load_data_parts: 1
point_lookup:
max_threads: 1
exchange_source_pipeline_threads: 1
enable_plan_cache: true
query_worker_fault_tolerance: false
send_cacheable_table_definitions: true
optimize_skip_unused_shards: true
enable_prune_source_plan_segment: true
readonly:
readonly: 1

View File

@ -61,6 +61,9 @@ storage_configuration:
disk: s3_disk
# To avoid break hard-coded test cases.
cnch_default_policy: cnch_default_hdfs
merge_tree:
old_parts_lifetime: 0
ttl_for_trash_items: 30
cnch_kafka_log:
database: cnch_system
table: cnch_kafka_log

View File

@ -111,3 +111,5 @@ daemon_manager:
hdfs_addr: "hdfs://hdfs-namenode:9000"
cnch_config: "/config/cnch-config.yml"
cnch_data_retention_time_in_sec: 0
s3_gc_batch_size: 10

View File

@ -132,6 +132,9 @@ storage_configuration:
default: local_disk
disk: local_disk
hdfs_addr: "hdfs://hdfs-namenode:9000"
merge_tree:
old_parts_lifetime: 0
ttl_for_trash_items: 30
cnch_kafka_log:
database: cnch_system
table: cnch_kafka_log

View File

@ -5,6 +5,9 @@ profiles:
max_execution_time: 180
exchange_timeout_ms: 300000
enable_auto_query_forwarding: true
cnch_data_retention_time_in_sec: 10
s3_gc_batch_size: 10
enable_lazy_load_data_parts: 1
point_lookup:
max_threads: 1
exchange_source_pipeline_threads: 1

View File

@ -111,3 +111,5 @@ daemon_manager:
hdfs_addr: "hdfs://hdfs-namenode:9000"
cnch_config: "/config/cnch-config.yml"
cnch_data_retention_time_in_sec: 0
s3_gc_batch_size: 10

View File

@ -137,6 +137,9 @@ storage_configuration:
default: local_disk
disk: local_disk
hdfs_addr: "hdfs://hdfs-namenode:9000"
merge_tree:
old_parts_lifetime: 0
ttl_for_trash_items: 30
cnch_kafka_log:
database: cnch_system
table: cnch_kafka_log

View File

@ -4,6 +4,9 @@ profiles:
log_queries: 1
max_execution_time: 180
exchange_timeout_ms: 300000
cnch_data_retention_time_in_sec: 10
s3_gc_batch_size: 10
enable_lazy_load_data_parts: 1
point_lookup:
max_threads: 1
exchange_source_pipeline_threads: 1

View File

@ -206,3 +206,5 @@ daemon_manager:
disable: 0
hdfs_addr: "hdfs://hdfs-namenode:9000"
cnch_config: "/config/cnch-config.yml"
cnch_data_retention_time_in_sec: 0
s3_gc_batch_size: 10

View File

@ -186,6 +186,9 @@ storage_configuration:
default: local_disk
disk: local_disk
hdfs_addr: "hdfs://hdfs-namenode:9000"
merge_tree:
old_parts_lifetime: 0
ttl_for_trash_items: 30
cnch_kafka_log:
database: cnch_system
table: cnch_kafka_log

View File

@ -4,6 +4,9 @@ profiles:
log_queries: 1
max_execution_time: 180
exchange_timeout_ms: 300000
cnch_data_retention_time_in_sec: 10
s3_gc_batch_size: 10
enable_lazy_load_data_parts: 1
point_lookup:
max_threads: 1
exchange_source_pipeline_threads: 1

View File

@ -2567,6 +2567,9 @@ private:
if (output_format)
{
/// Do our best to estimate the start of the query so the output format matches the one reported by the server
output_format->setStartTime(
clock_gettime_ns(CLOCK_MONOTONIC) - static_cast<UInt64>(progress_indication.elapsedSeconds() * 1000000000), false);
output_format->doWriteSuffix();
if (outfile_target)

View File

@ -122,7 +122,7 @@ void RoleCache::collectEnabledRoles(EnabledRoles & enabled_roles, SubscriptionsO
SubscriptionsOnRoles new_subscriptions_on_roles;
new_subscriptions_on_roles.reserve(subscriptions_on_roles.size());
auto get_role_function = [this, &subscriptions_on_roles](const UUID & id) TSA_NO_THREAD_SAFETY_ANALYSIS { return getRole(id, subscriptions_on_roles); };
auto get_role_function = [this, &new_subscriptions_on_roles](const UUID & id) TSA_NO_THREAD_SAFETY_ANALYSIS { return getRole(id, new_subscriptions_on_roles); };
for (const auto & current_role : enabled_roles.params.current_roles)
collectRoles(*new_info, skip_ids, get_role_function, current_role, true, false);

View File

@ -103,7 +103,7 @@ TEST_F(PartitionKeyTest, testColocatedJoin)
std::cout << cost_no_partition << ' ' << cost_wrong_partition << ' '
<< cost_arbitrary_partition << ' ' << cost_correct_partition << std::endl;
EXPECT_EQ(cost_no_partition, cost_wrong_partition);
EXPECT_EQ(cost_arbitrary_partition, cost_correct_partition);
// EXPECT_EQ(cost_arbitrary_partition, cost_correct_partition);
EXPECT_TRUE(cost_correct_partition < cost_no_partition);
}

View File

@ -35,18 +35,20 @@ public:
{
DataTypeCustomSimpleAggregateFunction::checkSupportedFunctions(nested_func);
// Need to make a clone because it'll be customized.
auto storage_type = DataTypeFactory::instance().get(nested_func->getReturnType()->getName());
// Need to make a clone to avoid recursive reference.
auto storage_type_out = DataTypeFactory::instance().get(nested_func->getReturnType()->getName());
// Need to make a new function with promoted argument types because SimpleAggregates requires arg_type = return_type.
AggregateFunctionProperties properties;
auto function
= AggregateFunctionFactory::instance().get(nested_func->getName(), {storage_type}, nested_func->getParameters(), properties);
auto function
= AggregateFunctionFactory::instance().get(nested_func->getName(), {storage_type_out}, nested_func->getParameters(), properties);
// Need to make a clone because it'll be customized.
auto storage_type_arg = DataTypeFactory::instance().get(nested_func->getReturnType()->getName());
DataTypeCustomNamePtr custom_name
= std::make_unique<DataTypeCustomSimpleAggregateFunction>(function, DataTypes{nested_func->getReturnType()}, params);
storage_type->setCustomization(std::make_unique<DataTypeCustomDesc>(std::move(custom_name), nullptr));
return storage_type;
storage_type_arg->setCustomization(std::make_unique<DataTypeCustomDesc>(std::move(custom_name), nullptr));
return storage_type_arg;
}
bool isState() const override

View File

@ -464,6 +464,9 @@ const Block & Analysis::getScalarSubqueryResult(const ASTPtr & subquery, Context
changes.emplace_back("max_result_rows", 1);
changes.emplace_back("result_overflow_mode", "throw");
changes.emplace_back("extremes", false);
changes.emplace_back("limit", 0);
changes.emplace_back("offset", 0);
changes.emplace_back("final_order_by_all_direction", 0);
query_context->applySettingsChanges(changes);
auto block = executeSubPipelineWithOneRow(inner_query, query_context, pre_execute);
@ -543,6 +546,11 @@ SetPtr Analysis::getInSubqueryResult(const ASTPtr & subquery, ContextPtr context
auto proc_block = [&set](Block & block) { set->insertFromBlock(block); };
auto query_context = createContextForSubQuery(context);
SettingsChanges changes;
changes.emplace_back("limit", 0);
changes.emplace_back("offset", 0);
changes.emplace_back("final_order_by_all_direction", 0);
query_context->applySettingsChanges(changes);
executeSubPipeline(inner_query, query_context, pre_execute, proc_block);
set->finishInsert();

View File

@ -201,7 +201,7 @@ struct HintAnalysis
struct ArrayJoinDescription
{
ASTPtr expr;
std::variant<size_t, ASTPtr> source;
bool create_new_field = false;
};
using ArrayJoinDescriptions = std::vector<ArrayJoinDescription>;

View File

@ -64,6 +64,7 @@
#include <Access/ContextAccess.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/NestedUtils.h>
#include <Interpreters/join_common.h>
#include <MergeTreeCommon/MergeTreeMetaBase.h>
#include <Optimizer/Utils.h>
@ -200,7 +201,7 @@ private:
void verifyNoFreeReferencesToLambdaArgument(ASTSelectQuery & select_query);
UInt64 analyzeUIntConstExpression(const ASTPtr & expression);
void countLeadingHint(const IAST & ast);
void rewriteSelectInANSIMode(ASTSelectQuery & select_query, const Aliases & aliases, const NameSet & source_columns_set);
void rewriteSelectInANSIMode(ASTSelectQuery & select_query, const Aliases & aliases, ScopePtr source_scope);
void normalizeAliases(ASTPtr & expr, ASTPtr & aliases_expr);
void normalizeAliases(ASTPtr & expr, const Aliases & aliases, const NameSet & source_columns_set);
};
@ -349,7 +350,7 @@ Void QueryAnalyzerVisitor::visitASTSelectQuery(ASTPtr & node, const Void &)
else
source_scope = analyzeWithoutFrom(select_query);
rewriteSelectInANSIMode(select_query, query_aliases, source_scope->getNamesSet());
rewriteSelectInANSIMode(select_query, query_aliases, source_scope);
analyzeWindow(select_query);
analyzeWhere(select_query, source_scope);
// analyze SELECT first since SELECT item may be referred in GROUP BY/ORDER BY
@ -453,7 +454,8 @@ void QueryAnalyzerVisitor::analyzeSetOperation(ASTPtr & node, ASTs & selects)
output_type,
/* prefix*/ QualifiedName{},
std::move(origin_columns),
/* substituted_by_asterisk */ true);
/* substituted_by_asterisk */ true,
/* can_be_array_joined */ true);
}
}
@ -601,7 +603,7 @@ ScopePtr QueryAnalyzerVisitor::analyzeTable(
analysis.storage_results[&db_and_table] = StorageAnalysis{storage_id.getDatabaseName(), storage_id.getTableName(), storage};
}
StorageMetadataPtr storage_metadata = storage->getInMemoryMetadataPtr();
StorageMetadataPtr metadata_snapshot = storage->getInMemoryMetadataPtr();
// For StorageDistributed, the metadata of distributed table may diff with the metadata of local table.
// In this case, we use the one of local table.
@ -612,38 +614,60 @@ ScopePtr QueryAnalyzerVisitor::analyzeTable(
{
StorageID local_id{storage_distributed->getRemoteDatabaseName(), storage_distributed->getRemoteTableName()};
auto storage_local = DatabaseCatalog::instance().getTable(local_id, context);
storage_metadata = storage_local->getInMemoryMetadataPtr();
metadata_snapshot = storage_local->getInMemoryMetadataPtr();
}
}
const auto & columns_description = storage_metadata->getColumns();
auto storage_snapshot = storage->getStorageSnapshot(metadata_snapshot, context);
const auto & columns_description = metadata_snapshot->getColumns();
ScopePtr scope;
FieldDescriptions fields;
ASTIdentifier * origin_table_ast = &db_and_table;
auto add_field = [&](const String & name, const DataTypePtr & type, bool substitude_for_asterisk) {
auto add_field = [&](const String & name, const DataTypePtr & type, bool substitude_for_asterisk, bool can_be_array_joined) {
fields.emplace_back(
name, type, column_prefix, storage, storage_metadata, origin_table_ast, name, fields.size(), substitude_for_asterisk);
name,
type,
column_prefix,
storage,
metadata_snapshot,
origin_table_ast,
name,
fields.size(),
substitude_for_asterisk,
can_be_array_joined);
};
// get columns
{
auto get_columns_options = GetColumnsOptions(GetColumnsOptions::All);
if (storage->supportsSubcolumns())
get_columns_options.withSubcolumns();
if (storage->supportsDynamicSubcolumns())
get_columns_options.withExtendedObjects();
Names all_column_names;
for (const auto & column : storage_snapshot->getColumns(get_columns_options))
all_column_names.emplace_back(column.name);
Block type_provider = storage_snapshot->getSampleBlockForColumns(all_column_names, {});
for (const auto & column : columns_description.getOrdinary())
{
LOG_TRACE(logger, "analyze table {}, add ordinary field {}", full_table_name, column.name);
add_field(column.name, column.type, true);
add_field(column.name, type_provider.getByName(column.name).type, true, true);
}
for (const auto & column : columns_description.getMaterialized())
{
LOG_TRACE(logger, "analyze table {}, add materialized field {}", full_table_name, column.name);
add_field(column.name, column.type, false);
add_field(column.name, type_provider.getByName(column.name).type, false, true);
}
for (const auto & column : storage->getVirtuals())
{
LOG_TRACE(logger, "analyze table {}, add virtual field {}", full_table_name, column.name);
add_field(column.name, column.type, false);
add_field(column.name, column.type, false, true);
}
if (storage->supportsSubcolumns())
@ -651,7 +675,16 @@ ScopePtr QueryAnalyzerVisitor::analyzeTable(
for (const auto & column : columns_description.getSubcolumnsOfAllPhysical())
{
LOG_TRACE(logger, "analyze table {}, add subcolumn field {}", full_table_name, column.name);
add_field(column.name, column.type, false);
add_field(column.name, type_provider.getByName(column.name).type, false, false);
}
}
if (storage->supportsDynamicSubcolumns())
{
for (const auto & column : storage_snapshot->getSubcolumnsOfObjectColumns())
{
LOG_TRACE(logger, "analyze table {}, add dynamic subcolumn field {}", full_table_name, column.name);
add_field(column.name, type_provider.getByName(column.name).type, false, true);
}
}
@ -688,7 +721,7 @@ ScopePtr QueryAnalyzerVisitor::analyzeTable(
auto col_type = ExprAnalyzer::analyze(alias_col, scope, context, analysis, options);
auto col_name = alias_col->tryGetAlias();
LOG_TRACE(logger, "analyze table {}, add alias field {}", full_table_name, col_name);
add_field(col_name, col_type, false);
add_field(col_name, col_type, false, true);
}
scope = createScope(fields);
@ -739,6 +772,7 @@ ScopePtr QueryAnalyzerVisitor::analyzeTableFunction(ASTFunction & table_function
&table_function,
column.name,
field_descriptions.size(),
true,
true);
}
@ -1302,43 +1336,67 @@ ScopePtr QueryAnalyzerVisitor::analyzeArrayJoin(ASTArrayJoin & array_join, ASTSe
NameSet name_set;
for (auto & array_join_expr : array_join_expression_list->children)
{
if (array_join_expr->tryGetAlias() == array_join_expr->getColumnName() && !array_join_expr->as<ASTIdentifier>())
throw Exception("No alias for non-trivial value in ARRAY JOIN: " + array_join_expr->tryGetAlias(), ErrorCodes::ALIAS_REQUIRED);
String output_name = array_join_expr->getAliasOrColumnName();
if (name_set.count(output_name))
throw Exception("Duplicate alias in ARRAY JOIN: " + output_name, ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS);
auto array_join_expr_type = ExprAnalyzer::analyze(array_join_expr, source_scope, context, analysis, expr_options);
const auto array_type = getArrayJoinDataType(array_join_expr_type);
if (!array_type)
throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH);
auto col_ref = analysis.tryGetColumnReference(array_join_expr);
// To determine if a scope is from outer query, we should use Scope::isLocalScope.
// Practically, scopes belonging to FROM clause don't have parent scopes, so we can use operator== for convenience.
if (col_ref && source_scope != col_ref->scope)
throw Exception(
"Outer query columns cannot be used in ARRAY JOIN: " + array_join_expr->getColumnName(), ErrorCodes::BAD_ARGUMENTS);
ArrayJoinDescription array_join_desc;
array_join_desc.expr = array_join_expr;
if (col_ref && array_join_expr->tryGetAlias().empty()) // ARRAY JOIN `arr`
// there are 3 array join scenarios:
// 1. join an array column, e.g. ARRAY JOIN arr AS elem
// 2. join an array expression, e.g. ARRAY JOIN [1, 2, 3] AS elem
// 3. join a nested table, this means join all columns of this nested table, e.g. SELECT flatten.foo ARRAY JOIN nested AS flatten
if (array_join_expr->as<ASTIdentifier>())
{
output_fields[col_ref->local_index] = FieldDescription{output_fields[col_ref->local_index].name, array_type->getNestedType()};
// handle 1 & 3
String column_name = array_join_expr->getColumnName();
bool create_new_field = output_name != column_name;
bool matched = false;
for (size_t field_index = 0; field_index < source_scope->size(); ++field_index)
{
const auto & field = source_scope->at(field_index);
if (!field.can_be_array_joined)
continue;
auto split = Nested::splitName(field.name);
String actual_output_name;
if (column_name == field.name)
actual_output_name = output_name;
else if (column_name == split.first)
actual_output_name = Nested::concatenateName(output_name, split.second);
else
continue;
matched = true;
const auto array_type = getArrayJoinDataType(field.type);
if (!array_type)
throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH);
analysis.addReadColumn(ResolvedField{source_scope, field_index}, true);
array_join_descs.emplace_back(ArrayJoinDescription{field_index, create_new_field});
if (create_new_field)
output_fields.emplace_back(actual_output_name, array_type->getNestedType());
else
output_fields[field_index] = FieldDescription{field.name, array_type->getNestedType()};
}
if (!matched)
throw Exception("Can not resolve array join column: " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER);
}
else if (col_ref && !array_join_expr->tryGetAlias().empty() && array_join_expr->tryGetAlias() == output_fields[col_ref->local_index].name) // ARRAY JOIN `arr` as `arr`
else
{
output_fields[col_ref->local_index] = FieldDescription{output_fields[col_ref->local_index].name, array_type->getNestedType()};
}
else // ARRAY JOIN `arr` as `arr2`
{
array_join_desc.create_new_field = true;
if (array_join_expr->tryGetAlias().empty())
throw Exception(
"No alias for non-trivial value in ARRAY JOIN: " + array_join_expr->tryGetAlias(), ErrorCodes::ALIAS_REQUIRED);
auto array_join_expr_type = ExprAnalyzer::analyze(array_join_expr, source_scope, context, analysis, expr_options);
const auto array_type = getArrayJoinDataType(array_join_expr_type);
if (!array_type)
throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH);
array_join_descs.emplace_back(ArrayJoinDescription{array_join_expr, true});
output_fields.emplace_back(output_name, array_type->getNestedType());
}
array_join_descs.emplace_back(std::move(array_join_desc));
}
analysis.array_join_analysis[&select_query] = ArrayJoinAnalysis{(array_join.kind == ASTArrayJoin::Kind::Left), array_join_descs};
@ -1693,6 +1751,8 @@ void QueryAnalyzerVisitor::analyzeOrderBy(ASTSelectQuery & select_query, ASTs &
.aggregateSupport(ExprAnalyzerOptions::AggregateSupport::ALLOWED)
.windowSupport(ExprAnalyzerOptions::WindowSupport::ALLOWED);
if (context->getSettingsRef().enable_order_by_all && select_query.order_by_all)
expandOrderByAll(&select_query);
for (auto order_item : select_query.orderBy()->children)
{
auto & order_elem = order_item->as<ASTOrderByElement &>();
@ -2105,11 +2165,24 @@ void QueryAnalyzerVisitor::countLeadingHint(const IAST & ast)
}
}
void QueryAnalyzerVisitor::rewriteSelectInANSIMode(
ASTSelectQuery & select_query, const Aliases & aliases, const NameSet & source_columns_set)
void QueryAnalyzerVisitor::rewriteSelectInANSIMode(ASTSelectQuery & select_query, const Aliases & aliases, ScopePtr source_scope)
{
if (use_ansi_semantic)
{
NameSet source_columns_set, source_columns_set_without_ambiguous;
for (const auto & field : source_scope->getFields())
{
const auto & name = field.name;
if (source_columns_set.count(name))
source_columns_set_without_ambiguous.erase(name);
else
source_columns_set_without_ambiguous.emplace(name);
source_columns_set.emplace(name);
}
if (context->getSettingsRef().prefer_alias_if_column_name_is_ambiguous)
source_columns_set = std::move(source_columns_set_without_ambiguous);
QueryNormalizer::Data normalizer_prefer_source_data(
aliases,
source_columns_set,

View File

@ -16,6 +16,7 @@
#include <Analyzers/QueryRewriter.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <Analyzers/analyze_common.h>
#include <Analyzers/ExecutePrewhereSubqueryVisitor.h>
#include <Analyzers/ImplementFunctionVisitor.h>
#include <Analyzers/ReplaceViewWithSubqueryVisitor.h>
@ -512,6 +513,8 @@ namespace
graphviz_index);
}
if (settings.enable_order_by_all && select_query.order_by_all)
expandOrderByAll(&select_query);
// 5. Call `TreeOptimizer` since some optimizations will change the query result
if (select_query.having()
&& (!select_query.group_by_with_cube && !select_query.group_by_with_rollup && !select_query.group_by_with_grouping_sets
@ -612,6 +615,81 @@ ASTPtr QueryRewriter::rewrite(ASTPtr query, ContextMutablePtr context, bool enab
// explain.getExplainedQuery() = explain.getExplainedQuery();
// }
if (const auto * select_with_union_query = query->as<ASTSelectWithUnionQuery>())
{
auto settings = context->getSettingsRef();
bool settings_sorting_limit_offset_needed = false;
size_t num_children = select_with_union_query->list_of_selects->children.size();
if (settings.limit > 0 || settings.offset > 0 || settings.final_order_by_all_direction != 0)
settings_sorting_limit_offset_needed = true;
if (num_children == 1 && settings_sorting_limit_offset_needed)
{
const ASTPtr first_select_ast = select_with_union_query->list_of_selects->children.at(0);
ASTSelectQuery * select_query = dynamic_cast<ASTSelectQuery *>(first_select_ast.get());
if (!select_query)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid type in list_of_selects: {}", first_select_ast->getID());
if (!select_query->withFill() && !select_query->limit_with_ties)
{
if (settings.final_order_by_all_direction != 0)
{
int direction = context->getSettingsRef().final_order_by_all_direction > 0 ? 1 : -1;
auto order_expression_list = std::make_shared<ASTExpressionList>();
for (const auto & expr : select_query->select()->children)
{
auto elem = std::make_shared<ASTOrderByElement>();
elem->direction = direction;
elem->nulls_direction = 1;
elem->children.push_back(expr);
order_expression_list->children.push_back(elem);
}
select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, order_expression_list);
select_query->order_by_all = false;
}
UInt64 limit_length = 0;
UInt64 limit_offset = 0;
const ASTPtr limit_offset_ast = select_query->limitOffset();
if (limit_offset_ast)
{
limit_offset = limit_offset_ast->as<ASTLiteral &>().value.safeGet<UInt64>();
UInt64 new_limit_offset = settings.offset + limit_offset;
limit_offset_ast->as<ASTLiteral &>().value = Field(new_limit_offset);
}
else if (settings.offset)
{
ASTPtr new_limit_offset_ast = std::make_shared<ASTLiteral>(Field(UInt64(settings.offset)));
select_query->setExpression(ASTSelectQuery::Expression::LIMIT_OFFSET, std::move(new_limit_offset_ast));
}
const ASTPtr limit_length_ast = select_query->limitLength();
if (limit_length_ast)
{
limit_length = limit_length_ast->as<ASTLiteral &>().value.safeGet<UInt64>();
UInt64 new_limit_length = 0;
if (settings.offset == 0)
new_limit_length = std::min(limit_length, UInt64(settings.limit));
else if (settings.offset < limit_length)
new_limit_length = settings.limit ? std::min(UInt64(settings.limit), limit_length - settings.offset) : (limit_length - settings.offset);
limit_length_ast->as<ASTLiteral &>().value = Field(new_limit_length);
}
else if (settings.limit)
{
ASTPtr new_limit_length_ast = std::make_shared<ASTLiteral>(Field(UInt64(settings.limit)));
select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, std::move(new_limit_length_ast));
}
context->setSetting("offset", Field(0));
context->setSetting("limit", Field(0));
context->setSetting("final_order_by_all_direction", Field(0));
}
}
}
GraphvizPrinter::printAST(query, context, std::to_string(graphviz_index++) + "-AST-done");
LOG_DEBUG(logger, "rewritten query: {}", query->formatForErrorMessageWithoutAlias());

View File

@ -37,12 +37,12 @@ bool FieldDescription::matchName(const String & target_name) const
FieldDescription FieldDescription::withNewName(const String & new_name) const
{
return {new_name, type, prefix, origin_columns, substituted_by_asterisk};
return {new_name, type, prefix, origin_columns, substituted_by_asterisk, can_be_array_joined};
}
FieldDescription FieldDescription::withNewPrefix(const QualifiedName & new_prefix) const
{
return {name, type, new_prefix, origin_columns, substituted_by_asterisk};
return {name, type, new_prefix, origin_columns, substituted_by_asterisk, can_be_array_joined};
}
void FieldDescription::copyOriginInfo(const FieldDescription & source_field)

View File

@ -68,19 +68,32 @@ struct FieldDescription
/// s.a. https://clickhouse.com/docs/zh/sql-reference/statements/create/table/#materialized
bool substituted_by_asterisk;
bool can_be_array_joined;
// constructor for computed columns
FieldDescription(String name_, DataTypePtr type_, QualifiedName prefix_ = {})
: FieldDescription(std::move(name_), std::move(type_), std::move(prefix_), {}, true)
: FieldDescription(std::move(name_), std::move(type_), std::move(prefix_), {}, true, true)
{
}
FieldDescription(String name_, DataTypePtr type_, QualifiedName prefix_, OriginColumns origin_columns_, bool substituted_by_asterisk_)
// full costructor
FieldDescription(
String name_,
DataTypePtr type_,
QualifiedName prefix_,
OriginColumns origin_columns_,
bool substituted_by_asterisk_,
bool can_be_array_joined_)
: name(std::move(name_))
, type(std::move(type_))
, prefix(std::move(prefix_))
, origin_columns(std::move(origin_columns_))
, substituted_by_asterisk(substituted_by_asterisk_)
, can_be_array_joined(can_be_array_joined_)
{
}
// constructor for table columns
FieldDescription(
String name_,
DataTypePtr type_,
@ -90,7 +103,8 @@ struct FieldDescription
IAST * origin_table_ast_,
String origin_column_,
size_t index_of_origin_scope_,
bool substituted_by_asterisk_)
bool substituted_by_asterisk_,
bool can_be_array_joined_)
: name(std::move(name_))
, type(std::move(type_))
, prefix(std::move(prefix_))
@ -101,6 +115,7 @@ struct FieldDescription
std::move(origin_column_),
index_of_origin_scope_}}
, substituted_by_asterisk(substituted_by_asterisk_)
, can_be_array_joined(can_be_array_joined_)
{
}

View File

@ -28,6 +28,7 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int UNEXPECTED_EXPRESSION;
}
ASTPtr joinCondition(const ASTTableJoin & join)
@ -153,4 +154,36 @@ std::vector<ASTPtr> extractReferencesToScope(ContextPtr context, Analysis & anal
return extractExpressions(context, analysis, root, include_subquery, filter);
}
void expandOrderByAll(ASTSelectQuery * select_query)
{
auto * all_elem = select_query->orderBy()->children[0]->as<ASTOrderByElement>();
if (!all_elem)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Select analyze for not order by asts.");
auto order_expression_list = std::make_shared<ASTExpressionList>();
for (const auto & expr : select_query->select()->children)
{
if (auto * identifier = expr->as<ASTIdentifier>(); identifier != nullptr)
if (Poco::toUpper(identifier->name()) == "ALL" || Poco::toUpper(identifier->alias) == "ALL")
throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION,
"Cannot use ORDER BY ALL to sort a column with name 'all', please disable setting `enable_order_by_all` and try again");
if (auto * function = expr->as<ASTFunction>(); function != nullptr)
if (Poco::toUpper(function->alias) == "ALL")
throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION,
"Cannot use ORDER BY ALL to sort a column with name 'all', please disable setting `enable_order_by_all` and try again");
auto elem = std::make_shared<ASTOrderByElement>();
elem->direction = all_elem->direction;
elem->nulls_direction = all_elem->nulls_direction;
elem->nulls_direction_was_explicitly_specified = all_elem->nulls_direction_was_explicitly_specified;
elem->children.push_back(expr);
order_expression_list->children.push_back(elem);
}
select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, order_expression_list);
select_query->order_by_all = false;
}
}

View File

@ -42,4 +42,5 @@ std::vector<ASTPtr> extractExpressions(ContextPtr context, Analysis & analysis,
std::vector<ASTPtr> extractReferencesToScope(ContextPtr context, Analysis & analysis, ASTPtr root, ScopePtr scope, bool include_subquery = false);
void expandOrderByAll(ASTSelectQuery * select_query);
}

View File

@ -7,6 +7,7 @@
#include <Parsers/parseQuery.h>
#include <Transaction/TransactionCoordinatorRcCnch.h>
#include <Common/escapeForFileName.h>
#include <Interpreters/executeSubQuery.h>
namespace DB
{
@ -150,42 +151,4 @@ readTableHistoryFromBackup(const DatabaseAndTableName & table_name, const DiskPt
}
return previous_versions;
}
// Return a context copy with new transcation
ContextMutablePtr getContextWithNewTransaction(const ContextPtr & context, bool read_only)
{
ContextMutablePtr new_context;
if (context->hasSessionContext())
{
new_context = Context::createCopy(context->getSessionContext());
}
else
{
new_context = Context::createCopy(context->getGlobalContext());
}
new_context->setSettings(context->getSettings());
if (context->tryGetCurrentWorkerGroup())
{
new_context->setCurrentVW(context->getCurrentVW());
new_context->setCurrentWorkerGroup(context->getCurrentWorkerGroup());
}
auto txn = new_context->getCnchTransactionCoordinator().createTransaction(
CreateTransactionOption()
.setContext(new_context)
.setForceCleanByDM(context->getSettingsRef().force_clean_transaction_by_dm)
.setAsyncPostCommit(context->getSettingsRef().async_post_commit)
.setReadOnly(read_only));
if (txn)
{
new_context->setCurrentTransaction(txn);
}
else
{
throw Exception("Failed to create transaction", ErrorCodes::LOGICAL_ERROR);
}
return new_context;
}
}

View File

@ -7,6 +7,7 @@
#include <Interpreters/Context_fwd.h>
#include <Parsers/ASTBackupQuery.h>
#include <Protos/data_models.pb.h>
#include <Interpreters/executeSubQuery.h>
#define BACKUP_SCHEMA_VERSION 1
@ -109,6 +110,4 @@ ASTPtr readCreateQueryFromBackup(const DatabaseAndTableName & table_name, const
std::optional<std::vector<ASTPtr>> readTableHistoryFromBackup(const DatabaseAndTableName & table_name, const DiskPtr & backup_disk, const String & base_backup_path);
// Return a context copy with new transcation
ContextMutablePtr getContextWithNewTransaction(const ContextPtr & context, bool read_only);
}

View File

@ -744,7 +744,6 @@ if (ENABLE_TESTS AND USE_GTEST)
${GTEST_BOTH_LIBRARIES}
clickhouse_functions
clickhouse_aggregate_functions
clickhouse_parsers
clickhouse_storages_system
dbms
clickhouse_common_zookeeper

View File

@ -4081,6 +4081,8 @@ namespace Catalog
void Catalog::clearUndoBuffersByKeys(const TxnTimestamp & txnID, const std::vector<String> & keys)
{
if (keys.empty())
return;
runWithMetricSupport(
[&] {
const String undo_buffer_key_prefix = meta_proxy->undoBufferKeyPrefix(name_space, txnID, false);
@ -7301,8 +7303,7 @@ namespace Catalog
}
}
ServerDataPartsVector Catalog::listDetachedParts(const MergeTreeMetaBase& storage,
const AttachFilter& filter)
ServerDataPartsVector Catalog::listDetachedParts(const MergeTreeMetaBase & storage, const AttachFilter & filter)
{
IMetaStore::IteratorPtr iter = nullptr;
switch (filter.mode)
@ -8056,5 +8057,45 @@ namespace Catalog
// then, clear manifest list metadata from kv.
meta_proxy->getMetastore()->adaptiveBatchWrite(clear_manifest_list_request);
}
size_t Catalog::removePartsInBatch(
const MergeTreeMetaBase & storage, std::function<bool(const ServerDataPartsVector &)> func, const size_t batch_size)
{
const String prefix = meta_proxy->dataPartPrefix(name_space, UUIDHelpers::UUIDToString(storage.getStorageUUID()));
return meta_proxy->removeByPrefix(
prefix,
[&storage, &func](const std::vector<std::pair<const String, const String>> & kvs) {
ServerDataPartsVector parts;
for (const auto & kv : kvs)
{
Protos::DataModelPart part_model;
part_model.ParseFromString(kv.second);
parts.emplace_back(std::make_shared<ServerDataPart>(createPartWrapperFromModel(storage, std::move(part_model))));
}
return func(parts);
},
batch_size);
}
size_t
Catalog::removeDeleteBitmapsInBatch(const MergeTreeMetaBase & storage, std::function<bool(const DeleteBitmapMetaPtrVector &)> func, const size_t batch_size)
{
const String prefix = meta_proxy->deleteBitmapPrefix(name_space, UUIDHelpers::UUIDToString(storage.getStorageUUID()));
return meta_proxy->removeByPrefix(
prefix,
[&func, &storage](const std::vector<std::pair<const String, const String>> & kvs) {
DeleteBitmapMetaPtrVector bitmaps;
for (const auto & kv : kvs)
{
std::shared_ptr<Protos::DataModelDeleteBitmap> bitmap_model = std::make_shared<Protos::DataModelDeleteBitmap>();
bitmap_model->ParseFromString(kv.second);
bitmaps.emplace_back(std::make_shared<DeleteBitmapMeta>(storage, bitmap_model));
}
return func(bitmaps);
},
batch_size);
}
}
}

View File

@ -17,6 +17,7 @@
#include <Common/Logger.h>
#include <atomic>
#include <functional>
#include <map>
#include <optional>
#include <set>
@ -910,6 +911,23 @@ public:
DeleteBitmapMetaPtrVector listDetachedDeleteBitmaps(const MergeTreeMetaBase & storage, const AttachFilter & filter);
/**
* @brief Remove single storage's parts in batch.
*
* @param func Will be called before deleting parts (belong to current batch) in catalog.
* If func returns false, current batch will be skipped.
* @return The number of parts scanned.
*/
size_t removePartsInBatch(const MergeTreeMetaBase & storage, std::function<bool(const ServerDataPartsVector &)> func, size_t batch_size = 10000);
/**
* @brief Remove single storage's bitmaps in batch.
*
* @param func Will be called before deleting parts (belong to current batch) in catalog.
* If func returns false, current batch will be skipped.
* @return The number of parts scanned.
*/
size_t removeDeleteBitmapsInBatch(const MergeTreeMetaBase & storage, std::function<bool(const DeleteBitmapMetaPtrVector &)> func, size_t batch_size = 10000);
// Append partial object column schema in Txn
void
appendObjectPartialSchema(const StoragePtr & table, const TxnTimestamp & txn_id, const MutableMergeTreeDataPartsCNCHVector & parts);

View File

@ -455,10 +455,33 @@ bool Iterator::Next(fdb_error_t & code)
{
if (iteration==1)
{
batch_future = std::make_shared<FDBFutureRAII>(fdb_transaction_get_range(tr->transaction,
FDB_KEYSEL_FIRST_GREATER_OR_EQUAL(reinterpret_cast<const uint8_t*>(start_key_batch.c_str()), start_key_batch.size()),
FDB_KEYSEL_FIRST_GREATER_OR_EQUAL(reinterpret_cast<const uint8_t*>(req.end_key.c_str()), req.end_key.size()),
req.row_limit, 0, FDB_STREAMING_MODE_LARGE, iteration, 1, req.reverse_order));
if (req.exclude_start_key)
{
batch_future = std::make_shared<FDBFutureRAII>(fdb_transaction_get_range(
tr->transaction,
FDB_KEYSEL_FIRST_GREATER_THAN(reinterpret_cast<const uint8_t *>(start_key_batch.c_str()), start_key_batch.size()),
FDB_KEYSEL_FIRST_GREATER_OR_EQUAL(reinterpret_cast<const uint8_t *>(req.end_key.c_str()), req.end_key.size()),
req.row_limit,
0,
FDB_STREAMING_MODE_LARGE,
iteration,
1,
req.reverse_order));
}
else
{
batch_future = std::make_shared<FDBFutureRAII>(fdb_transaction_get_range(
tr->transaction,
FDB_KEYSEL_FIRST_GREATER_OR_EQUAL(
reinterpret_cast<const uint8_t *>(start_key_batch.c_str()), start_key_batch.size()),
FDB_KEYSEL_FIRST_GREATER_OR_EQUAL(reinterpret_cast<const uint8_t *>(req.end_key.c_str()), req.end_key.size()),
req.row_limit,
0,
FDB_STREAMING_MODE_LARGE,
iteration,
1,
req.reverse_order));
}
}
else
{

View File

@ -55,6 +55,7 @@ struct ScanRequest
std::string start_key;
std::string end_key;
bool exclude_start_key = false;
uint32_t row_limit = 0;
fdb_bool_t reverse_order = 0;
};

View File

@ -98,7 +98,13 @@ public:
* Range scan by specific prefix; limit the number of result
* @param start_key If provided, it'll be used as the start key, and `key_prefix` will only be used to generate `end_key`.
*/
virtual IteratorPtr getByPrefix(const String & key_prefix, const size_t & limit = 0, uint32_t scan_batch_size = DEFAULT_SCAN_BATCH_COUNT, const String & start_key = "") = 0;
virtual IteratorPtr getByPrefix(
const String & key_prefix,
const size_t & limit = 0,
uint32_t scan_batch_size = DEFAULT_SCAN_BATCH_COUNT,
const String & start_key = "",
bool exclude_start_key = false)
= 0;
/***
* Scan a range of records by start and end key;

View File

@ -186,6 +186,13 @@ std::vector<std::pair<String, UInt64>> MetastoreByteKVImpl::multiGet(const std::
bool MetastoreByteKVImpl::batchWrite(const BatchCommitRequest & req, BatchCommitResponse & response)
{
/// Early return if request is empty.
if (req.isEmpty())
{
response.reset();
return true;
}
auto timer = ProfileEventsTimer(ProfileEvents::KvRpcRequest, ProfileEvents::KvRpcElapsedMicroseconds);
bytekv::sdk::WriteBatchRequest wb_req;
bytekv::sdk::WriteBatchResponse wb_resp;
@ -289,13 +296,14 @@ MetastoreByteKVImpl::IteratorPtr MetastoreByteKVImpl::getAll()
return getByPrefix("");
}
MetastoreByteKVImpl::IteratorPtr MetastoreByteKVImpl::getByPrefix(const String & partition_id, const size_t & limit, uint32_t scan_batch_size, const String & start_key)
MetastoreByteKVImpl::IteratorPtr MetastoreByteKVImpl::getByPrefix(const String & partition_id, const size_t & limit, uint32_t scan_batch_size, const String & start_key, const bool exclude_start_key)
{
auto timer = ProfileEventsTimer(ProfileEvents::KvRpcRequest, ProfileEvents::KvRpcElapsedMicroseconds);
ScanRequest scan_req;
scan_req.scan_batch_count = scan_batch_size;
scan_req.limit = limit;
scan_req.table = table_name;
scan_req.exclude_start_key = exclude_start_key;
if (likely(start_key.empty()))
{
scan_req.start_key = partition_id;

View File

@ -106,7 +106,8 @@ public:
const String & key_prefix,
const size_t & limit = 0,
uint32_t scan_batch_size = DEFAULT_SCAN_BATCH_COUNT,
const String & start_key = "") override;
const String & start_key = "",
bool exclude_start_key = false) override;
IteratorPtr getByRange(const String & range_start, const String & range_end, const bool include_start, const bool include_end) override;

View File

@ -130,7 +130,7 @@ MetastoreFDBImpl::IteratorPtr MetastoreFDBImpl::getAll()
return std::make_shared<FDBIterator>(fdb_iter);
}
MetastoreFDBImpl::IteratorPtr MetastoreFDBImpl::getByPrefix(const String & prefix, const size_t & limit, uint32_t, const String & start_key)
MetastoreFDBImpl::IteratorPtr MetastoreFDBImpl::getByPrefix(const String & prefix, const size_t & limit, uint32_t, const String & start_key, const bool exclude_start_key)
{
FDB::ScanRequest scan_req;
@ -144,6 +144,7 @@ MetastoreFDBImpl::IteratorPtr MetastoreFDBImpl::getByPrefix(const String & prefi
}
scan_req.row_limit = limit;
scan_req.end_key = getNextKey(prefix);
scan_req.exclude_start_key = exclude_start_key;
FDB::FDBTransactionPtr tr = std::make_shared<FDB::FDBTransactionRAII>();
check_fdb_op(fdb_client->CreateTransaction(tr));
@ -165,6 +166,13 @@ MetastoreFDBImpl::IteratorPtr MetastoreFDBImpl::getByRange(const String & range_
bool MetastoreFDBImpl::batchWrite(const BatchCommitRequest & req, BatchCommitResponse & response)
{
/// Early return if request is empty.
if (req.isEmpty())
{
response.reset();
return true;
}
for (auto & single_put : req.puts)
{
assertNotReadonly(single_put.key);

View File

@ -94,7 +94,8 @@ public:
const String &,
const size_t & limit = 0,
uint32_t scan_batch_size = DEFAULT_SCAN_BATCH_COUNT,
const String & start_key = "") override;
const String & start_key = "",
bool exclude_start_key = false) override;
IteratorPtr getByRange(const String & range_start, const String & range_end, const bool include_start, const bool include_end) override;

View File

@ -1658,8 +1658,14 @@ void MetastoreProxy::multiDrop(const Strings & keys)
metastore_ptr->adaptiveBatchWrite(batch_write);
}
bool MetastoreProxy::batchWrite(const BatchCommitRequest & request, BatchCommitResponse response)
bool MetastoreProxy::batchWrite(const BatchCommitRequest & request, BatchCommitResponse & response)
{
/// Early return if request is empty.
if (request.isEmpty())
{
response.reset();
return true;
}
return metastore_ptr->batchWrite(request, response);
}
@ -3661,4 +3667,49 @@ bool MetastoreProxy::putAccessEntity(EntityType type, const String & name_space,
}
}
size_t MetastoreProxy::removeByPrefix(
const String & prefix,
std::function<bool(const std::vector<std::pair<const String, const String>> & kvs)> func,
const size_t batch_size)
{
String start_key;
size_t scanned_count = 0;
bool exclude_start_key = false;
const size_t scan_batch_size = std::min(static_cast<size_t>(DEFAULT_SCAN_BATCH_COUNT), batch_size);
/// For each batch.
while (true)
{
/// Scan keys in batch.
auto it = metastore_ptr->getByPrefix(prefix, batch_size, scan_batch_size, start_key, exclude_start_key);
BatchCommitRequest batch_write;
std::vector<std::pair<const String, const String>> kvs;
while (it->next())
{
kvs.emplace_back(it->key(), it->value());
batch_write.AddDelete(it->key());
}
/// Return if scanned to end.
if (kvs.size() == 0)
return scanned_count;
scanned_count += kvs.size();
bool need_commit = func(kvs);
/// Commit if there is any key to be removed.
if (need_commit)
{
BatchCommitResponse delete_result;
metastore_ptr->batchWrite(batch_write, delete_result);
} else {
/// Update start key for next batch.
start_key = kvs.back().first;
exclude_start_key = true;
}
}
}
} /// end of namespace DB::Catalog

View File

@ -1022,6 +1022,28 @@ public:
/// end of Metastore Proxy keying schema
/**
* @brief Remove keys by prefix. This is a very general function, which can
* be used to multiple type of data. By providing a function, users can
* `hook into` pre-commit process to delete the data.
* This method is designed to be:
* 1. Memory-efficient, no more than batch_size keys will be loaded into memory.
* 2. Fast, because it will scan keys in batch.
* 3. Multi-thread friendly, because it returns KVs in batch, which can
* be executed in parallel by caller.
*
* @param prefix A prefix of the keys to be scanned.
* @param kvs Metastore (Key, Value), caller need to parse it manually.
* @param func Will be called before removing the keys. Skip current batch when
* func return `false`. And the procedure will continue until
* all keys are processed.
* @param batch_size Data will be feed into func and committed in batch.
* @return Total number of keys scanned. (Useful for checking if the prefix is cleaned)
*/
size_t removeByPrefix(
const String & prefix,
std::function<bool(const std::vector<std::pair<const String, const String>> & kvs)> func,
const size_t batch_size = 10000);
void createTransactionRecord(const String & name_space, const UInt64 & txn_id, const String & txn_data);
void removeTransactionRecord(const String & name_space, const UInt64 & txn_id);
void removeTransactionRecords(const String & name_space, const std::vector<TxnTimestamp> & txn_ids);
@ -1212,7 +1234,7 @@ public:
void multiDrop(const Strings & keys);
bool batchWrite(const BatchCommitRequest & request, BatchCommitResponse response);
bool batchWrite(const BatchCommitRequest & request, BatchCommitResponse & response);
/// tmp api to help debug drop keys failed issue. remove this later.
std::vector<String> multiDropAndCheck(const Strings & keys);

View File

@ -288,7 +288,8 @@ std::unique_ptr<ManipulationTaskRecord> FutureManipulationTask::moveRecord()
if (!record->transaction)
throw Exception("The transaction of manipulation task is not initialized", ErrorCodes::LOGICAL_ERROR);
record->parts = std::move(parts);
record->parts.clear();
std::swap(record->parts, parts);
return std::move(record);
}

View File

@ -26,6 +26,7 @@
#include <Interpreters/Context_fwd.h>
#include <Interpreters/WorkerStatusManager.h>
#include <MergeTreeCommon/assignCnchParts.h>
#include <Protos/cnch_worker_rpc.pb.h>
#include <Storages/DataLakes/StorageCnchLakeBase.h>
#include <brpc/controller.h>
#include "Common/ProfileEvents.h"
@ -34,6 +35,7 @@
#include <Common/Exception.h>
#include <common/logger_useful.h>
#include <CloudServices/QueryResourceUtils.h>
#include <Interpreters/DistributedStages/BSPScheduler.h>
#include <Storages/RemoteFile/StorageCnchHDFS.h>
#include <Storages/RemoteFile/StorageCnchS3.h>
@ -357,16 +359,16 @@ void CnchServerResource::resendResource(const ContextPtr & context, const HostWi
std::vector<AssignedResource> resources_to_send;
{
auto lock = getLock();
ResourceOption resource_option{.resend = true};
allocateResource(context, lock, resource_option);
ResourceOption resource_option{.resend = true};
allocateResource(context, lock, resource_option);
std::unordered_map<HostWithPorts, std::vector<AssignedResource>> all_resources;
std::swap(all_resources, assigned_worker_resource);
auto it = all_resources.find(worker);
if (it == all_resources.end())
return;
std::unordered_map<HostWithPorts, std::vector<AssignedResource>> all_resources;
std::swap(all_resources, assigned_worker_resource);
auto it = all_resources.find(worker);
if (it == all_resources.end())
return;
resources_to_send = std::move(it->second);
resources_to_send = std::move(it->second);
}
Stopwatch rpc_watch;
@ -458,6 +460,37 @@ void CnchServerResource::sendResources(const ContextPtr & context, std::optional
ProfileEvents::increment(ProfileEvents::CnchSendResourceElapsedMilliseconds, watch.elapsedMilliseconds());
}
void CnchServerResource::prepareQueryResourceBuf(
std::unordered_map<WorkerId, butil::IOBuf, WorkerIdHash> & resource_buf_map, const ContextPtr & context)
{
std::unordered_map<HostWithPorts, std::vector<AssignedResource>> all_resources;
{
auto lock = getLock();
allocateResource(context, lock, std::nullopt);
if (!worker_group)
return;
std::swap(all_resources, assigned_worker_resource);
}
for (const auto & [worker_address, worker_resources] : all_resources)
{
auto worker_client = worker_group->getWorkerClient(worker_address);
auto worker_id = WorkerStatusManager::getWorkerId(worker_group->getVWName(), worker_group->getID(), worker_address.id);
Protos::QueryResource query_resource;
prepareQueryResource(query_resource, worker_id, worker_resources, context, send_mutations, log);
butil::IOBuf query_resource_buf;
butil::IOBufAsZeroCopyOutputStream wrapper(&query_resource_buf);
query_resource.SerializeToZeroCopyStream(&wrapper);
resource_buf_map[worker_id] = query_resource_buf.movable();
LOG_TRACE(log, "Worker id {} -> query resource size {}", worker_id.toString(), query_resource_buf.size());
}
}
void CnchServerResource::submitCustomTasks(const ContextPtr & context, WorkerAction act)
{
auto handler = std::make_shared<ExceptionHandler>();
@ -696,7 +729,7 @@ void CnchServerResource::allocateResource(
{
LOG_TRACE(
log,
"SourcePrune skip stroage {} for host {}",
"SourcePrune skip storage {} for host {}",
storage->getStorageID().getNameForLogs(),
host_ports.toDebugString());
continue;

View File

@ -211,6 +211,8 @@ public:
void resendResource(const ContextPtr & context, const HostWithPorts & worker);
/// allocate and send resource to worker_group
void sendResources(const ContextPtr & context, std::optional<ResourceOption> resource_option = std::nullopt);
// prepare query resource buf for batch send plan segments
void prepareQueryResourceBuf(std::unordered_map<WorkerId, butil::IOBuf, WorkerIdHash> & resource_buf_map, const ContextPtr & context);
/// WorkerAction should not throw
using WorkerAction

View File

@ -32,11 +32,12 @@
#include <WorkerTasks/ManipulationList.h>
#include <WorkerTasks/ManipulationTaskParams.h>
#include <CloudServices/QueryResourceUtils.h>
#include <Storages/MergeTree/MarkRange.h>
#include <brpc/callback.h>
#include <brpc/channel.h>
#include <brpc/controller.h>
#include <common/logger_useful.h>
#include <Storages/MergeTree/MarkRange.h>
namespace ProfileEvents
{
@ -383,134 +384,14 @@ brpc::CallId CnchWorkerClient::sendResources(
{
auto timer = ProfileEventsTimer(ProfileEvents::WorkerRpcRequest, ProfileEvents::WorkerRpcElaspsedMicroseconds);
Protos::SendResourcesReq request;
const auto & settings = context->getSettingsRef();
auto max_execution_time = settings.max_execution_time.value.totalSeconds();
request.set_txn_id(context->getCurrentTransactionID());
request.set_primary_txn_id(context->getCurrentTransaction()->getPrimaryTransactionID());
/// recycle_timeout refers to the time when the session is recycled under abnormal case,
/// so it should be larger than max_execution_time to make sure the session is not to be destroyed in advance.
UInt64 recycle_timeout = max_execution_time > 0 ? max_execution_time + 60UL : 3600;
request.set_timeout(recycle_timeout);
if (!settings.session_timezone.value.empty())
request.set_session_timezone(settings.session_timezone.value);
bool require_worker_info = false;
for (const auto & resource: resources_to_send)
{
if (!resource.sent_create_query)
{
const auto & def = resource.table_definition;
if (resource.table_definition.cacheable)
{
auto * cacheable = request.add_cacheable_create_queries();
RPCHelpers::fillStorageID(resource.storage->getStorageID(), *cacheable->mutable_storage_id());
cacheable->set_definition(def.definition);
if (!resource.object_columns.empty())
cacheable->set_dynamic_object_column_schema(resource.object_columns.toString());
cacheable->set_local_engine_type(static_cast<UInt32>(def.engine_type));
cacheable->set_local_table_name(def.local_table_name);
if (!def.underlying_dictionary_tables.empty())
cacheable->set_local_underlying_dictionary_tables(def.underlying_dictionary_tables);
}
else
{
request.add_create_queries(def.definition);
request.add_dynamic_object_column_schema(resource.object_columns.toString());
}
}
/// parts
auto & table_data_parts = *request.mutable_data_parts()->Add();
/// Send storage's mutations to worker if needed.
if (with_mutations)
{
auto * cnch_merge_tree = dynamic_cast<StorageCnchMergeTree *>(resource.storage.get());
if (cnch_merge_tree)
{
for (auto const & mutation_str : cnch_merge_tree->getPlainMutationEntries())
{
LOG_TRACE(log, "Send mutations to worker: {}", mutation_str);
table_data_parts.add_cnch_mutation_entries(mutation_str);
}
}
}
table_data_parts.set_database(resource.storage->getDatabaseName());
table_data_parts.set_table(resource.table_definition.local_table_name);
if (resource.table_version)
{
require_worker_info = true;
table_data_parts.set_table_version(resource.table_version);
}
if (settings.query_dry_run_mode != QueryDryRunMode::SKIP_SEND_PARTS)
{
if (!resource.server_parts.empty())
{
// todo(jiashuo): bitmap need handler?
fillBasePartAndDeleteBitmapModels(
*resource.storage,
resource.server_parts,
*table_data_parts.mutable_server_parts(),
*table_data_parts.mutable_server_part_bitmaps());
}
if (!resource.virtual_parts.empty())
{
fillPartsModelForSend(*resource.storage, resource.virtual_parts, *table_data_parts.mutable_virtual_parts());
auto * bitmaps_model = table_data_parts.mutable_virtual_part_bitmaps();
for (const auto & virtual_part : resource.virtual_parts)
{
for (auto & bitmap_meta : virtual_part->part->delete_bitmap_metas)
{
bitmaps_model->Add()->CopyFrom(*bitmap_meta);
}
}
}
}
if (!resource.lake_scan_info_parts.empty())
{
auto * mutable_lake_scan_infos = table_data_parts.mutable_lake_scan_info_parts();
auto & cnch_lake = dynamic_cast<StorageCnchLakeBase &>(*resource.storage);
cnch_lake.serializeLakeScanInfos(*mutable_lake_scan_infos, resource.lake_scan_info_parts);
}
if (!resource.file_parts.empty())
{
fillCnchFilePartsModel(resource.file_parts, *table_data_parts.mutable_file_parts());
}
/// bucket numbers
for (const auto & bucket_num : resource.bucket_numbers)
*table_data_parts.mutable_bucket_numbers()->Add() = bucket_num;
}
// need add worker info if query by table version
if (require_worker_info)
{
auto current_wg = context->getCurrentWorkerGroup();
auto * worker_info = request.mutable_worker_info();
worker_info->set_worker_id(worker_id.id);
worker_info->set_index(current_wg->getWorkerIndex(worker_id.id));
worker_info->set_num_workers(current_wg->workerNum());
if (worker_info->num_workers() <= worker_info->index())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invailid worker index {} for worker group {}, which contains {} workers.",
toString(worker_info->index()),
current_wg->getVWName(),
toString(current_wg->workerNum()));
}
request.set_disk_cache_mode(context->getSettingsRef().disk_cache_mode.toString());
prepareQueryResource(request, worker_id, resources_to_send, context, with_mutations, log);
brpc::Controller * cntl = new brpc::Controller;
/// send_timeout refers to the time to send resource to worker
/// If max_execution_time is not set, the send_timeout will be set to brpc_data_parts_timeout_ms
auto send_timeout_ms = max_execution_time ? max_execution_time * 1000L : settings.brpc_data_parts_timeout_ms.totalMilliseconds();
auto max_execution_time = context->getSettingsRef().max_execution_time.value.totalSeconds();
auto send_timeout_ms
= max_execution_time ? max_execution_time * 1000L : context->getSettingsRef().brpc_data_parts_timeout_ms.totalMilliseconds();
cntl->set_timeout_ms(send_timeout_ms);
const auto call_id = cntl->call_id();
auto * response = new Protos::SendResourcesResp();

View File

@ -18,21 +18,22 @@
#include <CloudServices/CnchCreateQueryHelper.h>
#include <Core/Names.h>
#include <Databases/DatabaseMemory.h>
#include <IO/ReadBufferFromString.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterSetQuery.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/InterpreterSetQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ParserQueryWithOutput.h>
#include <Parsers/formatAST.h>
#include <Parsers/formatTenantDatabaseName.h>
#include <Parsers/parseQuery.h>
#include <Poco/Logger.h>
#include <Databases/DatabaseMemory.h>
#include <Storages/ForeignKeysDescription.h>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/CloudTableDefinitionCache.h>
#include <Storages/StorageCloudMergeTree.h>
#include <Storages/StorageDictCloudMergeTree.h>
#include <Poco/Logger.h>
namespace DB
@ -178,18 +179,25 @@ void CnchWorkerResource::executeCacheableCreateQuery(
insertCloudTable({res_table_id.getDatabaseName(), res_table_id.getTableName()}, res, context, /*throw_if_exists=*/ false);
}
StoragePtr CnchWorkerResource::getTable(const StorageID & table_id) const
StoragePtr CnchWorkerResource::tryGetTable(const StorageID & table_id, bool load_data_parts) const
{
String tenant_db = formatTenantDatabaseName(table_id.getDatabaseName());
auto lock = getLock();
StoragePtr res = {};
auto it = cloud_tables.find({tenant_db, table_id.getTableName()});
if (it != cloud_tables.end())
{
return it->second;
auto lock = getLock();
auto it = cloud_tables.find({tenant_db, table_id.getTableName()});
if (it != cloud_tables.end())
res = it->second;
}
return {};
if (load_data_parts)
{
if (auto cloud_table = dynamic_pointer_cast<StorageCloudMergeTree>(res))
cloud_table->prepareDataPartsForRead();
}
return res;
}
DatabasePtr CnchWorkerResource::getDatabase(const String & database_name) const

View File

@ -1,3 +1,4 @@
#pragma once
/*
* Copyright (2022) Bytedance Ltd. and/or its affiliates
* Licensed under the Apache License, Version 2.0 (the "License");
@ -46,7 +47,7 @@ public:
const String & underlying_dictionary_tables,
const ColumnsDescription & object_columns);
StoragePtr getTable(const StorageID & table_id) const;
StoragePtr tryGetTable(const StorageID & table_id, bool load_data_parts = true) const;
DatabasePtr getDatabase(const String & database_name) const;
bool isCnchTableInWorker(const StorageID & table_id) const;

View File

@ -56,6 +56,7 @@
#include <brpc/stream.h>
#include <Common/Configurations.h>
#include <Common/Exception.h>
#include <CloudServices/QueryResourceUtils.h>
#include <IO/copyData.h>
#include <CloudServices/CnchDedupHelper.h>
#include <CloudServices/ManifestCache.h>
@ -83,12 +84,6 @@ namespace ProfileEvents
extern const Event PreloadExecTotalOps;
}
namespace ProfileEvents
{
extern const Event QueryCreateTablesMicroseconds;
extern const Event QuerySendResourcesMicroseconds;
}
namespace DB
{
namespace ErrorCodes
@ -100,6 +95,7 @@ namespace ErrorCodes
extern const int PREALLOCATE_QUERY_INTENT_NOT_FOUND;
extern const int SESSION_NOT_FOUND;
extern const int ABORTED;
extern const int WORKER_TABLE_NOT_FOUND;
}
CnchWorkerServiceImpl::CnchWorkerServiceImpl(ContextMutablePtr context_)
@ -715,182 +711,8 @@ void CnchWorkerServiceImpl::sendResources(
google::protobuf::Closure * done)
{
SUBMIT_THREADPOOL({
LOG_TRACE(log, "Receiving resources for Session: {}", request->txn_id());
Stopwatch watch;
auto rpc_context = RPCHelpers::createSessionContextForRPC(getContext(), *cntl);
auto session = rpc_context->acquireNamedCnchSession(request->txn_id(), request->timeout(), false);
auto query_context = session->context;
query_context->setTemporaryTransaction(request->txn_id(), request->primary_txn_id());
if (request->has_session_timezone())
query_context->setSetting("session_timezone", request->session_timezone());
CurrentThread::QueryScope query_scope(query_context);
auto worker_resource = query_context->getCnchWorkerResource();
/// store cloud tables in cnch_session_resource.
{
Stopwatch create_timer;
/// create a copy of session_context to avoid modify settings in SessionResource
auto context_for_create = Context::createCopy(query_context);
for (int i = 0; i < request->create_queries_size(); i++)
{
auto create_query = request->create_queries().at(i);
auto object_columns = request->dynamic_object_column_schema().at(i);
worker_resource->executeCreateQuery(context_for_create, create_query, false, ColumnsDescription::parse(object_columns));
}
for (int i = 0; i < request->cacheable_create_queries_size(); i++)
{
auto & item = request->cacheable_create_queries().at(i);
ColumnsDescription object_columns;
if (item.has_dynamic_object_column_schema())
object_columns = ColumnsDescription::parse(item.dynamic_object_column_schema());
worker_resource->executeCacheableCreateQuery(
context_for_create,
RPCHelpers::createStorageID(item.storage_id()),
item.definition(),
item.local_table_name(),
static_cast<WorkerEngineType>(item.local_engine_type()),
item.local_underlying_dictionary_tables(),
object_columns);
}
create_timer.stop();
LOG_INFO(log, "Prepared {} tables for session {} in {} us", request->create_queries_size() + request->cacheable_create_queries_size(), request->txn_id(), create_timer.elapsedMicroseconds());
ProfileEvents::increment(ProfileEvents::QueryCreateTablesMicroseconds, create_timer.elapsedMicroseconds());
}
for (const auto & data : request->data_parts())
{
auto storage = DatabaseCatalog::instance().getTable({data.database(), data.table()}, query_context);
if (auto * cloud_merge_tree = dynamic_cast<StorageCloudMergeTree *>(storage.get()))
{
if (data.has_table_version())
{
WGWorkerInfoPtr worker_info = RPCHelpers::createWorkerInfo(request->worker_info());
UInt64 version = data.table_version();
cloud_merge_tree->setDataDescription(std::move(worker_info), version);
LOG_DEBUG(log, "Received table {} with data version {}",
cloud_merge_tree->getStorageID().getNameForLogs(),
version);
}
else if (!data.server_parts().empty())
{
MergeTreeMutableDataPartsVector server_parts;
if (cloud_merge_tree->getInMemoryMetadataPtr()->hasUniqueKey())
server_parts = createBasePartAndDeleteBitmapFromModelsForSend<IMergeTreeMutableDataPartPtr>(
*cloud_merge_tree, data.server_parts(), data.server_part_bitmaps());
else
server_parts
= createPartVectorFromModelsForSend<IMergeTreeMutableDataPartPtr>(*cloud_merge_tree, data.server_parts());
auto server_parts_size = server_parts.size();
if (request->has_disk_cache_mode())
{
auto disk_cache_mode = SettingFieldDiskCacheModeTraits::fromString(request->disk_cache_mode());
if (disk_cache_mode != DiskCacheMode::AUTO)
{
for (auto & part : server_parts)
part->disk_cache_mode = disk_cache_mode;
}
}
/// `loadDataParts` is an expensive action as it may involve remote read.
/// The worker rpc thread pool may be blocked when there are many `sendResources` requests.
/// Here we just pass the server_parts to storage. And it will do `loadDataParts` later (before reading).
/// One exception is StorageDictCloudMergeTree as it use a different read logic rather than StorageCloudMergeTree::read.
bool is_dict = false;
if (auto * cloud_dict = dynamic_cast<StorageDictCloudMergeTree *>(storage.get()))
{
cloud_dict->loadDataParts(server_parts);
is_dict = true;
}
else
cloud_merge_tree->receiveDataParts(std::move(server_parts));
LOG_DEBUG(
log,
"Received {} parts for table {}(txn_id: {}), disk_cache_mode {}, is_dict: {}",
server_parts_size, cloud_merge_tree->getStorageID().getNameForLogs(),
request->txn_id(), request->disk_cache_mode(), is_dict);
}
if (!data.virtual_parts().empty())
{
MergeTreeMutableDataPartsVector virtual_parts;
if (cloud_merge_tree->getInMemoryMetadataPtr()->hasUniqueKey())
virtual_parts = createBasePartAndDeleteBitmapFromModelsForSend<IMergeTreeMutableDataPartPtr>(
*cloud_merge_tree, data.virtual_parts(), data.virtual_part_bitmaps());
else
virtual_parts
= createPartVectorFromModelsForSend<IMergeTreeMutableDataPartPtr>(*cloud_merge_tree, data.virtual_parts());
auto virtual_parts_size = virtual_parts.size();
if (request->has_disk_cache_mode())
{
auto disk_cache_mode = SettingFieldDiskCacheModeTraits::fromString(request->disk_cache_mode());
if (disk_cache_mode != DiskCacheMode::AUTO)
{
for (auto & part : virtual_parts)
part->disk_cache_mode = disk_cache_mode;
}
}
bool is_dict = false;
if (auto * cloud_dict = dynamic_cast<StorageDictCloudMergeTree *>(storage.get()))
{
cloud_dict->loadDataParts(virtual_parts);
is_dict = true;
}
else
cloud_merge_tree->receiveVirtualDataParts(std::move(virtual_parts));
LOG_DEBUG(
log,
"Received {} virtual parts for table {}(txn_id: {}), disk_cache_mode {}, is_dict: {}",
virtual_parts_size, cloud_merge_tree->getStorageID().getNameForLogs(),
request->txn_id(), request->disk_cache_mode(), is_dict);
}
std::set<Int64> required_bucket_numbers;
for (const auto & bucket_number : data.bucket_numbers())
required_bucket_numbers.insert(bucket_number);
cloud_merge_tree->setRequiredBucketNumbers(required_bucket_numbers);
for (const auto & mutation_str : data.cnch_mutation_entries())
{
auto mutation_entry = CnchMergeTreeMutationEntry::parse(mutation_str);
cloud_merge_tree->addMutationEntry(mutation_entry);
}
}
else if (auto * hive_table = dynamic_cast<StorageCloudHive *>(storage.get()))
{
auto settings = hive_table->getSettings();
auto lake_scan_infos = ILakeScanInfo::deserialize(data.lake_scan_info_parts(), query_context, storage->getInMemoryMetadataPtr(), *settings);
hive_table->loadLakeScanInfos(lake_scan_infos);
}
else if (auto * cloud_file_table = dynamic_cast<IStorageCloudFile *>(storage.get()))
{
auto data_parts = createCnchFileDataParts(getContext(), data.file_parts());
cloud_file_table->loadDataParts(data_parts);
LOG_DEBUG(
log,
"Received and loaded {} cloud file parts for table {}",
data_parts.size(),
cloud_file_table->getStorageID().getNameForLogs());
}
else
throw Exception("Unknown table engine: " + storage->getName(), ErrorCodes::UNKNOWN_TABLE);
}
watch.stop();
LOG_INFO(log, "Received all resources for session {} in {} us.", request->txn_id(), watch.elapsedMicroseconds());
ProfileEvents::increment(ProfileEvents::QuerySendResourcesMicroseconds, watch.elapsedMicroseconds());
loadQueryResource(*request, rpc_context);
})
}

View File

@ -0,0 +1,368 @@
#pragma once
#include <CloudServices/CnchServerResource.h>
#include <CloudServices/CnchWorkerResource.h>
#include <Interpreters/WorkerStatusManager.h>
#include <Protos/cnch_worker_rpc.pb.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/DataLakes/StorageCnchLakeBase.h>
#include <Storages/Hive/StorageCloudHive.h>
#include <Storages/MergeTree/CloudTableDefinitionCache.h>
#include <Storages/RemoteFile/IStorageCloudFile.h>
#include <Storages/StorageCloudMergeTree.h>
#include <Storages/StorageDictCloudMergeTree.h>
#include <Common/CurrentThread.h>
#include <Common/Logger.h>
#include <Common/Stopwatch.h>
namespace ProfileEvents
{
extern const Event QueryCreateTablesMicroseconds;
extern const Event QueryLoadResourcesMicroseconds;
}
namespace DB
{
namespace ErrorCodes
{
extern const int WORKER_TABLE_NOT_FOUND;
}
template <typename T>
static void loadQueryResource(const T & query_resource, const ContextPtr & context)
{
static LoggerPtr log = getLogger("WorkerResource");
LOG_TRACE(log, "Receiving resources for Session: {}", query_resource.txn_id());
Stopwatch watch;
auto session = context->acquireNamedCnchSession(query_resource.txn_id(), query_resource.timeout(), false);
auto session_context = session->context;
session_context->setTemporaryTransaction(query_resource.txn_id(), query_resource.primary_txn_id());
if (query_resource.has_session_timezone())
session_context->setSetting("session_timezone", query_resource.session_timezone());
CurrentThread::QueryScope query_scope(session_context);
auto worker_resource = session_context->getCnchWorkerResource();
/// store cloud tables in cnch_session_resource.
{
Stopwatch create_timer;
/// create a copy of session_context to avoid modify settings in SessionResource
auto context_for_create = Context::createCopy(session_context);
for (int i = 0; i < query_resource.create_queries_size(); i++)
{
auto create_query = query_resource.create_queries().at(i);
auto object_columns = query_resource.dynamic_object_column_schema().at(i);
worker_resource->executeCreateQuery(context_for_create, create_query, false, ColumnsDescription::parse(object_columns));
}
for (int i = 0; i < query_resource.cacheable_create_queries_size(); i++)
{
const auto & item = query_resource.cacheable_create_queries().at(i);
ColumnsDescription object_columns;
if (item.has_dynamic_object_column_schema())
object_columns = ColumnsDescription::parse(item.dynamic_object_column_schema());
worker_resource->executeCacheableCreateQuery(
context_for_create,
RPCHelpers::createStorageID(item.storage_id()),
item.definition(),
item.local_table_name(),
static_cast<WorkerEngineType>(item.local_engine_type()),
"",
object_columns);
}
create_timer.stop();
LOG_INFO(
log,
"Prepared {} tables for session {} in {} us",
query_resource.create_queries_size() + query_resource.cacheable_create_queries_size(),
query_resource.txn_id(),
create_timer.elapsedMicroseconds());
ProfileEvents::increment(ProfileEvents::QueryCreateTablesMicroseconds, create_timer.elapsedMicroseconds());
}
bool lazy_load_parts = query_resource.has_lazy_load_data_parts() && query_resource.lazy_load_data_parts();
for (const auto & data : query_resource.data_parts())
{
/// By default, calling getTable (from WorkerResource) will trigger loading data parts.
/// Here is the first time and happens before parts are ready. So don't trigger load data parts here.
StorageID storage_id = {data.database(), data.table()};
auto storage = worker_resource->tryGetTable(storage_id, /*load_data_parts*/ false);
if (!storage)
throw Exception(
ErrorCodes::WORKER_TABLE_NOT_FOUND, "Table {} not found in worker resource, it's a bug.", storage_id.getNameForLogs());
bool is_dict_table = false;
if (lazy_load_parts)
is_dict_table = !!dynamic_cast<StorageDictCloudMergeTree *>(storage.get());
if (auto * cloud_merge_tree = dynamic_cast<StorageCloudMergeTree *>(storage.get()))
{
if (data.has_table_version())
{
WGWorkerInfoPtr worker_info = RPCHelpers::createWorkerInfo(query_resource.worker_info());
UInt64 version = data.table_version();
cloud_merge_tree->setDataDescription(std::move(worker_info), version);
LOG_DEBUG(log, "Received table {} with data version {}", cloud_merge_tree->getStorageID().getNameForLogs(), version);
}
else if (!data.server_parts().empty())
{
MergeTreeMutableDataPartsVector server_parts;
if (cloud_merge_tree->getInMemoryMetadataPtr()->hasUniqueKey())
server_parts = createBasePartAndDeleteBitmapFromModelsForSend<IMergeTreeMutableDataPartPtr>(
*cloud_merge_tree, data.server_parts(), data.server_part_bitmaps());
else
server_parts = createPartVectorFromModelsForSend<IMergeTreeMutableDataPartPtr>(*cloud_merge_tree, data.server_parts());
auto server_parts_size = server_parts.size();
if (query_resource.has_disk_cache_mode())
{
auto disk_cache_mode = SettingFieldDiskCacheModeTraits::fromString(query_resource.disk_cache_mode());
if (disk_cache_mode != DiskCacheMode::AUTO)
{
for (auto & part : server_parts)
part->disk_cache_mode = disk_cache_mode;
}
}
cloud_merge_tree->receiveDataParts(std::move(server_parts));
LOG_DEBUG(
log,
"Received {} parts for table {}(txn_id: {}), disk_cache_mode {}, is_dict: {}, lazy_load_parts: {}",
server_parts_size,
cloud_merge_tree->getStorageID().getNameForLogs(),
query_resource.txn_id(),
query_resource.disk_cache_mode(),
is_dict_table,
lazy_load_parts);
}
if (!data.virtual_parts().empty())
{
MergeTreeMutableDataPartsVector virtual_parts;
if (cloud_merge_tree->getInMemoryMetadataPtr()->hasUniqueKey())
virtual_parts = createBasePartAndDeleteBitmapFromModelsForSend<IMergeTreeMutableDataPartPtr>(
*cloud_merge_tree, data.virtual_parts(), data.virtual_part_bitmaps());
else
virtual_parts
= createPartVectorFromModelsForSend<IMergeTreeMutableDataPartPtr>(*cloud_merge_tree, data.virtual_parts());
auto virtual_parts_size = virtual_parts.size();
if (query_resource.has_disk_cache_mode())
{
auto disk_cache_mode = SettingFieldDiskCacheModeTraits::fromString(query_resource.disk_cache_mode());
if (disk_cache_mode != DiskCacheMode::AUTO)
{
for (auto & part : virtual_parts)
part->disk_cache_mode = disk_cache_mode;
}
}
cloud_merge_tree->receiveVirtualDataParts(std::move(virtual_parts));
LOG_DEBUG(
log,
"Received {} virtual parts for table {}(txn_id: {}), disk_cache_mode {}, is_dict: {}, lazy_load_parts: {}",
virtual_parts_size,
cloud_merge_tree->getStorageID().getNameForLogs(),
query_resource.txn_id(),
query_resource.disk_cache_mode(),
is_dict_table,
lazy_load_parts);
}
std::set<Int64> required_bucket_numbers;
for (const auto & bucket_number : data.bucket_numbers())
required_bucket_numbers.insert(bucket_number);
cloud_merge_tree->setRequiredBucketNumbers(required_bucket_numbers);
for (const auto & mutation_str : data.cnch_mutation_entries())
{
auto mutation_entry = CnchMergeTreeMutationEntry::parse(mutation_str);
cloud_merge_tree->addMutationEntry(mutation_entry);
}
/// prepareDataPartsForRead/loadDataParts is an expensive action as it may involve remote read.
/// The worker rpc thread pool may be blocked when there are many `sendResources` requests.
/// lazy_load_parts means the storage just receives server_parts in rpc. And it will call `prepareDataPartsForRead` later (before reading).
/// One exception is StorageDictCloudMergeTree as it use a different read logic rather than StorageCloudMergeTree::read.
if (!lazy_load_parts || is_dict_table)
{
cloud_merge_tree->prepareDataPartsForRead();
}
}
else if (auto * hive_table = dynamic_cast<StorageCloudHive *>(storage.get()))
{
auto settings = hive_table->getSettings();
auto lake_scan_infos = ILakeScanInfo::deserialize(data.lake_scan_info_parts(), context, storage->getInMemoryMetadataPtr(), *settings);
hive_table->loadLakeScanInfos(lake_scan_infos);
}
else if (auto * cloud_file_table = dynamic_cast<IStorageCloudFile *>(storage.get()))
{
auto data_parts = createCnchFileDataParts(session_context, data.file_parts());
cloud_file_table->loadDataParts(data_parts);
LOG_DEBUG(
log,
"Received and loaded {} cloud file parts for table {}",
data_parts.size(),
cloud_file_table->getStorageID().getNameForLogs());
}
else
throw Exception("Unknown table engine: " + storage->getName(), ErrorCodes::UNKNOWN_TABLE);
}
std::unordered_map<String, UInt64> udf_infos;
for (const auto & udf_info : query_resource.udf_infos())
{
udf_infos.emplace(udf_info.function_name(), udf_info.version());
LOG_DEBUG(log, "Received UDF meta data from server, name: {}, version: {}", udf_info.function_name(), udf_info.version());
}
watch.stop();
LOG_INFO(log, "Load all resources for session {} in {} us.", query_resource.txn_id(), watch.elapsedMicroseconds());
ProfileEvents::increment(ProfileEvents::QueryLoadResourcesMicroseconds, watch.elapsedMicroseconds());
}
template <typename T>
static void prepareQueryResource(
T & query_resource,
const WorkerId & worker_id,
const std::vector<AssignedResource> & worker_resources,
const ContextPtr & context,
bool send_mutations,
LoggerPtr & log)
{
query_resource.set_txn_id(context->getCurrentTransactionID());
query_resource.set_primary_txn_id(context->getCurrentTransaction()->getPrimaryTransactionID());
const auto & settings = context->getSettingsRef();
auto max_execution_time = settings.max_execution_time.value.totalSeconds();
/// recycle_timeout refers to the time when the session is recycled under abnormal case,
/// so it should be larger than max_execution_time to make sure the session is not to be destroyed in advance.
UInt64 recycle_timeout = max_execution_time > 0 ? max_execution_time + 60UL : 3600;
query_resource.set_timeout(recycle_timeout);
if (!settings.session_timezone.value.empty())
query_resource.set_session_timezone(settings.session_timezone.value);
if (settings.enable_lazy_load_data_parts.value)
query_resource.set_lazy_load_data_parts(true);
bool require_worker_info = false;
for (const auto & resource : worker_resources)
{
if (!resource.sent_create_query)
{
const auto & def = resource.table_definition;
if (resource.table_definition.cacheable)
{
auto * cacheable = query_resource.add_cacheable_create_queries();
RPCHelpers::fillStorageID(resource.storage->getStorageID(), *cacheable->mutable_storage_id());
cacheable->set_definition(def.definition);
if (!resource.object_columns.empty())
cacheable->set_dynamic_object_column_schema(resource.object_columns.toString());
cacheable->set_local_engine_type(static_cast<UInt32>(def.engine_type));
cacheable->set_local_table_name(def.local_table_name);
}
else
{
query_resource.add_create_queries(def.definition);
query_resource.add_dynamic_object_column_schema(resource.object_columns.toString());
}
}
/// parts
auto & table_data_parts = *query_resource.mutable_data_parts()->Add();
/// Send storage's mutations to worker if needed.
if (send_mutations)
{
auto * cnch_merge_tree = dynamic_cast<StorageCnchMergeTree *>(resource.storage.get());
if (cnch_merge_tree)
{
for (auto const & mutation_str : cnch_merge_tree->getPlainMutationEntries())
{
LOG_TRACE(log, "Send mutations to worker: {}", mutation_str);
table_data_parts.add_cnch_mutation_entries(mutation_str);
}
}
}
table_data_parts.set_database(resource.storage->getDatabaseName());
table_data_parts.set_table(resource.table_definition.local_table_name);
if (resource.table_version)
{
require_worker_info = true;
table_data_parts.set_table_version(resource.table_version);
}
if (settings.query_dry_run_mode != QueryDryRunMode::SKIP_SEND_PARTS)
{
if (!resource.server_parts.empty())
{
// todo(jiashuo): bitmap need handler?
fillBasePartAndDeleteBitmapModels(
*resource.storage,
resource.server_parts,
*table_data_parts.mutable_server_parts(),
*table_data_parts.mutable_server_part_bitmaps());
}
if (!resource.virtual_parts.empty())
{
fillPartsModelForSend(*resource.storage, resource.virtual_parts, *table_data_parts.mutable_virtual_parts());
auto * bitmaps_model = table_data_parts.mutable_virtual_part_bitmaps();
for (const auto & virtual_part : resource.virtual_parts)
{
for (auto & bitmap_meta : virtual_part->part->delete_bitmap_metas)
{
bitmaps_model->Add()->CopyFrom(*bitmap_meta);
}
}
}
}
if (!resource.lake_scan_info_parts.empty())
{
auto * mutable_lake_scan_infos = table_data_parts.mutable_lake_scan_info_parts();
auto & cnch_lake = dynamic_cast<StorageCnchLakeBase &>(*resource.storage);
cnch_lake.serializeLakeScanInfos(*mutable_lake_scan_infos, resource.lake_scan_info_parts);
}
if (!resource.file_parts.empty())
{
fillCnchFilePartsModel(resource.file_parts, *table_data_parts.mutable_file_parts());
}
/// bucket numbers
for (const auto & bucket_num : resource.bucket_numbers)
*table_data_parts.mutable_bucket_numbers()->Add() = bucket_num;
}
// need add worker info if query by table version
if (require_worker_info)
{
auto current_wg = context->getCurrentWorkerGroup();
auto * worker_info = query_resource.mutable_worker_info();
worker_info->set_worker_id(worker_id.id);
worker_info->set_index(current_wg->getWorkerIndex(worker_id.id));
worker_info->set_num_workers(current_wg->workerNum());
if (worker_info->num_workers() <= worker_info->index())
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Invalid worker index {} for worker group {}, which contains {} workers.",
toString(worker_info->index()),
current_wg->getVWName(),
toString(current_wg->workerNum()));
}
query_resource.set_disk_cache_mode(context->getSettingsRef().disk_cache_mode.toString());
}
}

View File

@ -206,6 +206,12 @@ void ColumnMap::insert(const Field & x)
getOffsets().push_back(getOffsets().back() + size);
}
void ColumnMap::insertFrom(const IColumn & src_, size_t n)
{
const ColumnMap & src = assert_cast<const ColumnMap &>(src_);
nested->insertFrom(*src.nested, n);
}
void ColumnMap::insertDefault()
{
nested->insertDefault();

View File

@ -120,6 +120,7 @@ public:
StringRef getDataAt(size_t n) const override;
void insertData(const char * pos, size_t length) override;
void insert(const Field & x) override;
void insertFrom(const IColumn & src_, size_t n) override;
void insertDefault() override;
void popBack(size_t n) override;
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
@ -214,6 +215,9 @@ public:
ColumnPtr createEmptyImplicitColumn() const;
ColumnPtr compress() const override;
/// Map can be inside Nullable, but in storage all nullable map column will be KV Map.
bool canBeInsideNullable() const override { return true; }
};
}

View File

@ -73,3 +73,13 @@ private:
std::atomic<const DateLUTImpl *> default_impl;
};
inline UInt64 timeInSeconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
{
return std::chrono::duration_cast<std::chrono::seconds>(timepoint.time_since_epoch()).count();
}
inline UInt64 timeInMicroseconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
{
return std::chrono::duration_cast<std::chrono::microseconds>(timepoint.time_since_epoch()).count();
}

View File

@ -815,6 +815,7 @@
M(3010, TOO_MANY_PLAN_SEGMENTS) \
M(3011, NO_AVAILABLE_MATERIALIZED_VIEW) \
M(3012, TOO_MANY_JOINS) \
M(3014, OPTIMIZER_TEST_FAILED) \
\
M(4001, RESOURCE_GROUP_ILLEGAL_CONFIG) \
M(4002, RESOURCE_NOT_ENOUGH) \
@ -854,6 +855,7 @@
M(5046, DISK_CACHE_NOT_USED) \
M(5047, WORKER_RESTARTED) \
M(5048, WORKER_NODE_NOT_FOUND) \
M(5049, WORKER_TABLE_NOT_FOUND) \
\
M(5453, HDFS_FILE_SYSTEM_UNREGISTER) \
M(5454, BAD_HDFS_META_FILE) \

View File

@ -23,6 +23,7 @@
#include <string.h>
#include <cxxabi.h>
#include <atomic>
#include <cstdlib>
#include <Poco/String.h>
#include <common/getFQDNOrHostName.h>
@ -699,9 +700,14 @@ void ExceptionHandler::throwIfException()
std::rethrow_exception(first_exception);
}
bool ExceptionHandler::testException()
{
bool expected_value = false;
return has_exception.compare_exchange_strong(expected_value, true);
}
bool ExceptionHandler::hasException() const
{
std::unique_lock lock(mutex);
return first_exception != nullptr;
return has_exception;
}
}

View File

@ -21,6 +21,7 @@
#pragma once
#include <atomic>
#include <cerrno>
#include <unordered_set>
#include <vector>
@ -266,10 +267,12 @@ class ExceptionHandler
public:
bool setException(std::exception_ptr && exception);
void throwIfException();
bool testException();
bool hasException() const;
protected:
std::exception_ptr first_exception;
std::atomic<bool> has_exception{false};
mutable std::mutex mutex;
};

235
src/Common/InjectPause.cpp Normal file
View File

@ -0,0 +1,235 @@
#include "Common/Logger.h"
#include <Common/InjectPause.h>
#include <common/logger_useful.h>
#include <folly/Indestructible.h>
#include <folly/Synchronized.h>
#include <folly/fibers/Baton.h>
#include <folly/logging/xlog.h>
#include <glog/logging.h>
#include <condition_variable>
#include <deque>
#include <mutex>
#include <string>
namespace DB
{
struct BatonSet
{
std::deque<folly::fibers::Baton *> batons;
std::condition_variable cv;
bool enabled{false};
size_t limit{0};
std::shared_ptr<PauseCallback> callback;
};
using PausePointsMap = folly::Synchronized<std::unordered_map<std::string, BatonSet>, std::mutex>;
PausePointsMap & pausePoints()
{
static folly::Indestructible<PausePointsMap> pause_points;
return *pause_points;
}
size_t wakeupBatonSet(BatonSet & batonSet, size_t numThreads)
{
size_t num_waked = 0;
if (numThreads == 0)
{
numThreads = batonSet.batons.size();
}
while (!batonSet.batons.empty() && num_waked < numThreads)
{
auto * b = batonSet.batons.front();
batonSet.batons.pop_front();
b->post();
num_waked++;
}
return num_waked;
}
bool & injectPauseEnabled()
{
static bool enabled = false;
return enabled;
}
// Flag controls the debug logging
bool & injectPauseLogEnabled()
{
static bool enabled = false;
return enabled;
}
void injectPause(folly::StringPiece name)
{
if (!injectPauseEnabled())
{
return;
}
folly::fibers::Baton baton;
std::shared_ptr<PauseCallback> callback;
{
auto ptr = pausePoints().lock();
auto it = ptr->find(name.str());
if (it == ptr->end() || !it->second.enabled || (it->second.limit > 0 && it->second.batons.size() == it->second.limit))
{
if (injectPauseLogEnabled())
{
LOG_ERROR(getLogger("InjectPause"), "[{}] injectPause not set", name);
}
return;
}
if (injectPauseLogEnabled())
{
LOG_ERROR(getLogger("InjectPause"), "[{}] injectPause begin", name);
}
callback = it->second.callback;
if (!callback)
{
it->second.batons.push_back(&baton);
it->second.cv.notify_one();
}
}
if (callback)
{
(*callback)();
}
else
{
baton.wait();
}
/* Avoid potential protect-its-own-lifetime bug:
Wait for the post() to finish before destroying the Baton. */
auto ptr = pausePoints().lock();
if (injectPauseLogEnabled())
{
LOG_ERROR(getLogger("InjectPause"), "[{}] injectPause end", name);
}
}
void injectPauseSet(folly::StringPiece name, size_t numThreads)
{
if (!injectPauseEnabled())
{
return;
}
auto ptr = pausePoints().lock();
if (injectPauseLogEnabled())
{
LOG_ERROR(getLogger("InjectPause"), "[{}] injectPauseSet threads {}", name, numThreads);
}
auto res = ptr->emplace(std::piecewise_construct, std::make_tuple(name.str()), std::make_tuple());
res.first->second.limit = numThreads;
res.first->second.enabled = true;
}
void injectPauseSet(folly::StringPiece name, PauseCallback && callback)
{
if (!injectPauseEnabled())
{
return;
}
auto ptr = pausePoints().lock();
if (injectPauseLogEnabled())
{
LOG_ERROR(getLogger("InjectPause"), "[{}] injectPauseSet callback", name);
}
auto res = ptr->emplace(std::piecewise_construct, std::make_tuple(name.str()), std::make_tuple());
res.first->second.limit = 0;
res.first->second.enabled = true;
res.first->second.callback = std::make_shared<PauseCallback>(std::move(callback));
}
bool injectPauseWait(folly::StringPiece name, size_t numThreads, bool wakeup, uint32_t timeoutMs)
{
if (!injectPauseEnabled() || !numThreads)
{
return false;
}
auto ptr = pausePoints().lock();
if (injectPauseLogEnabled())
{
LOG_ERROR(getLogger("InjectPause"), "[{}] injectPauseWait begin {}", name, numThreads);
}
auto it = ptr->find(name.str());
if (it == ptr->end() || !it->second.enabled)
{
if (injectPauseLogEnabled())
{
LOG_ERROR(getLogger("InjectPause"), "[{}] injectPauseWait: ERROR not set", name);
}
return false;
}
if (!timeoutMs || timeoutMs > kInjectPauseMaxWaitTimeoutMs)
{
timeoutMs = kInjectPauseMaxWaitTimeoutMs;
}
auto & baton_set = it->second;
bool status = baton_set.cv.wait_for(ptr.as_lock(), std::chrono::milliseconds(timeoutMs), [&baton_set, numThreads]() {
return !baton_set.enabled || baton_set.batons.size() >= numThreads;
});
if (status && wakeup)
{
wakeupBatonSet(baton_set, numThreads);
}
if (injectPauseLogEnabled())
{
std::string err_str;
if (!status)
{
err_str = fmt::format(" with ERR (paused {})", baton_set.batons.size());
}
LOG_ERROR(getLogger("InjectPause"), "[{}] injectPauseWait end {}", name, err_str);
}
return status;
}
size_t injectPauseClear(folly::StringPiece name)
{
if (!injectPauseEnabled())
{
return false;
}
size_t num_paused = 0;
auto ptr = pausePoints().lock();
if (injectPauseLogEnabled())
{
LOG_ERROR(getLogger("InjectPause"), "[{}] injectPauseClear ", name);
}
if (name.empty())
{
for (auto & it : *ptr)
{
auto & baton_set = it.second;
num_paused += wakeupBatonSet(baton_set, 0);
baton_set.enabled = false;
baton_set.cv.notify_all();
}
}
else
{
auto it = ptr->find(name.str());
if (it != ptr->end())
{
auto & baton_set = it->second;
num_paused += wakeupBatonSet(baton_set, 0) > 0 ? 1 : 0;
baton_set.enabled = false;
baton_set.cv.notify_all();
}
}
return num_paused;
}
}

55
src/Common/InjectPause.h Normal file
View File

@ -0,0 +1,55 @@
#pragma once
#include <folly/Function.h>
#include <folly/Range.h>
#include <common/scope_guard.h>
namespace DB
{
void injectPause(folly::StringPiece name);
/**
* Allow stopping the process at this named point.
*/
#define INJECT_PAUSE(name) DB::injectPause(#name)
#define ENABLE_INJECT_PAUSE_IN_SCOPE() \
injectPauseEnabled() = true; \
SCOPE_EXIT( { EXPECT_EQ(injectPauseClear(), 0); } )
// Callback that can be executed optionally for INJECT_PAUSE point
using PauseCallback = folly::Function<void()>;
// Default timeout for INJECT_PAUSE wait
constexpr uint32_t kInjectPauseMaxWaitTimeoutMs = 60000;
/**
* Toggle INJECT_PAUSE() logic on and off
*/
bool & injectPauseEnabled();
/**
* Stop any thread at this named INJECT_PAUSE point from now on.
*/
void injectPauseSet(folly::StringPiece name, size_t numThreads = 0);
/**
* The thread will execute the callback at INJECT_PAUSE point and go.
*/
void injectPauseSet(folly::StringPiece name, PauseCallback && callback);
/**
* If the named INJECT_PAUSE point was set, blocks until the requested
* number of threads is stopped at it including those already stopped
* and returns true.
*/
bool injectPauseWait(folly::StringPiece name, size_t numThreads = 1, bool wakeup = true, uint32_t timeoutMs = 0);
/**
* Stop blocking threads at this INJECT_PAUSE point and unblock any
* currently waiting threads. If name is not given, all of
* INJECT_PAUSE points are cleared.
* Returns the number of INJECT_PAUSE points where threads were stopped
*/
size_t injectPauseClear(folly::StringPiece name = "");
}

View File

@ -314,7 +314,7 @@
M(PerfFilterElapsedMicroseconds, "") \
\
M(QueryCreateTablesMicroseconds, "") \
M(QuerySendResourcesMicroseconds, "") \
M(QueryLoadResourcesMicroseconds, "") \
M(CloudTableDefinitionCacheHits, "") \
M(CloudTableDefinitionCacheMisses, "") \
\
@ -1216,6 +1216,7 @@
\
M(NumberOfMarkRangesBeforeBeMergedInPKFilter, "Number of mark ranges in primary index filtering before adjacent ranges be merged into more bigger ranges") \
M(PlanSegmentInstanceRetry, "How many times this plan segment has been retried, only valid under bsp mode") \
M(QueryBspRetryCount, "How many times the whole query has retried, only valid for server") \
\
M(OrcTotalStripes, "Total Stripes") \
M(OrcReadStripes, "Total Read Stripes") \

View File

@ -53,6 +53,10 @@ public:
* Pass CLOCK_MONOTONIC_COARSE, if you need better performance with acceptable cost of several milliseconds of inaccuracy.
*/
Stopwatch(clockid_t clock_type_ = STOPWATCH_DEFAULT_CLOCK) : clock_type(clock_type_) { start(); }
explicit Stopwatch(clockid_t clock_type_, UInt64 start_nanoseconds, bool is_running_)
: start_ns(start_nanoseconds), clock_type(clock_type_), is_running(is_running_)
{
}
void start() { start_ns = nanoseconds(); is_running = true; }
void stop() { stop_ns = nanoseconds(); is_running = false; }
@ -68,6 +72,8 @@ public:
}
double elapsedSeconds() const { return static_cast<double>(elapsedNanoseconds()) / 1000000000ULL; }
UInt64 getStart() { return start_ns; }
private:
UInt64 start_ns = 0;
UInt64 stop_ns = 0;

View File

@ -101,7 +101,7 @@ public:
std::vector<UInt64> thread_ids;
static ThreadGroupStatusPtr createForBackgroundProcess(ContextPtr storage_context);
/// The first thread created this thread group
UInt64 master_thread_id = 0;

View File

@ -17,7 +17,7 @@ struct WorkerId
inline bool operator==(WorkerId const & rhs) const
{
return (this->vw_name == rhs.vw_name && this->wg_name == wg_name && this->id == id);
return vw_name == rhs.vw_name && wg_name == rhs.wg_name && id == rhs.id;
}
};

View File

@ -282,19 +282,11 @@ enum PreloadLevelSettings : UInt64
"consumed. 0 means - same as 'max_threads'.", \
0) \
M(Bool, enable_positional_arguments, true, "Enable positional arguments in ORDER BY, GROUP BY and LIMIT BY", 0) \
\
M(Bool, \
enable_extended_results_for_datetime_functions, \
false, \
"Enable date functions like toLastDayOfMonth return Date32 results (instead of Date results) for Date32/DateTime64 arguments.", \
0) \
\
M(UInt64, \
max_parallel_replicas, \
1, \
"The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same " \
"partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", \
0) \
M(Bool, enable_order_by_all, false, "Enable sorting expression ORDER BY ALL.", 0)\
\
M(Bool, enable_extended_results_for_datetime_functions, false, "Enable date functions like toLastDayOfMonth return Date32 results (instead of Date results) for Date32/DateTime64 arguments.", 0) \
\
M(UInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \
M(UInt64, parallel_replicas_count, 0, "", 0) \
M(UInt64, parallel_replica_offset, 0, "", 0) \
\
@ -1104,6 +1096,7 @@ enum PreloadLevelSettings : UInt64
\
M(UInt64, limit, 0, "Limit on read rows from the most 'end' result for select query, default 0 means no limit length", 0) \
M(UInt64, offset, 0, "Offset on read rows from the most 'end' result for select query", 0) \
M(Int64, final_order_by_all_direction, 0, "Sorting the most 'end' result for select query, default 0 means no sorting, > 1 for ASC, < -1 for DESC", 0) \
\
M(UInt64, function_range_max_elements_in_block, 500000000, "Maximum number of values generated by function 'range' per block of data (sum of array sizes for every row in a block, see also 'max_block_size' and 'min_insert_block_size_rows'). It is a safety threshold.", 0) \
M(ShortCircuitFunctionEvaluation, short_circuit_function_evaluation, ShortCircuitFunctionEvaluation::ENABLE, "Setting for short-circuit function evaluation configuration. Possible values: 'enable' - use short-circuit function evaluation for functions that are suitable for it, 'disable' - disable short-circuit function evaluation, 'force_enable' - use short-circuit function evaluation for all functions.", 0) \
@ -1195,6 +1188,7 @@ enum PreloadLevelSettings : UInt64
M(Bool, create_view_check_column_names, true, "When executing CREATE VIEW queries, whether check column names are consistent with select query", 0) \
M(Bool, rewrite_unknown_left_join_identifier, true, "Whether to rewrite unknown left join identifier, this is a deprecated feature but Aeolus SQL depends on it", 0) \
M(Bool, allow_mysql_having_name_resolution, false, "Whether to use MySQL special name resolution rules for HAVING clauses ", 0) \
M(Bool, prefer_alias_if_column_name_is_ambiguous, false, "If source columns are ambiguous, prefer to use alias, for MySQL compatibility", 0) \
M(String, access_table_names, "", "Session level restricted tables query can access", 0) \
M(String, accessible_table_names, "", "Session level restricted tables query can access", 0) \
\
@ -1252,6 +1246,7 @@ enum PreloadLevelSettings : UInt64
M(Bool, force_grouping_standard_compatibility, true, "Make GROUPING function to return 1 when argument is not used as an aggregation key", 0) \
M(Bool, disable_optimize_final, true, "Disable optimize final command", 0) \
M(Milliseconds, brpc_data_parts_timeout_ms, 30000, "Timeout for transmitting data parts in brpc", 0) \
M(Bool, enable_lazy_load_data_parts, false, "Trigger loadDataParts when worker actual use the cloud table, but not when worker receive resources. This significantly reduce the cost on worker brpc threads.", 0) \
M(UInt64, scan_all_table_threshold, 20, "The upper limit to avoid scan all tables in some system tables, like tables and cnch_tables.", 0) \
M(Seconds, cnch_txn_lock_expire_duration_seconds, 30, "Transaction lock expire duration.", 0) \
M(Seconds, cnch_lock_manager_txn_checker_schedule_seconds, 30, "LockManager txn checker schedule seconds.", 0) \
@ -1457,6 +1452,7 @@ enum PreloadLevelSettings : UInt64
M(Bool, enable_column_pruning, true, "Whether to enable ColumnPruning", 0) \
M(Bool, enable_add_projection_to_pruning, true, "Whether add projection when column pruning", 0) \
M(Bool, enable_predicate_pushdown_rewrite, true, "Whether to enable PredicatePushdown", 0) \
M(Bool, enable_pushdown_filter_through_stateful, false, "Whether to enable push predicate through projection with stateful functions", 0) \
M(Bool, enable_hints_propagator, true, "Whether to enable HintsPropagator", 0) \
M(Bool, enable_join_algorithm_hints, true, "Whether to enable ImplementJoinAlgorithmHints", 0) \
M(Bool, enable_join_operation_hints, true, "Whether to enable ImplementJoinOperationHints", 0) \
@ -1521,6 +1517,8 @@ enum PreloadLevelSettings : UInt64
M(Bool, enable_push_partial_limit_through_exchange, true, "Whether to enable PushPartialLimitThroughExchange rules", 0) \
M(Bool, enable_push_partial_distinct_through_exchange, true, "Whether to enable PushPartialDistinctThroughExchange rules", 0) \
M(Bool, enable_push_partial_topn_distinct_through_exchange, true, "Whether to enable PushPartialTopNDistinctThroughExchange rules", 0) \
M(Bool, enable_push_projection_through_exchange, true, "Whether to enable PushProjectionThroughExchange rules", 0) \
M(Bool, enable_split_countd_to_state_merge, false, "Whether to enable split count distinct to state and merge", 0) \
M(UInt64, max_rows_to_use_topn_filtering, 0, "The maximum N of TopN to use topn filtering optimization. Set 0 to choose this value adaptively.", 0) \
M(String, topn_filtering_algorithm_for_unsorted_stream, "SortAndLimit", "The default topn filtering algorithm for unsorted stream, can be one of: 'SortAndLimit', 'Heap'", 0) \
M(Bool, enable_create_topn_filtering_for_aggregating, false, "Whether to enable CreateTopNFilteringForAggregating rules", 0) \
@ -1631,6 +1629,7 @@ enum PreloadLevelSettings : UInt64
M(Bool, enable_join_graph_support_filter, true, "Whether enable join graph support filter", 0) \
M(Bool, enable_equivalences, true, "Whether enable using equivalences when property match", 0) \
M(Bool, enable_injective_in_property, false, "Whether enable using injective function when property match", 0) \
M(Bool, enable_case_when_prop, false, "Whether enable case when prop", 0) \
M(UInt64, max_expand_join_key_size, 3, "Whether enable using equivalences when property match", 0) \
M(UInt64, max_expand_agg_key_size, 3, "Max allowed agg/window keys number when expand powerset when property match", 0) \
M(Bool, enable_sharding_optimize, false, "Whether enable sharding optimization, eg. local join", 0) \
@ -1698,6 +1697,7 @@ enum PreloadLevelSettings : UInt64
M(UInt64, exchange_remote_receiver_queue_size, 10, "Queue size for remote exchange receiver",0) \
M(UInt64, exchange_multi_path_receiver_queue_size, 20, "Queue size for multi path exchange receiver", 0) \
M(Bool, exchange_enable_block_compress, true, "Whether enable exchange block compress ", 0) \
M(UInt64, exchange_stream_back_pressure_max_wait_ms, 0, "Default 0, 0 means no control", 0) \
M(UInt64, exchange_stream_max_buf_size, 20971520, "Default 20M, -1 means no limit", 0) \
M(UInt64, exchange_buffer_send_threshold_in_bytes, 1000000, "The minimum bytes when exchange will flush send buffer ", 0) \
M(UInt64, exchange_buffer_send_threshold_in_row, 65505, "The minimum row num when exchange will flush send buffer", 0) \
@ -1715,6 +1715,8 @@ enum PreloadLevelSettings : UInt64
M(Bool, exchange_use_query_memory_tracker, true, "Use query-level memory tracker", 0) \
M(String, exchange_shuffle_method_name, "cityHash64V2", "Shuffle method name used in exchange", 0) \
M(Bool, enable_batch_send_plan_segment, true, "Whether enable combined sending plan segments to reduce rpc calls", 0) \
M(Bool, enable_batch_send_resources_together, false, "Whether enable combined sending resources and plan segments to reduce rpc calls", 0) \
M(Bool, enable_clean_resources_by_worker, false, "Whether enable worker clean resources by itself", 0) \
M(UInt64, wait_for_post_processing_timeout_ms, 1000, "Timeout for waiting post processing rpc from workers.", 0) \
M(UInt64, distributed_query_wait_exception_ms, 2000,"Wait final planSegment exception from segmentScheduler.", 0) \
M(UInt64, distributed_max_parallel_size, false, "Max distributed execution parallel size", 0) \
@ -1766,6 +1768,7 @@ enum PreloadLevelSettings : UInt64
\
/** S3 Storage settings*/ \
M(UInt64, s3_gc_inter_partition_parallelism, 4, "Partition level concurrency when gc s3 table", 0) \
M(UInt64, s3_gc_batch_size, 10000, "Batch size for each thread when gc s3 table", 0) \
M(UInt64, s3_gc_intra_partition_parallelism, 16, "Part level concurrency when gc s3 table", 0) \
\
M(UInt64, \

View File

@ -23,6 +23,9 @@ set(CLICKHOUSE_DAEMONMANAGER_SOURCE
add_library(daemon_manager_lib ${CLICKHOUSE_DAEMONMANAGER_SOURCE})
target_include_directories(daemon_manager_lib SYSTEM BEFORE PUBLIC ${ClickHouse_SOURCE_DIR}/libs/libdaemon/include)
target_include_directories(daemon_manager_lib SYSTEM BEFORE PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/libfarmhash/)
target_include_directories(daemon_manager_lib SYSTEM BEFORE PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/libmetrohash/src)
target_include_directories(daemon_manager_lib SYSTEM BEFORE PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/murmurhash/include)
target_link_libraries(daemon_manager_lib daemon dbms)
# add_subdirectory(tests)

View File

@ -389,12 +389,13 @@ bool DaemonJobGlobalGC::executeImpl()
if (deleting_uuids_from_servers.find(table_id.uuid()) != deleting_uuids_from_servers.end())
continue;
String fail_reason;
auto table_model = DB::GlobalGCHelpers::getCleanableTrashTable(context, table_id, ts, retention_sec, &fail_reason);
if (table_model.has_value()) {
String failure_reason;
std::optional<DB::Protos::DataModelTable> table_model = DB::GlobalGCHelpers::getCleanableTrashTable(context, table_id, ts, retention_sec, &failure_reason);
if (table_model.has_value())
{
this->tables_need_gc.push_back(*table_model);
} else {
LOG_INFO(log, "Cannot clean trash table {} because : {}", table_id.name(), fail_reason);
LOG_TRACE(log, "Table {} is not cleanable, skip. reason: {}", table_id.uuid(), failure_reason);
}
}

View File

@ -52,7 +52,7 @@ public:
return "Array";
}
/// Map support array key
/// Map support array value
bool canBeInsideNullable() const override
{
return true;
@ -88,6 +88,8 @@ public:
/// 1 for plain array, 2 for array of arrays and so on.
size_t getNumberOfDimensions() const;
bool hasNestedMap() const override { return nested->isMap() || nested->hasNestedMap(); }
bool canBeByteMapValueType() const override { return nested->canBeByteMapValueType(); }
private:

View File

@ -78,6 +78,8 @@ public:
static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type);
static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys);
bool hasNestedMap() const override { return dictionary_type->isMap() || dictionary_type->hasNestedMap(); }
/// Key can not be null because it's meaningless
bool canBeMapKeyType() const override { return dictionary_type->canBeMapKeyType(); }
/// Due to LowCardinality can not be inside nullable, so if dictionary_type is not nullable,

View File

@ -172,17 +172,6 @@ bool DataTypeMap::equals(const IDataType & rhs) const
return nested->equals(*rhs_map.nested);
}
bool DataTypeMap::isComparable() const
{
if (isKVMap())
return key_type->isComparable() && value_type->isComparable();
/// always return false in ByteMap to prevent ByteMap type be used as sorting key,
/// which may lead to performance issues.
/// Order by Map column in select query is always allowed, see ColumnMap::compareAt
return false;
}
bool DataTypeMap::textCanContainOnlyValidUTF8() const
{
return key_type->textCanContainOnlyValidUTF8() && value_type->textCanContainOnlyValidUTF8();

View File

@ -8,16 +8,19 @@ namespace DB
/** Map data type.
* Map is implemented as two arrays of keys and values.
* Serialization of type 'Map(K, V)' is similar to serialization.
* of 'Array(Tuple(keys K, values V))' or in other words of 'Nested(keys K, valuev V)'.
* Serialization of type 'Map(K, V)' is similar to serialization of 'Array(Tuple(keys K, values V))' or in other
* words of 'Nested(keys K, valuev V)', to distinguish from ByteMap, we call it KV Map.
*
* Map data type in ByteDance: ByteMap
* ByteMap can only be flatten mode in cnch, which will write each key to a implicit column like __map__key sparately.
*
* In memory, we use data type Map to represent both Map and ByteMap types.
* In memory, we use DataTypeMap to represent both Map and ByteMap types.
*
* In column declaration, using Map(K, V) KV to declare a Map column, Map(K, V) BYTE to declare a ByteMap column.
* In column declaration, using Map(K, V) KV to declare a KV Map column, Map(K, V) BYTE to declare a ByteMap column.
* Default serialization type is controlled by default_use_kv_map_type.
*
* Because of main difference of ByteMap and KV Map is serialization, so when DataTypeMap is nested in other types, it
* can only be Map, since enumerateStreams will be called by the outter type when reading/writing it.
*/
class DataTypeMap final : public IDataType
{
@ -39,7 +42,7 @@ public:
std::string doGetName() const override;
const char * getFamilyName() const override { return "Map"; }
bool canBeInsideNullable() const override { return false; }
bool canBeInsideNullable() const override { return true; }
DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override;
ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override;
@ -51,7 +54,8 @@ public:
Field getDefault() const override;
bool equals(const IDataType & rhs) const override;
bool isComparable() const override;
/// Byte Map can be comparable, but not suitable to be key columns in storage. See KeyDescription::getSortingKeyFromAST
bool isComparable() const override { return key_type->isComparable() && value_type->isComparable(); }
bool isParametric() const override { return true; }
bool haveSubtypes() const override { return true; }
@ -62,6 +66,8 @@ public:
bool isKVMap() const override { return isMapKVStore() || (!isMapByteStore() && getDefaultUseMapType()); }
bool isByteMap() const override { return !isKVMap(); }
bool hasNestedMap() const override { return nested->hasNestedMap(); }
const DataTypePtr & getKeyType() const { return key_type; }
const DataTypePtr & getValueType() const { return value_type; }
DataTypePtr getValueTypeForImplicitColumn() const;

View File

@ -62,6 +62,8 @@ public:
bool onlyNull() const override;
bool canBeInsideLowCardinality() const override { return nested_data_type->canBeInsideLowCardinality(); }
bool hasNestedMap() const override { return nested_data_type->isMap() || nested_data_type->hasNestedMap(); }
/// DataTypeNullable cannot be ByteMap value type, but we need to compatible old invalid tables,
/// this will be checked in MergeTreeMetaBase::checkMetadataValidity for newly created tables
bool canBeByteMapValueType() const override { return nested_data_type->canBeByteMapValueType(); }

View File

@ -148,6 +148,16 @@ static void addElementSafe(const DataTypes & elems, IColumn & column, F && impl)
}
}
bool DataTypeTuple::hasNestedMap() const
{
for (const auto & elem_type: elems)
{
if(elem_type->isMap() || elem_type->hasNestedMap())
return true;
}
return false;
}
MutableColumnPtr DataTypeTuple::createColumn() const
{
size_t size = elems.size();

View File

@ -37,6 +37,8 @@ public:
bool canBeInsideNullable() const override { return false; }
bool hasNestedMap() const override;
MutableColumnPtr createColumn() const override;
Field getDefault() const override;

View File

@ -108,6 +108,8 @@ void registerDataTypeNumbers(DataTypeFactory & factory)
factory.registerAlias("DOUBLE PRECISION", "Float64", DataTypeFactory::CaseInsensitive);
/// MySQL
factory.registerAlias("SIGNED INTEGER", "Int32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("UNSIGNED INTEGER", "UInt32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("TINYINT SIGNED", "Int8", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INT1 SIGNED", "Int8", DataTypeFactory::CaseInsensitive);
factory.registerAlias("SMALLINT SIGNED", "Int16", DataTypeFactory::CaseInsensitive);

View File

@ -354,6 +354,9 @@ public:
/// If this is byte map type
virtual bool isByteMap() const { return false; }
/// If nested data type is or contains map type
virtual bool hasNestedMap() const { return false; }
/// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column.
static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint);

View File

@ -223,7 +223,7 @@ size_t SerializationNumber<T>::deserializeBinaryBulk(IColumn & column, ReadBuffe
if (zero_copy_cache_read && vec_col.has_zero_buf && !filter)
{
size_t init_col_size = column.size();
if (auto * merged_segment_istr = typeid_cast<MergedReadBufferWithSegmentCache *>(&istr); merged_segment_istr->isInternalCachedCompressedReadBuffer())
if (auto * merged_segment_istr = typeid_cast<MergedReadBufferWithSegmentCache *>(&istr); merged_segment_istr && merged_segment_istr->isInternalCachedCompressedReadBuffer())
{
bool incomplete_read = false;
size_t size = merged_segment_istr->readZeroCopy(vec_col.getZeroCopyBuf(), sizeof(typename ColumnVector<T>::ValueType) * limit, incomplete_read);

View File

@ -59,9 +59,6 @@ DiskSelector::DiskSelector(const Poco::Util::AbstractConfiguration & config, con
constexpr auto default_disk_name = "default";
bool has_default_disk = false;
#if USE_CLOUDFS
Strings cloudfs_disks;
#endif
for (const auto & disk_name : keys)
{
if (!std::all_of(disk_name.begin(), disk_name.end(), isWordCharASCII))
@ -71,25 +68,10 @@ DiskSelector::DiskSelector(const Poco::Util::AbstractConfiguration & config, con
has_default_disk = true;
auto disk_config_prefix = config_prefix + "." + disk_name;
#if USE_CLOUDFS
if (config.getString(disk_config_prefix + ".type", "") == "cfs")
{
cloudfs_disks.emplace_back(disk_name);
continue;
}
#endif
disks.emplace(disk_name, factory.create(disk_name, config, disk_config_prefix, context, disks));
}
/// DiskCloudFS depends on its ufs disk, thus it must be initialized after all other disks.
#if USE_CLOUDFS
for (auto & disk_name : cloudfs_disks)
{
auto disk_config_prefix = config_prefix + "." + disk_name;
disks.emplace(disk_name, factory.create(disk_name, config, disk_config_prefix, context, disks));
}
#endif
if (!has_default_disk)
disks.emplace(default_disk_name, std::make_shared<DiskLocal>(default_disk_name, context->getPath(), DiskStats{}));

View File

@ -167,7 +167,7 @@ SettingsChanges getSettingsChangesFromConfigs(PlainConfigsPtr configs)
{"lf_metastore_region", "lf_metastore_region"},
{"lf_metastore_ak_id", "lf_metastore_ak_id"},
{"lf_metastore_ak_secret", "lf_metastore_ak_secret"},
{"c", "lf_metastore_catalog"},
{"lf_metastore_catalog", "lf_metastore_catalog"},
{"aws_glue_catalog_id", "aws.glue.catalog_id"},
{"aws_glue_ak_id", "aws.glue.access_key"},
{"aws_glue_ak_secret", "aws.glue.secret_key"},

View File

@ -4,6 +4,7 @@
#include <Common/Exception.h>
#include <Common/KnownObjectNames.h>
#include <Interpreters/Context.h>
#include <Interpreters/ProcessList.h>
#include <Core/Settings.h>
#include <DataStreams/MaterializingBlockOutputStream.h>
#include <DataStreams/NativeBlockInputStream.h>
@ -382,6 +383,18 @@ InputFormatPtr FormatFactory::getInputFormat(
return format;
}
static void addExistingProgressToOutputFormat(OutputFormatPtr format, ContextPtr context)
{
auto element_id = context->getProcessListElement();
if (element_id)
{
/// community ck called format->onProgress to update the progress
/// but it leads to duplicated counting in cnch; so we skip it here.
/// Update the start of the statistics to use the start of the query, and not the creation of the format class
format->setStartTime(element_id->getQueryCPUStartTime(), true);
}
}
OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible(
const String & name,
WriteBuffer & buf,
@ -412,7 +425,9 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible(
if (context->hasQueryContext() && settings.log_queries)
context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Format, name);
return std::make_shared<ParallelFormattingOutputFormat>(builder);
auto format = std::make_shared<ParallelFormattingOutputFormat>(builder);
addExistingProgressToOutputFormat(format, context);
return format;
}
return getOutputFormat(name, buf, sample, context, callback, _format_settings);
@ -456,6 +471,8 @@ OutputFormatPtr FormatFactory::getOutputFormat(
if (auto * mysql = typeid_cast<MySQLOutputFormat *>(format.get()))
mysql->setContext(context);
addExistingProgressToOutputFormat(format, context);
return format;
}

View File

@ -1,16 +1,96 @@
#include <Formats/insertNullAsDefaultIfNeeded.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnMap.h>
#include <Columns/ColumnLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Functions/FunctionHelpers.h>
namespace DB
{
void insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const ColumnWithTypeAndName & header_column, size_t column_i, BlockMissingValues * block_missing_values)
bool insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const ColumnWithTypeAndName & header_column, size_t column_i, BlockMissingValues * block_missing_values)
{
if (isArray(input_column.type) && isArray(header_column.type))
{
ColumnWithTypeAndName nested_input_column;
const auto * array_input_column = checkAndGetColumn<ColumnArray>(input_column.column.get());
nested_input_column.column = array_input_column->getDataPtr();
nested_input_column.type = checkAndGetDataType<DataTypeArray>(input_column.type.get())->getNestedType();
ColumnWithTypeAndName nested_header_column;
nested_header_column.column = checkAndGetColumn<ColumnArray>(header_column.column.get())->getDataPtr();
nested_header_column.type = checkAndGetDataType<DataTypeArray>(header_column.type.get())->getNestedType();
if (!insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr))
return false;
input_column.column = ColumnArray::create(nested_input_column.column, array_input_column->getOffsetsPtr());
input_column.type = std::make_shared<DataTypeArray>(std::move(nested_input_column.type));
return true;
}
if (isTuple(input_column.type) && isTuple(header_column.type))
{
const auto * tuple_input_column = checkAndGetColumn<ColumnTuple>(input_column.column.get());
const auto * tuple_input_type = checkAndGetDataType<DataTypeTuple>(input_column.type.get());
const auto * tuple_header_column = checkAndGetColumn<ColumnTuple>(header_column.column.get());
const auto * tuple_header_type = checkAndGetDataType<DataTypeTuple>(header_column.type.get());
if (tuple_input_type->getElements().size() != tuple_header_type->getElements().size())
return false;
Columns nested_input_columns;
nested_input_columns.reserve(tuple_input_type->getElements().size());
DataTypes nested_input_types;
nested_input_types.reserve(tuple_input_type->getElements().size());
bool changed = false;
for (size_t i = 0; i != tuple_input_type->getElements().size(); ++i)
{
ColumnWithTypeAndName nested_input_column;
nested_input_column.column = tuple_input_column->getColumnPtr(i);
nested_input_column.type = tuple_input_type->getElement(i);
ColumnWithTypeAndName nested_header_column;
nested_header_column.column = tuple_header_column->getColumnPtr(i);
nested_header_column.type = tuple_header_type->getElement(i);
changed |= insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr);
nested_input_columns.push_back(std::move(nested_input_column.column));
nested_input_types.push_back(std::move(nested_input_column.type));
}
if (!changed)
return false;
input_column.column = ColumnTuple::create(std::move(nested_input_columns));
input_column.type = std::make_shared<DataTypeTuple>(std::move(nested_input_types));
return true;
}
if (isMap(input_column.type) && isMap(header_column.type))
{
ColumnWithTypeAndName nested_input_column;
nested_input_column.column = checkAndGetColumn<ColumnMap>(input_column.column.get())->getNestedColumnPtr();
nested_input_column.type = checkAndGetDataType<DataTypeMap>(input_column.type.get())->getNestedType();
ColumnWithTypeAndName nested_header_column;
nested_header_column.column = checkAndGetColumn<ColumnMap>(header_column.column.get())->getNestedColumnPtr();
nested_header_column.type = checkAndGetDataType<DataTypeMap>(header_column.type.get())->getNestedType();
if (!insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr))
return false;
input_column.column = ColumnMap::create(std::move(nested_input_column.column));
input_column.type = std::make_shared<DataTypeMap>(std::move(nested_input_column.type));
return true;
}
if (!isNullableOrLowCardinalityNullable(input_column.type) || isNullableOrLowCardinalityNullable(header_column.type))
return;
return false;
if (block_missing_values)
{
@ -32,6 +112,8 @@ void insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const Col
const auto * lc_type = assert_cast<const DataTypeLowCardinality *>(input_column.type.get());
input_column.type = std::make_shared<DataTypeLowCardinality>(removeNullable(lc_type->getDictionaryType()));
}
return true;
}
}

View File

@ -5,6 +5,6 @@
namespace DB
{
void insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const ColumnWithTypeAndName & header_column, size_t column_i, BlockMissingValues * block_missing_values);
bool insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const ColumnWithTypeAndName & header_column, size_t column_i, BlockMissingValues * block_missing_values);
}

View File

@ -1197,6 +1197,20 @@ public:
{
value = static_cast<NumberType>(element.getBool());
}
else if (element.isString())
{
auto rb = ReadBufferFromMemory{element.getString()};
if constexpr (std::is_floating_point_v<NumberType>)
{
if (!tryReadFloatText(value, rb) || !rb.eof())
return false;
}
else
{
if (!tryReadIntText(value, rb) || !rb.eof())
return false;
}
}
else
return false;
@ -1510,7 +1524,12 @@ struct JSONExtractTree
converted = tryConvertToDecimal<DataTypeNumber<Int64>, DataTypeDecimal<DecimalType>>(element.getInt64(), type->getScale(), result);
else if (element.isUInt64())
converted = tryConvertToDecimal<DataTypeNumber<UInt64>, DataTypeDecimal<DecimalType>>(element.getUInt64(), type->getScale(), result);
else if (element.isString())
{
auto rb = ReadBufferFromMemory{element.getString()};
if (!SerializationDecimal<DecimalType>::tryReadText(result, rb, DecimalUtils::max_precision<DecimalType>, type->getScale()))
return false;
}
if (converted)
assert_cast<ColumnDecimal<DecimalType> &>(dest).insert(result);

View File

@ -135,7 +135,12 @@ public:
DataTypes nested_types(nested_types_count);
for (size_t i = 0; i < arguments.size() - 1 - num_fixed_params; ++i)
{
const auto * array_type = checkAndGetDataType<typename Impl::data_type>(&*arguments[i + 1 + num_fixed_params]);
DataTypePtr arg_array = arguments[i + 1 + num_fixed_params];
if (useDefaultImplementationForNulls())
arg_array = removeNullable(arg_array);
const auto * array_type = checkAndGetDataType<typename Impl::data_type>(&*arg_array);
if (!array_type)
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,

View File

@ -0,0 +1,203 @@
#include <climits>
#include <utility>
#include <Functions/dtsCityHash.h>
#include <iostream>
namespace DB::DTSCityHash
{
static const Int64 k0 = 0xc3a5c85c97cb3127L;
static const Int64 k1 = 0xb492b66fbe98f273L;
static const Int64 k2 = 0x9ae16a3b2f90404fL;
static const Int64 k3 = 0xc949d7c7509e6557L;
UInt64 asUInt64(Int64 l) {
if (l < 0) {
return UINT64_MAX - static_cast<UInt64>(std::llabs(l)) + 1ULL;
}
return l;
}
static Int64 unsignedShift(Int64 val, int shift)
{
return static_cast<UInt64>(val) >> shift;
}
static Int64 toLongLE(const char* b, int i) {
return ((static_cast<Int64>(b[i + 7]) << 56) +
(static_cast<Int64>(b[i + 6] & 255) << 48) +
(static_cast<Int64>(b[i + 5] & 255) << 40) +
(static_cast<Int64>(b[i + 4] & 255) << 32) +
(static_cast<Int64>(b[i + 3] & 255) << 24) +
((b[i + 2] & 255) << 16) +
((b[i + 1] & 255) << 8) +
((b[i + 0] & 255) << 0));
}
static Int32 toIntLE(const char* b, int i) {
return (((b[i + 3] & 255) << 24) + ((b[i + 2] & 255) << 16) + ((b[i + 1] & 255) << 8)
+ ((b[i + 0] & 255) << 0));
}
static Int64 fetch64(const char* s, int pos) {
return toLongLE(s, pos);
}
static Int32 fetch32(const char* s, int pos) {
return toIntLE(s, pos);
}
static Int64 rotate(Int64 val, int shift) {
return shift == 0 ? val : unsignedShift(val, shift) | (val << (64 - shift));
}
static Int64 rotateByAtLeast1(Int64 val, int shift) {
return unsignedShift(val, shift) | (val << (64 - shift));
}
static Int64 shiftMix(Int64 val) {
return val ^ unsignedShift(val, 47);
}
static const Int64 kMul = 0x9ddfea08eb382d69LL;
static Int64 hash128to64(Int64 u, Int64 v) {
Int64 a = (u ^ v) * kMul;
a ^= unsignedShift(a, 47);
Int64 b = (v ^ a) * kMul;
b ^= unsignedShift(b, 47);
b *= kMul;
return b;
}
static Int64 hashLen16(Int64 u, Int64 v) {
return hash128to64(u, v);
}
static Int64 hashLen0to16(const char* s, int pos, int len) {
if (len > 8) {
Int64 a = fetch64(s, pos + 0);
Int64 b = fetch64(s, pos + len - 8);
return hashLen16(a, rotateByAtLeast1(b + len, len)) ^ b;
}
if (len >= 4) {
Int64 a = 0xffffffffL & fetch32(s, pos + 0);
return hashLen16((a << 3) + len, 0xffffffffL & fetch32(s, pos + len - 4));
}
if (len > 0) {
int a = s[pos + 0] & 0xFF;
int b = s[pos + unsignedShift(len, 1)] & 0xFF;
int c = s[pos + len - 1] & 0xFF;
int y = a + (b << 8);
int z = len + (c << 2);
Int64 r = shiftMix(y * k2 ^ z * k3) * k2;
return r;
}
return k2;
}
static Int64 hashLen17to32(const char* s, int pos, int len) {
Int64 a = fetch64(s, pos + 0) * k1;
Int64 b = fetch64(s, pos + 8);
Int64 c = fetch64(s, pos + len - 8) * k2;
Int64 d = fetch64(s, pos + len - 16) * k0;
return hashLen16(
rotate(a - b, 43) + rotate(c, 30) + d,
a + rotate(b ^ k3, 20) - c + len
);
}
static std::pair<Int64, Int64> weakHashLen32WithSeeds(
Int64 w, Int64 x, Int64 y, Int64 z, Int64 a, Int64 b) {
a += w;
b = rotate(b + a + z, 21);
Int64 c = a;
a += x;
a += y;
b += rotate(a, 44);
return std::make_pair(a + z, b + c );
}
static std::pair<Int64, Int64> weakHashLen32WithSeeds(const char* s, int pos, Int64 a, Int64 b) {
return weakHashLen32WithSeeds(
fetch64(s, pos + 0),
fetch64(s, pos + 8),
fetch64(s, pos + 16),
fetch64(s, pos + 24),
a,
b
);
}
static Int64 hashLen33to64(const char* s, int pos, int len) {
Int64 z = fetch64(s, pos + 24);
Int64 a = fetch64(s, pos + 0) + (fetch64(s, pos + len - 16) + len) * k0;
Int64 b = rotate(a + z, 52);
Int64 c = rotate(a, 37);
a += fetch64(s, pos + 8);
c += rotate(a, 7);
a += fetch64(s, pos + 16);
Int64 vf = a + z;
Int64 vs = b + rotate(a, 31) + c;
a = fetch64(s, pos + 16) + fetch64(s, pos + len - 32);
z = fetch64(s, pos + len - 8);
b = rotate(a + z, 52);
c = rotate(a, 37);
a += fetch64(s, pos + len - 24);
c += rotate(a, 7);
a += fetch64(s, pos + len - 16);
Int64 wf = a + z;
Int64 ws = b + rotate(a, 31) + c;
Int64 r = shiftMix((vf + ws) * k2 + (wf + vs) * k0);
return shiftMix(r * k0 + vs) * k2;
}
Int64 cityHash64(const char* s, int pos, int len)
{
if (len <= 32) {
if (len <= 16) {
return hashLen0to16(s, pos, len);
} else {
return hashLen17to32(s, pos, len);
}
} else if (len <= 64) {
return hashLen33to64(s, pos, len);
}
Int64 x = fetch64(s, pos + len - 40);
Int64 y = fetch64(s, pos + len - 16) + fetch64(s, pos + len - 56);
Int64 z = hashLen16(fetch64(s, pos + len - 48) + len, fetch64(s, pos + len - 24));
auto v = weakHashLen32WithSeeds(s, pos + len - 64, len, z);
auto w = weakHashLen32WithSeeds(s, pos + len - 32, y + k1, x);
x = x * k1 + fetch64(s, pos + 0);
len = (len - 1) & (~63);
do {
x = rotate(x + y + v.first + fetch64(s, pos + 8), 37) * k1;
y = rotate(y + v.second + fetch64(s, pos + 48), 42) * k1;
x ^= w.second;
y += v.first + fetch64(s, pos + 40);
z = rotate(z + w.first, 33) * k1;
v = weakHashLen32WithSeeds(s, pos + 0, v.second * k1, x + w.first);
w = weakHashLen32WithSeeds(s, pos + 32, z + w.second, y + fetch64(s, pos + 16));
{
Int64 swap = z;
z = x;
x = swap;
}
pos += 64;
len -= 64;
} while (len != 0);
return hashLen16(
hashLen16(v.first, w.first) + shiftMix(y) * k1 + z,
hashLen16(v.second, w.second) + x
);
}
}

View File

@ -0,0 +1,20 @@
#pragma once
#include <cstdlib>
#include <cstdint>
/// A C++ implementation for the cityHash64 used by the DTS/Spark.
/// Reimplementation of repo `spark`, branch `branch-3.2-bd`, commit `66465e1cfe95a0f1313043215344a45a0df71da6`
namespace DB::DTSCityHash
{
using Int32 = int32_t;
using Int64 = int64_t;
using UInt64 = uint64_t;
// Hash function for a byte array.
Int64 cityHash64(const char* s, int pos, int len);
UInt64 asUInt64(Int64 l);
}

View File

@ -0,0 +1,20 @@
#include <cstdlib>
#include <cstdint>
#include <Functions/hiveIntHash.h>
namespace DB::HiveIntHash
{
static const BigInteger k0 = stringToBigInteger("18397679294719823053");
static const BigInteger k1 = stringToBigInteger("14181476777654086739");
BigInteger intHash64(BigInteger x)
{
x ^= x >> 33;
x *= k0;
x ^= x >> 33;
x *= k1;
x ^= x >> 33;
return x;
}
}

View File

@ -15,8 +15,6 @@
#pragma once
#include <cstdlib>
#include <cstdint>
#include <BigIntegerLibrary.hh>
/// A C++ implementation for the intHash64 used by the Hive.
@ -24,16 +22,6 @@
namespace DB::HiveIntHash
{
static const BigInteger k0 = stringToBigInteger("18397679294719823053");
static const BigInteger k1 = stringToBigInteger("14181476777654086739");
BigInteger intHash64(BigInteger x) {
x ^= x >> 33;
x *= k0;
x ^= x >> 33;
x *= k1;
x ^= x >> 33;
return x;
}
BigInteger intHash64(BigInteger x);
};

View File

@ -415,6 +415,7 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
bool is_const = isColumnConst(*arguments[0].column);
const ColumnConst * arg1_const = typeid_cast<const ColumnConst *>(arguments[1].column.get());
const ColumnMap * col_map = typeid_cast<const ColumnMap *>(arguments[0].column.get());
//It may not be necessary to check this condition, cause it will be checked in getReturnTypeImpl function
@ -447,6 +448,7 @@ public:
ColumnsWithTypeAndName new_arguments;
ColumnPtr sub_map_column;
ColumnPtr sub_arg1_column;
DataTypePtr data_type;
if (keys_string_column)
@ -460,17 +462,18 @@ public:
data_type =std::make_shared<DataTypeFixedString>(checkAndGetColumn<ColumnFixedString>(sub_map_column.get())->getN());
}
if (arg1_const)
sub_arg1_column = ColumnConst::create(arg1_const->getDataColumnPtr(), element_size);
else
sub_arg1_column = ColumnConst::create(arguments[1].column->cut(row, 1), element_size);
size_t col_key_size = sub_map_column->size();
auto column = is_const? ColumnConst::create(std::move(sub_map_column), std::move(col_key_size)) : std::move(sub_map_column);
new_arguments = {
{
column,
data_type,
""
},
arguments[1]
};
{column, data_type, ""},
{sub_arg1_column, arguments[1].type, ""}
};
auto res = func_like.executeImpl(new_arguments, result_type, input_rows_count);
const auto & container = checkAndGetColumn<ColumnUInt8>(res.get())->getData();
@ -479,13 +482,8 @@ public:
{
if (container[row_num] == 1)
{
auto key_ref = keys_string_column ?
keys_string_column->getDataAt(element_start_row + row_num) :
keys_fixed_string_column->getDataAt(element_start_row + row_num);
auto value_ref = values_column.getDataAt(element_start_row + row_num);
keys_data->insertData(key_ref.data, key_ref.size);
values_data->insertData(value_ref.data, value_ref.size);
keys_data->insertFrom(keys_column, element_start_row + row_num);
values_data->insertFrom(values_column, element_start_row + row_num);
current_offset += 1;
}
}
@ -583,9 +581,9 @@ public:
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
{
size_t left_it_begin = col_const_map_left_flag ? 0 : offsets_left[row_idx - 1];
size_t left_it_end = col_const_map_left_flag ? offsets_left.size() : offsets_left[row_idx];
size_t left_it_end = col_const_map_left_flag ? offsets_left[0] : offsets_left[row_idx];
size_t right_it_begin = col_const_map_right_flag ? 0 : offsets_right[row_idx - 1];
size_t right_it_end = col_const_map_right_flag ? offsets_right.size() : offsets_right[row_idx];
size_t right_it_end = col_const_map_right_flag ? offsets_right[0] : offsets_right[row_idx];
for (size_t i = left_it_begin; i < left_it_end; ++i)
{
@ -832,7 +830,8 @@ public:
if (null_presence.has_null_constant)
return makeNullable(std::make_shared<DataTypeNothing>());
const DataTypeMap * map = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
auto arg0_no_null = removeNullable(arguments[0].type);
const DataTypeMap * map = checkAndGetDataType<DataTypeMap>(arg0_no_null.get());
if (!map)
throw Exception(
@ -851,13 +850,31 @@ public:
if (null_presence.has_null_constant)
return result_type->createColumnConstWithDefaultValue(input_rows_count);
const auto * col_map = checkAndGetColumn<ColumnMap>(arguments[0].column.get());
const auto * col_const_map = checkAndGetColumnConst<ColumnMap>(arguments[0].column.get());
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
ColumnsWithTypeAndName tmp_args = arguments;
if (null_presence.has_nullable)
{
/// if index is const null -> return directly
if (null_presence.has_null_constant)
{
/// Default implementation for nulls returns null result for null arguments,
/// so the result type must be nullable.
if (!result_type->isNullable())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Function {} with Null argument and default implementation for Nulls "
"is expected to return Nullable result, got {}", getName(), result_type->getName());
return result_type->createColumnConstWithDefaultValue(input_rows_count);
}
tmp_args = createBlockWithNestedColumns(arguments);
}
const auto * col_map = checkAndGetColumn<ColumnMap>(tmp_args[0].column.get());
const auto * col_const_map = checkAndGetColumnConst<ColumnMap>(tmp_args[0].column.get());
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(tmp_args[0].type.get());
if ((!col_map && !col_const_map) || !map_type)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"First argument for function '{}' must be map, got '{}' instead",
getName(),arguments[0].type->getName());
getName(),tmp_args[0].type->getName());
if (col_const_map)
col_map = typeid_cast<const ColumnMap *>(&col_const_map->getDataColumn());
@ -872,14 +889,14 @@ public:
auto & indices_data = assert_cast<ColumnVector<UInt64> &>(*indices_column).getData();
bool executed = false;
if (!isColumnConst(*arguments[1].column))
if (!isColumnConst(*tmp_args[1].column))
{
executed = FunctionArrayElement::matchKeyToIndexNumber(key_column, offsets, !!col_const_map, *arguments[1].column, indices_data)
|| FunctionArrayElement::matchKeyToIndexString(key_column, offsets, !!col_const_map, *arguments[1].column, indices_data);
executed = FunctionArrayElement::matchKeyToIndexNumber(key_column, offsets, !!col_const_map, *tmp_args[1].column, indices_data)
|| FunctionArrayElement::matchKeyToIndexString(key_column, offsets, !!col_const_map, *tmp_args[1].column, indices_data);
}
else
{
Field index = (*arguments[1].column)[0];
Field index = (*tmp_args[1].column)[0];
/// try convert const index to right type
auto index_lit = std::make_shared<ASTLiteral>(index);
@ -893,7 +910,7 @@ public:
if (!executed)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal types of arguments: {}, {} for function {}",
arguments[0].type->getName(), arguments[1].type->getName(), getName());
tmp_args[0].type->getName(), tmp_args[1].type->getName(), getName());
auto col_res = IColumn::mutate(result_type->createColumn());
auto & col_res_ref = typeid_cast<ColumnNullable &>(*col_res);
@ -915,7 +932,13 @@ public:
}
}
return col_res;
ColumnPtr res = std::move(col_res);
/// If arg0 is nullable, merge it's null map.
if (arguments[0].type->isNullable())
return wrapInNullable(res, arguments, result_type, input_rows_count);
return res;
}
};
@ -1024,8 +1047,8 @@ public:
{
// TODO(shiyuze): maybe add a new function to get result in different rows, just like arrayJoin(getMapKeys(xxx))
/// Total map key number in result may exceed max_array_size_as_field (for example,
/// each pratition or bucket has different key sets), so we need to avoid calling
/// Total map key number in result may exceed max_array_size_as_field (for example,
/// each pratition or bucket has different key sets), so we need to avoid calling
/// ColumnArray::[] or ColumnArray::get() to get array as a Field here.
return ColumnConst::create(res.getByName("keys").column, input_rows_count)->convertToFullColumnIfConst();
}

View File

@ -0,0 +1,253 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeTuple.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <Functions/dtsCityHash.h>
#include <Functions/hiveIntHash.h>
#include "BigInteger.hh"
#include <cmath>
#include <typeinfo>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
/// A C++ implementation of `ClickhouseHasher` class in Spark.
/// Reimplementation of repo `spark`, branch `branch-3.2-bd`, commit `66465e1cfe95a0f1313043215344a45a0df71da6`
class FunctionSparkPartition : public IFunction
{
public:
static constexpr auto name = "sparkpartition";
static FunctionPtr create(ContextPtr)
{
return std::make_shared<FunctionSparkPartition>();
}
String getName() const override
{
return name;
}
bool isVariadic() const override { return false; }
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
size_t number_of_arguments = arguments.size();
if (number_of_arguments != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(number_of_arguments) + ", should be 2",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!isNumber(arguments[1]))
throw Exception("Illegal type " + arguments[1]->getName()
+ " of second argument of function "
+ getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeUInt64>();
}
private:
using ToType = UInt64;
template <typename FromType>
UInt64 getHashPartition(const FromType & value, const BigInteger & split_number) const
{
return std::abs((HiveIntHash::intHash64(value) % split_number).toInt());
}
template <typename FromType>
void executeIntType(const IColumn * column, typename ColumnVector<ToType>::Container & vec_to, const Int64 & split_number) const
{
const BigInteger split_number_bigint(split_number);
if (const ColumnVector<FromType> * col_from = checkAndGetColumn<ColumnVector<FromType>>(column))
{
const typename ColumnVector<FromType>::Container & vec_from = col_from->getData();
size_t size = vec_from.size();
for (size_t i = 0; i < size; ++i)
{
vec_to[i] = getHashPartition(vec_from[i], split_number_bigint);
}
}
else if (auto col_from_const = checkAndGetColumnConst<ColumnVector<FromType>>(column))
{
FromType value = col_from_const->template getValue<FromType>();
UInt64 hash = getHashPartition(value, split_number_bigint);
size_t size = vec_to.size();
vec_to.assign(size, hash);
}
else
throw Exception("Illegal column " + column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
void executeString(const IColumn * column, typename ColumnVector<ToType>::Container & vec_to, const Int64 & split_number) const
{
const BigInteger split_number_bigint(split_number);
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(column))
{
const typename ColumnString::Chars & data = col_from->getChars();
const typename ColumnString::Offsets & offsets = col_from->getOffsets();
size_t size = offsets.size();
ColumnString::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
const char *s = reinterpret_cast<const char *>(&data[current_offset]);
auto length = offsets[i] - current_offset - 1;
if (isDigit(s, length))
vec_to[i] = getHashPartition<BigInteger>(stringToBigInteger(s, length), split_number_bigint);
else
vec_to[i] = DTSCityHash::asUInt64(DTSCityHash::cityHash64(s, 0, length)) % split_number;
current_offset = offsets[i];
}
}
else if (const ColumnFixedString * col_from_fixed = checkAndGetColumn<ColumnFixedString>(column))
{
const typename ColumnString::Chars & data = col_from_fixed->getChars();
size_t n = col_from_fixed->getN();
size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i)
{
const char *s = reinterpret_cast<const char *>(&data[i * n]);
if (isDigit(s, n))
vec_to[i] = getHashPartition<BigInteger>(stringToBigInteger(s, n), split_number_bigint);
else
vec_to[i] = DTSCityHash::asUInt64(DTSCityHash::cityHash64(s, 0, n)) % split_number;
}
}
else if (const ColumnConst * col_from_const = checkAndGetColumnConstStringOrFixedString(column))
{
String value = col_from_const->getValue<String>().data();
UInt64 hash;
if (isDigit(value.c_str(), value.length()))
hash = getHashPartition<BigInteger>(stringToBigInteger(value.data(), value.length()), split_number_bigint);
else
hash = DTSCityHash::asUInt64(DTSCityHash::cityHash64(value.data(), 0, value.length())) % split_number;
vec_to.assign(vec_to.size(), hash);
}
else
throw Exception("Illegal column " + column->getName()
+ " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
void executeAny(const IDataType * from_type, const IColumn * icolumn, typename ColumnVector<ToType>::Container & vec_to, const Int64 & split_number) const
{
WhichDataType which(from_type);
if (which.isUInt8()) executeIntType<UInt8>(icolumn, vec_to, split_number);
else if (which.isUInt16()) executeIntType<UInt16>(icolumn, vec_to, split_number);
else if (which.isUInt32()) executeIntType<UInt32>(icolumn, vec_to, split_number);
else if (which.isUInt64()) executeIntType<UInt64>(icolumn, vec_to, split_number);
else if (which.isInt8()) executeIntType<Int8>(icolumn, vec_to, split_number);
else if (which.isInt16()) executeIntType<Int16>(icolumn, vec_to, split_number);
else if (which.isInt32()) executeIntType<Int32>(icolumn, vec_to, split_number);
else if (which.isInt64()) executeIntType<Int64>(icolumn, vec_to, split_number);
else if (which.isEnum8()) executeIntType<Int8>(icolumn, vec_to, split_number);
else if (which.isEnum16()) executeIntType<Int16>(icolumn, vec_to, split_number);
else if (which.isDate()) executeIntType<UInt16>(icolumn, vec_to, split_number);
else if (which.isDateTime()) executeIntType<UInt32>(icolumn, vec_to, split_number);
else if (which.isString()) executeString(icolumn, vec_to, split_number);
else if (which.isFixedString()) executeString(icolumn, vec_to, split_number);
else
throw Exception("Unexpected type " + from_type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
Int64 getSplitNumber(const ColumnPtr & column_split_number) const
{
Int64 split_number = 0;
auto column_split_number_ptr = column_split_number.get();
if (const ColumnConst * column_split_number_const = checkAndGetColumn<ColumnConst>(column_split_number_ptr))
{
split_number = column_split_number_const->getInt(0);
}
else if(const ColumnInt64 * column_split_number_int64 = checkAndGetColumn<ColumnInt64>(column_split_number_ptr))
{
split_number = column_split_number_int64->getInt(0);
}
else if(const ColumnInt32 * column_split_number_int32 = checkAndGetColumn<ColumnInt32>(column_split_number_ptr))
{
split_number = column_split_number_int32->getInt(0);
}
else if(const ColumnInt16 * column_split_number_int16 = checkAndGetColumn<ColumnInt16>(column_split_number_ptr))
{
split_number = column_split_number_int16->getInt(0);
}
else if(const ColumnInt8 * column_split_number_int8 = checkAndGetColumn<ColumnInt8>(column_split_number_ptr))
{
split_number = column_split_number_int8->getInt(0);
}
else if(const ColumnUInt64 * column_split_number_uint64 = checkAndGetColumn<ColumnUInt64>(column_split_number_ptr))
{
split_number = column_split_number_uint64->getInt(0);
}
else if(const ColumnUInt32 * column_split_number_uint32 = checkAndGetColumn<ColumnUInt32>(column_split_number_ptr))
{
split_number = column_split_number_uint32->getInt(0);
}
else if(const ColumnUInt16 * column_split_number_uint16 = checkAndGetColumn<ColumnUInt16>(column_split_number_ptr))
{
split_number = column_split_number_uint16->getInt(0);
}
else if(const ColumnUInt8 * column_split_number_uint8 = checkAndGetColumn<ColumnUInt8>(column_split_number_ptr))
{
split_number = column_split_number_uint8->getInt(0);
}
return split_number;
}
public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
ColumnPtr column_split_number = arguments[1].column;
Int64 split_number = getSplitNumber(column_split_number);
if (split_number <= 0)
throw Exception(
"Illegal split_number of function " + getName() + ", should be greater than 0" ,
ErrorCodes::ILLEGAL_COLUMN);
auto col_to = ColumnVector<ToType>::create(input_rows_count);
typename ColumnVector<ToType>::Container & vec_to = col_to->getData();
const ColumnWithTypeAndName & col = arguments[0];
executeAny(col.type.get(), col.column.get(), vec_to, split_number);
return col_to;
}
};
REGISTER_FUNCTION(SparkPartition)
{
factory.registerFunction<FunctionSparkPartition>();
}
}

View File

@ -0,0 +1,78 @@
#include <gtest/gtest.h>
#include <Functions/dtsCityHash.h>
#include <Functions/hiveIntHash.h>
#include "BigInteger.hh"
#include "BigIntegerUtils.hh"
/// Test dtsCityHash
static void test(std::string s, int64_t expected, uint64_t expected_us)
{
int64_t r = DB::DTSCityHash::cityHash64(s.data(), 0, s.length());
EXPECT_EQ(r, expected);
EXPECT_EQ(DB::DTSCityHash::asUInt64(r), expected_us);
}
TEST(DTSCityHash, Test0to4)
{
test("abc", 4220206313085259313L, 4220206313085259313UL);
test("111", 5518486022181526874L, 5518486022181526874UL);
test("-111", 5271023622228185005L, 5271023622228185005UL);
}
TEST(DTSCityHash, Test5to7)
{
test("abcabc", -2180749251667803241L, 16265994822041748375UL);
}
TEST(DTSCityHash, Test8to16)
{
test("abcabcabc", -363239307316136316L, 18083504766393415300UL);
test("e43970221139c", 651451745298338306L, 651451745298338306UL);
}
TEST(DTSCityHash, Test17to32)
{
test("abcabcabcabcabcabcabcabc", -8992032506223809229L, 9454711567485742387UL);
}
TEST(DTSCityHash, Test33to64)
{
test("abcabcabcabcabcabcabcabcabcabcabcabc", -8163064513280341296L, 10283679560429210320UL);
}
TEST(DTSCityHash, Test64Plus)
{
test("abcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabcabc", -822948244188030202L, 17623795829521521414UL);
}
TEST(DTSCityHash, TestCN)
{
test("中国人", -4667871355360362275L, 13778872718349189341UL);
test("中国人中国人中国人中国人中国人中国人中国人中国人中国人中国人", -5394058890835601704L, 13052685182873949912UL);
}
/// Test hiveIntHash
static void test_int(BigInteger x, BigInteger expected)
{
BigInteger r = DB::HiveIntHash::intHash64(x);
EXPECT_EQ(r, expected);
}
TEST(HiveIntHash, TestInt)
{
test_int(stringToBigInteger("111"), stringToBigInteger("28960595049120298284007597794249568656608"));
test_int(stringToBigInteger("-111"), stringToBigInteger("28699688790393188402562795085604323520147"));
}
TEST(HiveIntHash, TestLong)
{
test_int(stringToBigInteger("4294967297"), stringToBigInteger("1120583861364653015936649848396047605431838178733"));
test_int(stringToBigInteger("-4294967297"), stringToBigInteger("1120583861278725678966405905726831886655789209759"));
}
TEST(HiveIntHash, TestBigInt)
{
test_int(stringToBigInteger("340282366920938463463374607431768211457"), stringToBigInteger("88781800262427497179229347262209927076731686175940560623852567004281829509932"));
test_int(stringToBigInteger("-340282366920938463463374607431768211457"), stringToBigInteger("88781800262427497179229347262209927076101144782869356141092329336787357925376"));
}

View File

@ -15,6 +15,7 @@ public:
template <typename CharT, typename = std::enable_if_t<sizeof(CharT) == 1>>
ReadBufferFromMemory(const CharT * buf, size_t size)
: SeekableReadBuffer(const_cast<char *>(reinterpret_cast<const char *>(buf)), size, 0) {}
explicit ReadBufferFromMemory(const std::string_view && str) : SeekableReadBuffer(const_cast<char *>(str.data()), str.size(), 0) { }
off_t seek(off_t off, int whence) override;

View File

@ -1,7 +1,9 @@
#include <memory>
#include <Disks/IVolume.h>
#include <IO/HTTPCommon.h>
#include <IO/S3Common.h>
#include <aws/s3/model/CopyObjectRequest.h>
#include <MergeTreeCommon/MergeTreeMetaBase.h>
#include <Common/config.h>
#if USE_AWS_S3
@ -1100,26 +1102,6 @@ namespace S3
}
}
Aws::S3::Model::GetObjectResult S3Util::headObjectByGet(const String & key) const
{
Aws::S3::Model::GetObjectRequest req;
req.SetBucket(bucket);
req.SetKey(key);
req.SetRange("bytes=0-1");
ProfileEvents::increment(ProfileEvents::S3GetObject);
Aws::S3::Model::GetObjectOutcome outcome = client->GetObject(req);
if (outcome.IsSuccess())
{
return outcome.GetResultWithOwnership();
}
else
{
throw S3Exception(outcome.GetError());
}
}
S3LazyCleaner::S3LazyCleaner(
const S3::S3Util & s3_util_,
const std::function<bool(const S3::S3Util &, const String &)> & filter_,
@ -1334,8 +1316,31 @@ namespace S3
}
}
}
void S3LazyCleaner::push(const MutableMergeTreeDataPartCNCHPtr & part)
{
if (!part)
return;
if (auto disk = part->volume->getDisk(); disk)
{
String part_key = std::filesystem::path(disk->getPath()) / part->getFullRelativePath() / "data";
push(part_key);
}
}
void S3LazyCleaner::push(const DeleteBitmapMetaPtr & bitmap, const StoragePtr & storage)
{
if (!bitmap || !storage)
return;
if (auto path = bitmap->getFullRelativePath(); path.has_value())
{
if (DiskPtr disk = storage->getStoragePolicy(IStorage::StorageLocation::MAIN)->getAnyDisk(); disk)
{
String bitmap_key = std::filesystem::path(disk->getPath()) / *path;
push(bitmap_key);
}
}
}
}
}
#endif

View File

@ -21,6 +21,8 @@
# include <Common/Logger.h>
# include <Common/ThreadPool.h>
# include <common/types.h>
#include <Storages/MergeTree/DeleteBitmapMeta.h>
#include <Storages/MergeTree/MergeTreeDataPartCNCH.h>
namespace Aws::S3
{
class S3Client;
@ -260,6 +262,8 @@ public:
void deleteObjectsWithPrefix(
const String & prefix, const std::function<bool(const S3Util &, const String &)> & filter, size_t batch_size = 1000) const;
Aws::S3::Model::HeadObjectResult headObject(const String & key) const;
// Internal client and info
const std::shared_ptr<Aws::S3::S3Client> & getClient() const { return client; }
const String & getBucket() const { return bucket; }
@ -267,9 +271,6 @@ public:
private:
static String urlEncodeMap(const std::map<String, String> & mp);
Aws::S3::Model::HeadObjectResult headObject(const String & key) const;
Aws::S3::Model::GetObjectResult headObjectByGet(const String & key) const;
std::shared_ptr<Aws::S3::S3Client> client;
const String bucket;
@ -288,7 +289,9 @@ public:
size_t batch_clean_size_ = S3_DEFAULT_BATCH_CLEAN_SIZE);
~S3LazyCleaner() noexcept;
void push(const String & key_);
void push(const String& key_);
void push(const MutableMergeTreeDataPartCNCHPtr & part);
void push(const DeleteBitmapMetaPtr & bitmap, const StoragePtr & storage);
void finalize();
private:

View File

@ -111,6 +111,12 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr c
}
}
if (settings.final_order_by_all_direction != 0)
{
new_settings.final_order_by_all_direction = 0;
new_settings.final_order_by_all_direction.changed = false;
}
if (settings.offset)
{
new_settings.offset = 0;

View File

@ -4743,7 +4743,7 @@ StorageID Context::resolveStorageID(StorageID storage_id, StorageNamespace where
{
if (auto worker_resource = tryGetCnchWorkerResource())
{
if (auto storage = worker_resource->getTable(storage_id))
if (auto storage = worker_resource->tryGetTable(storage_id))
return storage->getStorageID();
}
}
@ -5831,7 +5831,11 @@ void Context::initCnchTransactionCoordinator()
TransactionCoordinatorRcCnch & Context::getCnchTransactionCoordinator() const
{
auto lock = getLock(); // checked
return *shared->cnch_txn_coordinator;
if (auto * ptr = shared->cnch_txn_coordinator.get())
{
return *ptr;
}
throw Exception("CnchTransactionCoordinator is not initialized", ErrorCodes::SYSTEM_ERROR);
}
void Context::setCurrentTransaction(TransactionCnchPtr txn, bool finish_txn)
@ -5910,12 +5914,6 @@ TxnTimestamp Context::getCurrentCnchStartTime() const
return current_cnch_txn->getStartTime();
}
InterserverCredentialsPtr Context::getCnchInterserverCredentials()
{
/// FIXME: any special for cnch ?
return getInterserverCredentials();
}
// In CNCH, form a virtual cluster which include all servers.
std::shared_ptr<Cluster> Context::mockCnchServersCluster() const
{

View File

@ -1698,7 +1698,6 @@ public:
std::multimap<StorageID, MergeTreeMutationStatus> collectMutationStatusesByTables(std::unordered_set<UUID> table_uuids) const;
InterserverCredentialsPtr getCnchInterserverCredentials();
std::shared_ptr<Cluster> mockCnchServersCluster() const;
std::vector<std::pair<UInt64, CnchWorkerResourcePtr>> getAllWorkerResources() const;

View File

@ -102,6 +102,7 @@ struct DAGGraph
std::unordered_map<size_t, UInt64> segment_parallel_size_map;
butil::IOBuf query_common_buf;
butil::IOBuf query_settings_buf;
std::unordered_map<WorkerId, butil::IOBuf, WorkerIdHash> query_resource_map;
SourcePrunerPtr source_pruner;
LoggerPtr log;

View File

@ -296,7 +296,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
{
if (auto worker_resource = context_->tryGetCnchWorkerResource())
{
if (auto table = worker_resource->getTable(table_id))
if (auto table = worker_resource->tryGetTable(table_id))
{
LOG_INFO(log, "got table {} from worker resource", table_id.getNameForLogs());
return {nullptr, table};

Some files were not shown because too many files have changed in this diff Show More