Merge remote-tracking branch 'rschu1ze/master' into clang-tidy-hdr

This commit is contained in:
Robert Schulze 2024-02-29 16:01:33 +00:00
commit 29cd994302
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
34 changed files with 574 additions and 416 deletions

View File

@ -67,8 +67,6 @@ jobs:
test_name: Compatibility check (amd64)
runner_type: style-checker
data: ${{ needs.RunConfig.outputs.data }}
run_command: |
python3 compatibility_check.py --check-name "Compatibility check (amd64)" --check-glibc --check-distributions
CompatibilityCheckAarch64:
needs: [RunConfig, BuilderDebAarch64]
if: ${{ !failure() && !cancelled() }}
@ -77,8 +75,6 @@ jobs:
test_name: Compatibility check (aarch64)
runner_type: style-checker
data: ${{ needs.RunConfig.outputs.data }}
run_command: |
python3 compatibility_check.py --check-name "Compatibility check (aarch64)" --check-glibc
#########################################################################################
#################################### ORDINARY BUILDS ####################################
#########################################################################################

View File

@ -73,8 +73,6 @@ jobs:
test_name: Compatibility check (amd64)
runner_type: style-checker
data: ${{ needs.RunConfig.outputs.data }}
run_command: |
python3 compatibility_check.py --check-name "Compatibility check (amd64)" --check-glibc --check-distributions
CompatibilityCheckAarch64:
needs: [RunConfig, BuilderDebAarch64]
if: ${{ !failure() && !cancelled() }}
@ -83,8 +81,6 @@ jobs:
test_name: Compatibility check (aarch64)
runner_type: style-checker
data: ${{ needs.RunConfig.outputs.data }}
run_command: |
python3 compatibility_check.py --check-name "Compatibility check (aarch64)" --check-glibc
#########################################################################################
#################################### ORDINARY BUILDS ####################################
#########################################################################################

View File

@ -117,8 +117,6 @@ jobs:
test_name: Compatibility check (amd64)
runner_type: style-checker
data: ${{ needs.RunConfig.outputs.data }}
run_command: |
python3 compatibility_check.py --check-name "Compatibility check (amd64)" --check-glibc --check-distributions
CompatibilityCheckAarch64:
needs: [RunConfig, BuilderDebAarch64]
if: ${{ !failure() && !cancelled() }}
@ -127,8 +125,6 @@ jobs:
test_name: Compatibility check (aarch64)
runner_type: style-checker
data: ${{ needs.RunConfig.outputs.data }}
run_command: |
python3 compatibility_check.py --check-name "Compatibility check (aarch64)" --check-glibc
#########################################################################################
#################################### ORDINARY BUILDS ####################################
#########################################################################################

View File

@ -68,8 +68,6 @@ jobs:
test_name: Compatibility check (amd64)
runner_type: style-checker
data: ${{ needs.RunConfig.outputs.data }}
run_command: |
python3 compatibility_check.py --check-name "Compatibility check (amd64)" --check-glibc --check-distributions
CompatibilityCheckAarch64:
needs: [RunConfig, BuilderDebAarch64]
if: ${{ !failure() && !cancelled() }}
@ -78,8 +76,6 @@ jobs:
test_name: Compatibility check (aarch64)
runner_type: style-checker
data: ${{ needs.RunConfig.outputs.data }}
run_command: |
python3 compatibility_check.py --check-name "Compatibility check (aarch64)" --check-glibc
#########################################################################################
#################################### ORDINARY BUILDS ####################################
#########################################################################################

View File

@ -16,6 +16,7 @@
#ci_set_reduced
#ci_set_arm
#ci_set_integration
#ci_set_analyzer
## To run specified job in CI:
#job_<JOB NAME>

View File

@ -21,7 +21,7 @@ Columns:
- `hostname` ([LowCardinality(String)](../../sql-reference/data-types/string.md)) — Hostname of the server executing the query.
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the thread has finished execution of the query.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the thread has finished execution of the query.
- `event_time_microsecinds` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the thread has finished execution of the query with microseconds precision.
- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the thread has finished execution of the query with microseconds precision.
- `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Start time of query execution.
- `query_start_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Start time of query execution with microsecond precision.
- `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution.
@ -32,8 +32,7 @@ Columns:
- `memory_usage` ([Int64](../../sql-reference/data-types/int-uint.md)) — The difference between the amount of allocated and freed memory in context of this thread.
- `peak_memory_usage` ([Int64](../../sql-reference/data-types/int-uint.md)) — The maximum difference between the amount of allocated and freed memory in context of this thread.
- `thread_name` ([String](../../sql-reference/data-types/string.md)) — Name of the thread.
- `thread_number` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Internal thread ID.
- `thread_id` ([Int32](../../sql-reference/data-types/int-uint.md)) — thread ID.
- `thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — OS thread ID.
- `master_thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — OS initial ID of initial thread.
- `query` ([String](../../sql-reference/data-types/string.md)) — Query string.
- `is_initial_query` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Query type. Possible values:

View File

@ -6651,6 +6651,7 @@ void QueryAnalyzer::initializeTableExpressionData(const QueryTreeNodePtr & table
if (column_default && column_default->kind == ColumnDefaultKind::Alias)
{
auto alias_expression = buildQueryTree(column_default->expression, scope.context);
alias_expression = buildCastFunction(alias_expression, column_name_and_type.type, scope.context, false /*resolve*/);
auto column_node = std::make_shared<ColumnNode>(column_name_and_type, std::move(alias_expression), table_expression_node);
column_name_to_column_node.emplace(column_name_and_type.name, column_node);
alias_columns_to_resolve.emplace_back(column_name_and_type.name, column_node);
@ -6683,9 +6684,7 @@ void QueryAnalyzer::initializeTableExpressionData(const QueryTreeNodePtr & table
alias_column_resolve_scope,
false /*allow_lambda_expression*/,
false /*allow_table_expression*/);
auto & resolved_expression = alias_column_to_resolve->getExpression();
if (!resolved_expression->getResultType()->equals(*alias_column_to_resolve->getResultType()))
resolved_expression = buildCastFunction(resolved_expression, alias_column_to_resolve->getResultType(), scope.context, true);
column_name_to_column_node = std::move(alias_column_resolve_scope.column_name_to_column_node);
column_name_to_column_node[alias_column_to_resolve_name] = alias_column_to_resolve;
}

View File

@ -252,8 +252,21 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
if (which_type.isDateTime64() && src.getType() == Field::Types::Decimal64)
{
/// Already in needed type.
return src;
const auto & from_type = src.get<Decimal64>();
const auto & to_type = static_cast<const DataTypeDateTime64 &>(type);
const auto scale_from = from_type.getScale();
const auto scale_to = to_type.getScale();
const auto scale_multiplier_diff = scale_from > scale_to ? from_type.getScaleMultiplier() / to_type.getScaleMultiplier() : to_type.getScaleMultiplier() / from_type.getScaleMultiplier();
if (scale_multiplier_diff == 1) /// Already in needed type.
return src;
/// in case if we need to make DateTime64(a) from DateTime64(b), a != b, we need to convert datetime value to the right scale
const UInt64 value = scale_from > scale_to ? from_type.getValue().value / scale_multiplier_diff : from_type.getValue().value * scale_multiplier_diff;
return DecimalField(
DecimalUtils::decimalFromComponentsWithMultiplier<DateTime64>(value, 0, 1),
scale_to);
}
/// For toDate('xxx') in 1::Int64, we CAST `src` to UInt64, which may

View File

@ -121,12 +121,7 @@ Block getHeaderForProcessingStage(
auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(left_table_expression);
const auto & query_context = query_info.planner_context->getQueryContext();
NamesAndTypes columns;
const auto & column_name_to_column = table_expression_data.getColumnNameToColumn();
for (const auto & column_name : table_expression_data.getSelectedColumnsNames())
columns.push_back(column_name_to_column.at(column_name));
auto columns = table_expression_data.getColumns();
auto new_query_node = buildSubqueryToReadColumnsFromTableExpression(columns, left_table_expression, query_context);
query = new_query_node->toAST();
}

View File

@ -29,13 +29,34 @@ namespace
class CollectSourceColumnsVisitor : public InDepthQueryTreeVisitor<CollectSourceColumnsVisitor>
{
public:
explicit CollectSourceColumnsVisitor(PlannerContextPtr & planner_context_, bool keep_alias_columns_ = true)
explicit CollectSourceColumnsVisitor(PlannerContext & planner_context_)
: planner_context(planner_context_)
, keep_alias_columns(keep_alias_columns_)
{}
void visitImpl(QueryTreeNodePtr & node)
{
/// Special case for USING clause which contains references to ALIAS columns.
/// We can not modify such ColumnNode.
if (auto * join_node = node->as<JoinNode>())
{
if (!join_node->isUsingJoinExpression())
return;
auto & using_list = join_node->getJoinExpression()->as<ListNode&>();
for (auto & using_element : using_list)
{
auto & column_node = using_element->as<ColumnNode&>();
/// This list contains column nodes from left and right tables.
auto & columns_from_subtrees = column_node.getExpressionOrThrow()->as<ListNode&>().getNodes();
/// Visit left table column node.
visitUsingColumn(columns_from_subtrees[0]);
/// Visit right table column node.
visitUsingColumn(columns_from_subtrees[1]);
}
return;
}
auto * column_node = node->as<ColumnNode>();
if (!column_node)
return;
@ -51,55 +72,22 @@ public:
/// JOIN using expression
if (column_node->hasExpression() && column_source_node_type == QueryTreeNodeType::JOIN)
{
auto & columns_from_subtrees = column_node->getExpression()->as<ListNode &>().getNodes();
if (columns_from_subtrees.size() != 2)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Expected two columns in JOIN using expression for column {}", column_node->dumpTree());
visit(columns_from_subtrees[0]);
visit(columns_from_subtrees[1]);
return;
}
auto & table_expression_data = planner_context->getOrCreateTableExpressionData(column_source_node);
auto & table_expression_data = planner_context.getOrCreateTableExpressionData(column_source_node);
if (isAliasColumn(node))
if (column_node->hasExpression() && column_source_node_type != QueryTreeNodeType::ARRAY_JOIN)
{
/// Column is an ALIAS column with expression
/// Replace ALIAS column with expression
bool column_already_exists = table_expression_data.hasColumn(column_node->getColumnName());
if (!column_already_exists)
{
CollectSourceColumnsVisitor visitor_for_alias_column(planner_context);
/// While we are processing expression of ALIAS columns we should not add source columns to selected.
/// See also comment for `select_added_columns`
visitor_for_alias_column.select_added_columns = false;
visitor_for_alias_column.keep_alias_columns = keep_alias_columns;
visitor_for_alias_column.visit(column_node->getExpression());
if (!keep_alias_columns)
{
/// For PREWHERE we can just replace ALIAS column with it's expression,
/// because ActionsDAG for PREWHERE applied right on top of table expression
/// and cannot affect subqueries or other table expressions.
node = column_node->getExpression();
return;
}
auto column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(node);
ActionsDAGPtr alias_column_actions_dag = std::make_shared<ActionsDAG>();
PlannerActionsVisitor actions_visitor(planner_context, false);
auto outputs = actions_visitor.visit(alias_column_actions_dag, column_node->getExpression());
if (outputs.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Expected single output in actions dag for alias column {}. Actual {}", column_node->dumpTree(), outputs.size());
const auto & column_name = column_node->getColumnName();
const auto & alias_node = alias_column_actions_dag->addAlias(*outputs[0], column_name);
alias_column_actions_dag->addOrReplaceInOutputs(alias_node);
table_expression_data.addAliasColumn(column_node->getColumn(), column_identifier, alias_column_actions_dag, select_added_columns);
auto column_identifier = planner_context.getGlobalPlannerContext()->createColumnIdentifier(node);
table_expression_data.addAliasColumnName(column_node->getColumnName(), column_identifier);
}
node = column_node->getExpression();
visitImpl(node);
return;
}
@ -114,58 +102,45 @@ public:
bool column_already_exists = table_expression_data.hasColumn(column_node->getColumnName());
if (column_already_exists)
{
/// Column may be added when we collected data for ALIAS column
/// But now we see it directly in the query, so make sure it's marked as selected
if (select_added_columns)
table_expression_data.markSelectedColumn(column_node->getColumnName());
return;
auto column_identifier = planner_context.getGlobalPlannerContext()->createColumnIdentifier(node);
table_expression_data.addColumn(column_node->getColumn(), column_identifier);
}
static bool needChildVisit(const QueryTreeNodePtr & parent, const QueryTreeNodePtr & child_node)
{
if (auto * join_node = parent->as<JoinNode>())
{
if (join_node->getJoinExpression() == child_node && join_node->isUsingJoinExpression())
return false;
}
auto column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(node);
table_expression_data.addColumn(column_node->getColumn(), column_identifier, select_added_columns);
}
static bool isAliasColumn(const QueryTreeNodePtr & node)
{
const auto * column_node = node->as<ColumnNode>();
if (!column_node || !column_node->hasExpression())
return false;
const auto & column_source = column_node->getColumnSourceOrNull();
if (!column_source)
return false;
return column_source->getNodeType() != QueryTreeNodeType::JOIN &&
column_source->getNodeType() != QueryTreeNodeType::ARRAY_JOIN;
}
static bool needChildVisit(const QueryTreeNodePtr & parent_node, const QueryTreeNodePtr & child_node)
{
auto child_node_type = child_node->getNodeType();
return !(child_node_type == QueryTreeNodeType::QUERY ||
child_node_type == QueryTreeNodeType::UNION ||
isAliasColumn(parent_node));
}
void setKeepAliasColumns(bool keep_alias_columns_)
{
keep_alias_columns = keep_alias_columns_;
return !(child_node_type == QueryTreeNodeType::QUERY || child_node_type == QueryTreeNodeType::UNION);
}
private:
PlannerContextPtr & planner_context;
/// Replace ALIAS columns with their expressions or register them in table expression data.
/// Usually we can replace them when we build some "local" actions DAG
/// (for example Row Policy or PREWHERE) that is applied on top of the table expression.
/// In other cases, we keep ALIAS columns as ColumnNode with an expression child node,
/// and handle them in the Planner by inserting ActionsDAG to compute them after reading from storage.
bool keep_alias_columns = true;
void visitUsingColumn(QueryTreeNodePtr & node)
{
auto & column_node = node->as<ColumnNode&>();
if (column_node.hasExpression())
{
auto & table_expression_data = planner_context.getOrCreateTableExpressionData(column_node.getColumnSource());
bool column_already_exists = table_expression_data.hasColumn(column_node.getColumnName());
if (column_already_exists)
return;
/// Flag `select_added_columns` indicates if we should mark column as explicitly selected.
/// For example, for table with columns (a Int32, b ALIAS a+1) and query SELECT b FROM table
/// Column `b` is selected explicitly by user, but not `a` (that is also read though).
/// Distinguishing such columns is important for checking access rights for ALIAS columns.
bool select_added_columns = true;
auto column_identifier = planner_context.getGlobalPlannerContext()->createColumnIdentifier(node);
table_expression_data.addAliasColumnName(column_node.getColumnName(), column_identifier);
visitImpl(column_node.getExpressionOrThrow());
}
else
visitImpl(node);
}
PlannerContext & planner_context;
};
class CollectPrewhereTableExpressionVisitor : public ConstInDepthQueryTreeVisitor<CollectPrewhereTableExpressionVisitor>
@ -299,7 +274,7 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr
}
}
CollectSourceColumnsVisitor collect_source_columns_visitor(planner_context);
CollectSourceColumnsVisitor collect_source_columns_visitor(*planner_context);
for (auto & node : query_node_typed.getChildren())
{
if (!node || node == query_node_typed.getPrewhere())
@ -325,26 +300,21 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr
}
auto & table_expression_data = planner_context->getOrCreateTableExpressionData(prewhere_table_expression);
const auto & read_column_names = table_expression_data.getColumnNames();
NameSet required_column_names_without_prewhere(read_column_names.begin(), read_column_names.end());
const auto & selected_column_names = table_expression_data.getSelectedColumnsNames();
required_column_names_without_prewhere.insert(selected_column_names.begin(), selected_column_names.end());
const auto & column_names = table_expression_data.getColumnNames();
NameSet required_column_names_without_prewhere(column_names.begin(), column_names.end());
collect_source_columns_visitor.setKeepAliasColumns(false);
collect_source_columns_visitor.visit(query_node_typed.getPrewhere());
auto prewhere_actions_dag = std::make_shared<ActionsDAG>();
QueryTreeNodePtr query_tree_node = query_node_typed.getPrewhere();
PlannerActionsVisitor visitor(planner_context, false /*use_column_identifier_as_action_node_name*/);
auto expression_nodes = visitor.visit(prewhere_actions_dag, query_tree_node);
auto expression_nodes = visitor.visit(prewhere_actions_dag, query_node_typed.getPrewhere());
if (expression_nodes.size() != 1)
throw Exception(ErrorCodes::ILLEGAL_PREWHERE,
"Invalid PREWHERE. Expected single boolean expression. In query {}",
query_node->formatASTForErrorMessage());
prewhere_actions_dag->getOutputs().push_back(expression_nodes.back());
prewhere_actions_dag->getOutputs().push_back(expression_nodes[0]);
for (const auto & prewhere_input_node : prewhere_actions_dag->getInputs())
if (required_column_names_without_prewhere.contains(prewhere_input_node->result_name))
@ -354,9 +324,9 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr
}
}
void collectSourceColumns(QueryTreeNodePtr & expression_node, PlannerContextPtr & planner_context, bool keep_alias_columns)
void collectSourceColumns(QueryTreeNodePtr & expression_node, PlannerContextPtr & planner_context)
{
CollectSourceColumnsVisitor collect_source_columns_visitor(planner_context, keep_alias_columns);
CollectSourceColumnsVisitor collect_source_columns_visitor(*planner_context);
collect_source_columns_visitor.visit(expression_node);
}

View File

@ -19,6 +19,6 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr
*
* ALIAS table column nodes are registered in table expression data and replaced in query tree with inner alias expression.
*/
void collectSourceColumns(QueryTreeNodePtr & expression_node, PlannerContextPtr & planner_context, bool keep_alias_columns = true);
void collectSourceColumns(QueryTreeNodePtr & expression_node, PlannerContextPtr & planner_context);
}

View File

@ -451,7 +451,6 @@ private:
std::unordered_map<QueryTreeNodePtr, std::string> node_to_node_name;
const PlannerContextPtr planner_context;
ActionNodeNameHelper action_node_name_helper;
bool use_column_identifier_as_action_node_name;
};
PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag,
@ -459,7 +458,6 @@ PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag,
bool use_column_identifier_as_action_node_name_)
: planner_context(planner_context_)
, action_node_name_helper(node_to_node_name, *planner_context, use_column_identifier_as_action_node_name_)
, use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_)
{
actions_stack.emplace_back(std::move(actions_dag), nullptr);
}
@ -505,8 +503,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
{
auto column_node_name = action_node_name_helper.calculateActionNodeName(node);
const auto & column_node = node->as<ColumnNode &>();
if (column_node.hasExpression() && !use_column_identifier_as_action_node_name)
return visitImpl(column_node.getExpression());
Int64 actions_stack_size = static_cast<Int64>(actions_stack.size() - 1);
for (Int64 i = actions_stack_size; i >= 0; --i)
{

View File

@ -86,7 +86,7 @@ namespace
/// Check if current user has privileges to SELECT columns from table
/// Throws an exception if access to any column from `column_names` is not granted
/// If `column_names` is empty, check access to any columns and return names of accessible columns
NameSet checkAccessRights(const TableNode & table_node, const Names & column_names, const ContextPtr & query_context)
NameSet checkAccessRights(const TableNode & table_node, Names & column_names, const ContextPtr & query_context)
{
/// StorageDummy is created on preliminary stage, ignore access check for it.
if (typeid_cast<const StorageDummy *>(table_node.getStorage().get()))
@ -353,7 +353,9 @@ void prepareBuildQueryPlanForTableExpression(const QueryTreeNodePtr & table_expr
NameSet columns_names_allowed_to_select;
if (table_node)
{
const auto & column_names_with_aliases = table_expression_data.getSelectedColumnsNames();
auto column_names_with_aliases = columns_names;
const auto & alias_columns_names = table_expression_data.getAliasColumnsNames();
column_names_with_aliases.insert(column_names_with_aliases.end(), alias_columns_names.begin(), alias_columns_names.end());
columns_names_allowed_to_select = checkAccessRights(*table_node, column_names_with_aliases, query_context);
}
@ -862,28 +864,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
max_block_size,
max_streams);
const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions();
if (!alias_column_expressions.empty() && query_plan.isInitialized() && from_stage == QueryProcessingStage::FetchColumns)
{
ActionsDAGPtr merged_alias_columns_actions_dag = std::make_shared<ActionsDAG>(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag->getInputs();
for (const auto & [column_name, alias_column_actions_dag] : alias_column_expressions)
{
const auto & current_outputs = alias_column_actions_dag->getOutputs();
action_dag_outputs.insert(action_dag_outputs.end(), current_outputs.begin(), current_outputs.end());
merged_alias_columns_actions_dag->mergeNodes(std::move(*alias_column_actions_dag));
}
for (const auto * output_node : action_dag_outputs)
merged_alias_columns_actions_dag->addOrReplaceInOutputs(*output_node);
merged_alias_columns_actions_dag->removeUnusedActions(false);
auto alias_column_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(merged_alias_columns_actions_dag));
alias_column_step->setStepDescription("Compute alias columns");
query_plan.addStep(std::move(alias_column_step));
}
for (const auto & filter_info_and_description : where_filters)
{
const auto & [filter_info, description] = filter_info_and_description;
@ -927,8 +907,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
else
{
/// Create step which reads from empty source if storage has no data.
const auto & column_names = table_expression_data.getSelectedColumnsNames();
auto source_header = storage_snapshot->getSampleBlockForColumns(column_names);
auto source_header = storage_snapshot->getSampleBlockForColumns(table_expression_data.getColumnNames());
Pipe pipe(std::make_shared<NullSource>(source_header));
auto read_from_pipe = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
read_from_pipe->setStepDescription("Read from NullSource");
@ -1045,6 +1024,57 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP
plan_to_add_cast.addStep(std::move(cast_join_columns_step));
}
/// Actions to calculate table columns that have a functional representation (ALIASes and subcolumns)
/// and used in USING clause of JOIN expression.
struct UsingAliasKeyActions
{
UsingAliasKeyActions(
const ColumnsWithTypeAndName & left_plan_output_columns,
const ColumnsWithTypeAndName & right_plan_output_columns
)
: left_alias_columns_keys(std::make_shared<ActionsDAG>(left_plan_output_columns))
, right_alias_columns_keys(std::make_shared<ActionsDAG>(right_plan_output_columns))
{}
void addLeftColumn(QueryTreeNodePtr & node, const ColumnsWithTypeAndName & plan_output_columns, const PlannerContextPtr & planner_context)
{
addColumnImpl(left_alias_columns_keys, node, plan_output_columns, planner_context);
}
void addRightColumn(QueryTreeNodePtr & node, const ColumnsWithTypeAndName & plan_output_columns, const PlannerContextPtr & planner_context)
{
addColumnImpl(right_alias_columns_keys, node, plan_output_columns, planner_context);
}
ActionsDAGPtr getLeftActions()
{
left_alias_columns_keys->projectInput();
return std::move(left_alias_columns_keys);
}
ActionsDAGPtr getRightActions()
{
right_alias_columns_keys->projectInput();
return std::move(right_alias_columns_keys);
}
private:
void addColumnImpl(ActionsDAGPtr & alias_columns_keys, QueryTreeNodePtr & node, const ColumnsWithTypeAndName & plan_output_columns, const PlannerContextPtr & planner_context)
{
auto & column_node = node->as<ColumnNode&>();
if (column_node.hasExpression())
{
auto dag = buildActionsDAGFromExpressionNode(column_node.getExpressionOrThrow(), plan_output_columns, planner_context);
const auto & left_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(node);
dag->addOrReplaceInOutputs(dag->addAlias(*dag->getOutputs().front(), left_inner_column_identifier));
alias_columns_keys->mergeInplace(std::move(*dag));
}
}
ActionsDAGPtr left_alias_columns_keys;
ActionsDAGPtr right_alias_columns_keys;
};
JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_expression,
JoinTreeQueryPlan left_join_tree_query_plan,
JoinTreeQueryPlan right_join_tree_query_plan,
@ -1113,6 +1143,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
if (join_node.isUsingJoinExpression())
{
UsingAliasKeyActions using_alias_key_actions{left_plan_output_columns, right_plan_output_columns};
auto & join_node_using_columns_list = join_node.getJoinExpression()->as<ListNode &>();
for (auto & join_node_using_node : join_node_using_columns_list.getNodes())
{
@ -1122,9 +1154,13 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
auto & left_inner_column_node = inner_columns_list.getNodes().at(0);
auto & left_inner_column = left_inner_column_node->as<ColumnNode &>();
using_alias_key_actions.addLeftColumn(left_inner_column_node, left_plan_output_columns, planner_context);
auto & right_inner_column_node = inner_columns_list.getNodes().at(1);
auto & right_inner_column = right_inner_column_node->as<ColumnNode &>();
using_alias_key_actions.addRightColumn(right_inner_column_node, right_plan_output_columns, planner_context);
const auto & join_node_using_column_node_type = join_node_using_column_node.getColumnType();
if (!left_inner_column.getColumnType()->equals(*join_node_using_column_node_type))
{
@ -1138,6 +1174,14 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
right_plan_column_name_to_cast_type.emplace(right_inner_column_identifier, join_node_using_column_node_type);
}
}
auto left_alias_columns_keys_step = std::make_unique<ExpressionStep>(left_plan.getCurrentDataStream(), using_alias_key_actions.getLeftActions());
left_alias_columns_keys_step->setStepDescription("Actions for left table alias column keys");
left_plan.addStep(std::move(left_alias_columns_keys_step));
auto right_alias_columns_keys_step = std::make_unique<ExpressionStep>(right_plan.getCurrentDataStream(), using_alias_key_actions.getRightActions());
right_alias_columns_keys_step->setStepDescription("Actions for right table alias column keys");
right_plan.addStep(std::move(right_alias_columns_keys_step));
}
auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map<std::string, DataTypePtr> & plan_column_name_to_cast_type)

View File

@ -55,7 +55,7 @@ public:
/// Return true if column with name exists, false otherwise
bool hasColumn(const std::string & column_name) const
{
return column_name_to_column.contains(column_name);
return alias_columns_names.contains(column_name) || column_name_to_column.contains(column_name);
}
/** Add column in table expression data.
@ -63,40 +63,37 @@ public:
*
* Logical error exception is thrown if column already exists.
*/
void addColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, bool is_selected_column = true)
void addColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier)
{
if (hasColumn(column.name))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Column with name {} already exists", column.name);
column_names.push_back(column.name);
addColumnImpl(column, column_identifier, is_selected_column);
addColumnImpl(column, column_identifier);
}
/// Add alias column
void addAliasColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, ActionsDAGPtr actions_dag, bool is_selected_column = true)
/** Add column if it does not exists in table expression data.
* Column identifier must be created using global planner context.
*/
void addColumnIfNotExists(const NameAndTypePair & column, const ColumnIdentifier & column_identifier)
{
alias_column_expressions.emplace(column.name, std::move(actions_dag));
addColumnImpl(column, column_identifier, is_selected_column);
if (hasColumn(column.name))
return;
addColumnImpl(column, column_identifier);
}
/// Mark existing column as selected
void markSelectedColumn(const std::string & column_name)
/// Add alias column name
void addAliasColumnName(const std::string & column_name, const ColumnIdentifier & column_identifier)
{
auto [_, inserted] = selected_column_names_set.emplace(column_name);
if (inserted)
selected_column_names.push_back(column_name);
alias_columns_names.insert(column_name);
column_name_to_column_identifier.emplace(column_name, column_identifier);
}
/// Get columns that are requested from table expression, including ALIAS columns
const Names & getSelectedColumnsNames() const
/// Get alias columns names
const NameSet & getAliasColumnsNames() const
{
return selected_column_names;
}
/// Get ALIAS columns names mapped to expressions
const std::unordered_map<std::string, ActionsDAGPtr> & getAliasColumnExpressions() const
{
return alias_column_expressions;
return alias_columns_names;
}
/// Get column name to column map
@ -105,7 +102,7 @@ public:
return column_name_to_column;
}
/// Get column names that are read from table expression
/// Get column names
const Names & getColumnNames() const
{
return column_names;
@ -122,6 +119,23 @@ public:
return result;
}
ColumnIdentifiers getColumnIdentifiers() const
{
ColumnIdentifiers result;
result.reserve(column_identifier_to_column_name.size());
for (const auto & [column_identifier, _] : column_identifier_to_column_name)
result.push_back(column_identifier);
return result;
}
/// Get column name to column identifier map
const ColumnNameToColumnIdentifier & getColumnNameToIdentifier() const
{
return column_name_to_column_identifier;
}
/// Get column identifier to column name map
const ColumnNameToColumnIdentifier & getColumnIdentifierToColumnName() const
{
@ -145,6 +159,18 @@ public:
return it->second;
}
/** Get column for column name.
* Null is returned if there are no column for column name.
*/
const NameAndTypePair * getColumnOrNull(const std::string & column_name) const
{
auto it = column_name_to_column.find(column_name);
if (it == column_name_to_column.end())
return nullptr;
return &it->second;
}
/** Get column identifier for column name.
* Exception is thrown if there are no column identifier for column name.
*/
@ -174,6 +200,24 @@ public:
return &it->second;
}
/** Get column name for column identifier.
* Exception is thrown if there are no column name for column identifier.
*/
const std::string & getColumnNameOrThrow(const ColumnIdentifier & column_identifier) const
{
auto it = column_identifier_to_column_name.find(column_identifier);
if (it == column_identifier_to_column_name.end())
{
auto column_identifiers = getColumnIdentifiers();
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Column name for column identifier {} does not exists. There are only column identifiers: {}",
column_identifier,
fmt::join(column_identifiers.begin(), column_identifiers.end(), ", "));
}
return it->second;
}
/** Get column name for column identifier.
* Null is returned if there are no column name for column identifier.
*/
@ -252,36 +296,23 @@ public:
}
private:
void addColumnImpl(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, bool add_to_selected_columns)
void addColumnImpl(const NameAndTypePair & column, const ColumnIdentifier & column_identifier)
{
if (add_to_selected_columns)
markSelectedColumn(column.name);
column_names.push_back(column.name);
column_name_to_column.emplace(column.name, column);
column_name_to_column_identifier.emplace(column.name, column_identifier);
column_identifier_to_column_name.emplace(column_identifier, column.name);
}
/// Set of columns that are physically read from table expression
/// In case of ALIAS columns it contains source column names that are used to calculate alias
/// This source column may be not used by user
/// Valid for table, table function, array join, query, union nodes
Names column_names;
/// Set of columns that are SELECTed from table expression
/// It may contain ALIAS columns.
/// Mainly it's used to determine access to which columns to check
/// For example user may have an access to column `a ALIAS x + y` but not to `x` and `y`
/// In that case we can read `x` and `y` and calculate `a`, but not return `x` and `y` to user
Names selected_column_names;
/// To deduplicate columns in `selected_column_names`
NameSet selected_column_names_set;
/// Expression to calculate ALIAS columns
std::unordered_map<std::string, ActionsDAGPtr> alias_column_expressions;
/// Valid for table, table function, array join, query, union nodes
ColumnNameToColumn column_name_to_column;
/// Valid only for table node
NameSet alias_columns_names;
/// Valid for table, table function, array join, query, union nodes
ColumnNameToColumnIdentifier column_name_to_column_identifier;

View File

@ -469,19 +469,12 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression,
NameSet table_expression_required_names_without_filter)
{
const auto & query_context = planner_context->getQueryContext();
auto filter_query_tree = buildQueryTree(filter_expression, query_context);
QueryAnalysisPass query_analysis_pass(table_expression);
query_analysis_pass.run(filter_query_tree, query_context);
return buildFilterInfo(std::move(filter_query_tree), table_expression, planner_context, std::move(table_expression_required_names_without_filter));
}
FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree,
const QueryTreeNodePtr & table_expression,
PlannerContextPtr & planner_context,
NameSet table_expression_required_names_without_filter)
{
if (table_expression_required_names_without_filter.empty())
{
auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression);
@ -489,7 +482,7 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree,
table_expression_required_names_without_filter.insert(table_expression_names.begin(), table_expression_names.end());
}
collectSourceColumns(filter_query_tree, planner_context, false /*keep_alias_columns*/);
collectSourceColumns(filter_query_tree, planner_context);
collectSets(filter_query_tree, *planner_context);
auto filter_actions_dag = std::make_shared<ActionsDAG>();

View File

@ -89,11 +89,6 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression,
PlannerContextPtr & planner_context,
NameSet table_expression_required_names_without_filter = {});
FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree,
const QueryTreeNodePtr & table_expression,
PlannerContextPtr & planner_context,
NameSet table_expression_required_names_without_filter = {});
ASTPtr parseAdditionalResultFilter(const Settings & settings);
}

View File

@ -1434,13 +1434,8 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes)
if (query_info.planner_context)
{
const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression);
const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions();
for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName())
{
/// ALIAS columns cannot be used in the filter expression without being calculated in ActionsDAG,
/// so they should not be added to the input nodes.
if (alias_column_expressions.contains(column_name))
continue;
const auto & column = table_expression_data.getColumnOrThrow(column_name);
node_name_to_input_node_column.emplace(column_identifier, ColumnWithTypeAndName(column.type, column_name));
}

View File

@ -744,32 +744,6 @@ StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery(
namespace
{
class ReplaseAliasColumnsVisitor : public InDepthQueryTreeVisitor<ReplaseAliasColumnsVisitor>
{
static QueryTreeNodePtr getColumnNodeAliasExpression(const QueryTreeNodePtr & node)
{
const auto * column_node = node->as<ColumnNode>();
if (!column_node || !column_node->hasExpression())
return nullptr;
const auto & column_source = column_node->getColumnSourceOrNull();
if (!column_source || column_source->getNodeType() == QueryTreeNodeType::JOIN
|| column_source->getNodeType() == QueryTreeNodeType::ARRAY_JOIN)
return nullptr;
auto column_expression = column_node->getExpression();
column_expression->setAlias(column_node->getColumnName());
return column_expression;
}
public:
void visitImpl(QueryTreeNodePtr & node)
{
if (auto column_expression = getColumnNodeAliasExpression(node))
node = column_expression;
}
};
QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info,
const StorageSnapshotPtr & distributed_storage_snapshot,
const StorageID & remote_storage_id,
@ -822,8 +796,6 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info,
replacement_table_expression->setAlias(query_info.table_expression->getAlias());
auto query_tree_to_modify = query_info.query_tree->cloneAndReplace(query_info.table_expression, std::move(replacement_table_expression));
ReplaseAliasColumnsVisitor replase_alias_columns_visitor;
replase_alias_columns_visitor.visit(query_tree_to_modify);
return buildQueryTreeForShard(query_info.planner_context, query_tree_to_modify);
}

View File

@ -4,3 +4,4 @@ test_distributed_type_object/test.py::test_distributed_type_object
test_merge_table_over_distributed/test.py::test_global_in
test_merge_table_over_distributed/test.py::test_select_table_name_from_merge_over_distributed
test_passing_max_partitions_to_read_remotely/test.py::test_default_database_on_cluster
test_select_access_rights/test_main.py::test_alias_columns

View File

@ -21,6 +21,7 @@ class Labels(metaclass=WithIter):
CI_SET_REDUCED = "ci_set_reduced"
CI_SET_ARM = "ci_set_arm"
CI_SET_INTEGRATION = "ci_set_integration"
CI_SET_ANALYZER = "ci_set_analyzer"
libFuzzer = "libFuzzer"
@ -398,6 +399,10 @@ bugfix_validate_check = DigestConfig(
],
)
# common test params
compatibility_test_common_params = {
"digest": compatibility_check_digest,
"run_command": "compatibility_check.py",
}
statless_test_common_params = {
"digest": stateless_check_digest,
"run_command": 'functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT',
@ -647,6 +652,16 @@ CI_CONFIG = CIConfig(
JobNames.INTEGRATION_TEST,
]
),
Labels.CI_SET_ANALYZER: LabelConfig(
run_jobs=[
JobNames.STYLE_CHECK,
JobNames.FAST_TEST,
Build.PACKAGE_RELEASE,
Build.PACKAGE_ASAN,
JobNames.STATELESS_TEST_ANALYZER_S3_REPLICATED_RELEASE,
JobNames.INTEGRATION_TEST_ASAN_ANALYZER,
]
),
Labels.CI_SET_REDUCED: LabelConfig(
run_jobs=[
job
@ -1038,13 +1053,13 @@ CI_CONFIG = CIConfig(
JobNames.COMPATIBILITY_TEST: TestConfig(
Build.PACKAGE_RELEASE,
job_config=JobConfig(
required_on_release_branch=True, digest=compatibility_check_digest
required_on_release_branch=True, **compatibility_test_common_params # type: ignore
),
),
JobNames.COMPATIBILITY_TEST_ARM: TestConfig(
Build.PACKAGE_AARCH64,
job_config=JobConfig(
required_on_release_branch=True, digest=compatibility_check_digest
required_on_release_branch=True, **compatibility_test_common_params # type: ignore
),
),
JobNames.UNIT_TEST: TestConfig(

View File

@ -2,6 +2,7 @@
import argparse
import logging
import os
import subprocess
import sys
from pathlib import Path
@ -122,11 +123,7 @@ def get_run_commands_distributions(
def parse_args():
parser = argparse.ArgumentParser("Check compatibility with old distributions")
parser.add_argument("--check-name", required=True)
parser.add_argument("--check-glibc", action="store_true")
parser.add_argument(
"--check-distributions", action="store_true"
) # currently hardcoded to x86, don't enable for ARM
parser.add_argument("--check-name", required=False)
return parser.parse_args()
@ -134,6 +131,13 @@ def main():
logging.basicConfig(level=logging.INFO)
args = parse_args()
check_name = args.check_name or os.getenv("CHECK_NAME")
assert check_name
check_glibc = True
# currently hardcoded to x86, don't enable for ARM
check_distributions = (
"aarch64" not in check_name.lower() and "arm64" not in check_name.lower()
)
stopwatch = Stopwatch()
@ -150,7 +154,7 @@ def main():
"clickhouse-common-static_" in url or "clickhouse-server_" in url
)
download_builds_filter(args.check_name, reports_path, packages_path, url_filter)
download_builds_filter(check_name, reports_path, packages_path, url_filter)
for package in packages_path.iterdir():
if package.suffix == ".deb":
@ -166,11 +170,11 @@ def main():
run_commands = []
if args.check_glibc:
if check_glibc:
check_glibc_commands = get_run_commands_glibc(packages_path, result_path)
run_commands.extend(check_glibc_commands)
if args.check_distributions:
if check_distributions:
centos_image = pull_image(get_docker_image(IMAGE_CENTOS))
ubuntu_image = pull_image(get_docker_image(IMAGE_UBUNTU))
check_distributions_commands = get_run_commands_distributions(
@ -195,9 +199,9 @@ def main():
# See https://sourceware.org/glibc/wiki/Glibc%20Timeline
max_glibc_version = ""
if "amd64" in args.check_name:
if "amd64" in check_name:
max_glibc_version = "2.4"
elif "aarch64" in args.check_name:
elif "aarch64" in check_name:
max_glibc_version = "2.18" # because of build with newer sysroot?
else:
raise Exception("Can't determine max glibc version")
@ -205,8 +209,8 @@ def main():
state, description, test_results, additional_logs = process_result(
result_path,
server_log_path,
args.check_glibc,
args.check_distributions,
check_glibc,
check_distributions,
max_glibc_version,
)

View File

@ -25,7 +25,6 @@ from stopwatch import Stopwatch
from tee_popen import TeePopen
from upload_result_helper import upload_results
NAME = "Push to Dockerhub"
TEMP_PATH = Path(RUNNER_TEMP) / "docker_images_check"
TEMP_PATH.mkdir(parents=True, exist_ok=True)
@ -177,6 +176,9 @@ def main():
stopwatch = Stopwatch()
args = parse_args()
NAME = f"Push to Dockerhub {args.suffix}"
if args.push:
logging.info("login to docker hub")
docker_login()

View File

@ -41,7 +41,7 @@ def started_cluster():
CREATE TABLE mydb.filtered_table2 (a UInt8, b UInt8, c UInt8, d UInt8) ENGINE MergeTree ORDER BY a;
INSERT INTO mydb.filtered_table2 values (0, 0, 0, 0), (1, 2, 3, 4), (4, 3, 2, 1), (0, 0, 6, 0);
CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, bb ALIAS b + 1, c UInt16 ALIAS a + bb - 1) ENGINE MergeTree ORDER BY a;
CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a;
INSERT INTO mydb.filtered_table3 values (0, 0), (0, 1), (1, 0), (1, 1);
CREATE TABLE mydb.`.filtered_table4` (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a;

View File

@ -60,7 +60,7 @@ def started_cluster():
CREATE TABLE mydb.filtered_table2 (a UInt8, b UInt8, c UInt8, d UInt8) ENGINE MergeTree ORDER BY a;
INSERT INTO mydb.filtered_table2 values (0, 0, 0, 0), (1, 2, 3, 4), (4, 3, 2, 1), (0, 0, 6, 0);
CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, bb ALIAS b + 1, c UInt16 ALIAS a + bb - 1) ENGINE MergeTree ORDER BY a;
CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a;
INSERT INTO mydb.filtered_table3 values (0, 0), (0, 1), (1, 0), (1, 1);
CREATE TABLE mydb.`.filtered_table4` (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a;
@ -113,7 +113,6 @@ def test_smoke():
assert node.query("SELECT a FROM mydb.filtered_table3") == TSV([[0], [1]])
assert node.query("SELECT b FROM mydb.filtered_table3") == TSV([[1], [0]])
assert node.query("SELECT bb FROM mydb.filtered_table3") == TSV([[2], [1]])
assert node.query("SELECT c FROM mydb.filtered_table3") == TSV([[1], [1]])
assert node.query("SELECT a + b FROM mydb.filtered_table3") == TSV([[1], [1]])
assert node.query("SELECT a FROM mydb.filtered_table3 WHERE c = 1") == TSV(

View File

@ -1,63 +1,63 @@
1
1979-12-12 21:21:21.127 3.7 0
2299-12-31 23:37:36.788 1.1 0
2299-12-31 23:37:36.789 2.34 0
1979-12-12 21:21:21.123 1.1 0
1979-12-12 21:21:21.123 2.34 0
1979-12-12 21:21:21.127 3.7 340.00000000000006
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:22.000 1.3345 0
1979-12-12 21:21:23.000 1.54 0.20550000000000002
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.127 3.7 0
2299-12-31 23:37:36.788 1.1 0
2299-12-31 23:37:36.789 2.34 0
1979-12-12 21:21:21.123 1.1 0
1979-12-12 21:21:21.123 2.34 0
1979-12-12 21:21:21.127 3.7 0.0000010200000000000004
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:22.000 1.3345 0
1979-12-12 21:21:23.000 1.54 6.165000000000001e-10
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.127 3.7 0
2299-12-31 23:37:36.788 1.1 0
2299-12-31 23:37:36.789 2.34 0
1979-12-12 21:21:21.123 1.1 0
1979-12-12 21:21:21.123 2.34 0
1979-12-12 21:21:21.127 3.7 0.00136
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:22.000 1.3345 0
1979-12-12 21:21:23.000 1.54 8.22e-7
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.127 3.7 0
2299-12-31 23:37:36.788 1.1 0
2299-12-31 23:37:36.789 2.34 0
1979-12-12 21:21:21.123 1.1 0
1979-12-12 21:21:21.123 2.34 0
1979-12-12 21:21:21.127 3.7 1.7000000000000004
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:22.000 1.3345 0
1979-12-12 21:21:23.000 1.54 0.0010275000000000002
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.127 3.7 0
2299-12-31 23:37:36.788 1.1 0
2299-12-31 23:37:36.789 2.34 0
1979-12-12 21:21:21.123 1.1 0
1979-12-12 21:21:21.123 2.34 0
1979-12-12 21:21:21.127 3.7 2040.0000000000005
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:22.000 1.3345 0
1979-12-12 21:21:23.000 1.54 1.233
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.127 3.7 0
2299-12-31 23:37:36.788 1.1 0
2299-12-31 23:37:36.789 2.34 0
1979-12-12 21:21:21.123 1.1 0
1979-12-12 21:21:21.123 2.34 0
1979-12-12 21:21:21.127 3.7 142800.00000000003
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:22.000 1.3345 0
1979-12-12 21:21:23.000 1.54 86.31
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.127 3.7 0
2299-12-31 23:37:36.788 1.1 0
2299-12-31 23:37:36.789 2.34 0
1979-12-12 21:21:21.123 1.1 0
1979-12-12 21:21:21.123 2.34 0
1979-12-12 21:21:21.127 3.7 9792000.000000002
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:22.000 1.3345 0
1979-12-12 21:21:23.000 1.54 5918.400000000001
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.127 3.7 0
2299-12-31 23:37:36.788 1.1 0
2299-12-31 23:37:36.789 2.34 0
1979-12-12 21:21:21.123 1.1 0
1979-12-12 21:21:21.123 2.34 0
1979-12-12 21:21:21.127 3.7 264384000.00000003
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:22.000 1.3345 0
1979-12-12 21:21:23.000 1.54 159796.80000000002
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.127 3.7 0
2299-12-31 23:37:36.788 1.1 0
2299-12-31 23:37:36.789 2.34 0
1979-12-12 21:21:21.123 1.1 0
1979-12-12 21:21:21.123 2.34 0
1979-12-12 21:21:21.127 3.7 2056320000.0000002
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:22.000 1.3345 0
1979-12-12 21:21:23.000 1.54 1242864

View File

@ -18,7 +18,7 @@ SELECT (
SELECT
ts,
metric,
nonNegativeDerivative(metric, ts) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv
nonNegativeDerivative(metric, ts) OVER (PARTITION BY metric ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv
FROM nnd
LIMIT 5, 1
) = (
@ -29,37 +29,37 @@ SELECT (
FROM nnd
LIMIT 5, 1
);
SELECT ts, metric, nonNegativeDerivative(metric, ts) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS deriv FROM nnd;
SELECT ts, metric, nonNegativeDerivative(metric, ts) OVER (PARTITION BY id>3 ORDER BY ts, metric ASC Rows BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS deriv FROM nnd;
-- Nanosecond
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 3 NANOSECOND) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN 2 PRECEDING AND 2 FOLLOWING) AS deriv FROM nnd;
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 3 NANOSECOND) OVER (PARTITION BY id>3 ORDER BY ts, metric ASC Rows BETWEEN 2 PRECEDING AND 2 FOLLOWING) AS deriv FROM nnd;
-- Microsecond
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 4 MICROSECOND) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS deriv FROM nnd;
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 4 MICROSECOND) OVER (PARTITION BY id>3 ORDER BY ts, metric ASC Rows BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS deriv FROM nnd;
-- Millisecond
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 5 MILLISECOND) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS deriv FROM nnd;
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 5 MILLISECOND) OVER (PARTITION BY id>3 ORDER BY ts, metric ASC Rows BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS deriv FROM nnd;
-- Second
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 6 SECOND) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 6 SECOND) OVER (PARTITION BY id>3 ORDER BY ts, metric ASC Rows BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
-- Minute
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 7 MINUTE) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND 2 FOLLOWING) AS deriv FROM nnd;
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 7 MINUTE) OVER (PARTITION BY id>3 ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND 2 FOLLOWING) AS deriv FROM nnd;
-- Hour
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 8 HOUR) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 8 HOUR) OVER (PARTITION BY id>3 ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
-- Day
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 9 DAY) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN 3 PRECEDING AND 3 FOLLOWING) AS deriv FROM nnd;
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 9 DAY) OVER (PARTITION BY id>3 ORDER BY ts, metric ASC Rows BETWEEN 3 PRECEDING AND 3 FOLLOWING) AS deriv FROM nnd;
-- Week
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 10 WEEK) OVER (PARTITION BY id>3 ORDER BY ts ASC Rows BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 10 WEEK) OVER (PARTITION BY id>3 ORDER BY ts, metric ASC Rows BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
-- shall not work for month, quarter, year (intervals with floating number of seconds)
-- Month
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 11 MONTH) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 11 MONTH) OVER (PARTITION BY metric ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- Quarter
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 12 QUARTER) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 12 QUARTER) OVER (PARTITION BY metric ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- Year
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 13 YEAR) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 13 YEAR) OVER (PARTITION BY metric ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- test against wrong arguments/types
SELECT ts, metric, nonNegativeDerivative(metric, 1, INTERVAL 3 NANOSECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS }
SELECT ts, metric, nonNegativeDerivative('string not datetime', ts, INTERVAL 3 NANOSECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS }
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 3 NANOSECOND, id) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS }
SELECT ts, metric, nonNegativeDerivative(metric) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS }
SELECT ts, metric, nonNegativeDerivative(metric, 1, INTERVAL 3 NANOSECOND) OVER (PARTITION BY metric ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS }
SELECT ts, metric, nonNegativeDerivative('string not datetime', ts, INTERVAL 3 NANOSECOND) OVER (PARTITION BY metric ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS }
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 3 NANOSECOND, id) OVER (PARTITION BY metric ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS }
SELECT ts, metric, nonNegativeDerivative(metric) OVER (PARTITION BY metric ORDER BY ts, metric ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS }
-- cleanup
DROP TABLE IF EXISTS nnd;

View File

@ -55,33 +55,33 @@ Header: a2 String
Header: __table1.a2 String
__table1.k UInt64
__table4.d2 String
Expression (DROP unused columns after JOIN)
Expression ((Actions for left table alias column keys + DROP unused columns after JOIN))
Header: __table1.a2 String
__table1.k UInt64
Join (JOIN FillRightFirst)
Header: __table1.a2 String
__table1.k UInt64
Expression (DROP unused columns after JOIN)
Expression ((Actions for left table alias column keys + DROP unused columns after JOIN))
Header: __table1.a2 String
__table1.k UInt64
Join (JOIN FillRightFirst)
Header: __table1.a2 String
__table1.k UInt64
Expression (Change column names to column identifiers)
Expression ((Actions for left table alias column keys + Change column names to column identifiers))
Header: __table1.a2 String
__table1.k UInt64
ReadFromMemoryStorage
Header: a2 String
k UInt64
Expression (Change column names to column identifiers)
Expression ((Actions for right table alias column keys + Change column names to column identifiers))
Header: __table2.k UInt64
ReadFromMemoryStorage
Header: k UInt64
Expression (Change column names to column identifiers)
Expression ((Actions for right table alias column keys + Change column names to column identifiers))
Header: __table3.k UInt64
ReadFromMemoryStorage
Header: k UInt64
Expression (Change column names to column identifiers)
Expression ((Actions for right table alias column keys + Change column names to column identifiers))
Header: __table4.d2 String
__table4.k UInt64
ReadFromMemoryStorage

View File

@ -1,37 +0,0 @@
The reference time point is 2023-06-30 23:59:30
───────────────────────────────────────────────
The argument is before the reference time point
───────────────────────────────────────────────
Row 1:
──────
syslog_arg: Jun 30 23:58:30
res: 2023-06-30 23:58:30
res_null: 2023-06-30 23:58:30
res_zero: 2023-06-30 23:58:30
res_us: 2023-06-30 23:58:30
res_us_null: 2023-06-30 23:58:30
res_us_zero: 2023-06-30 23:58:30
res64: 2023-06-30 23:58:30.000
res64_null: 2023-06-30 23:58:30.000
res64_zero: 2023-06-30 23:58:30.000
res64_us: 2023-06-30 23:58:30.000
res64_us_null: 2023-06-30 23:58:30.000
res64_us_zero: 2023-06-30 23:58:30.000
──────────────────────────────────────────────
The argument is after the reference time point
──────────────────────────────────────────────
Row 1:
──────
syslog_arg: Jul 1 00:00:30
res: 2022-07-01 00:00:30
res_null: 2022-07-01 00:00:30
res_zero: 2022-07-01 00:00:30
res_us: 2022-07-01 00:00:30
res_us_null: 2022-07-01 00:00:30
res_us_zero: 2022-07-01 00:00:30
res64: 2022-07-01 00:00:30.000
res64_null: 2022-07-01 00:00:30.000
res64_zero: 2022-07-01 00:00:30.000
res64_us: 2022-07-01 00:00:30.000
res64_us_null: 2022-07-01 00:00:30.000
res64_us_zero: 2022-07-01 00:00:30.000

View File

@ -1,54 +0,0 @@
SET session_timezone = 'UTC';
SELECT 'The reference time point is 2023-06-30 23:59:30';
SELECT '───────────────────────────────────────────────';
SELECT 'The argument is before the reference time point';
SELECT '───────────────────────────────────────────────';
WITH
toDateTime('2023-06-30 23:59:30') AS dt_ref,
now() AS dt_now,
date_sub(MINUTE, 1, dt_now) as dt_before,
dateDiff(SECOND, dt_ref, dt_now) AS time_shift,
formatDateTime(dt_before, '%b %e %T') AS syslog_before
SELECT
formatDateTime(dt_before - time_shift, '%b %e %T') AS syslog_arg,
parseDateTimeBestEffort(syslog_before) - time_shift AS res,
parseDateTimeBestEffortOrNull(syslog_before) - time_shift AS res_null,
parseDateTimeBestEffortOrZero(syslog_before) - time_shift AS res_zero,
parseDateTimeBestEffortUS(syslog_before) - time_shift AS res_us,
parseDateTimeBestEffortUSOrNull(syslog_before) - time_shift AS res_us_null,
parseDateTimeBestEffortUSOrZero(syslog_before) - time_shift AS res_us_zero,
parseDateTime64BestEffort(syslog_before) - time_shift AS res64,
parseDateTime64BestEffortOrNull(syslog_before) - time_shift AS res64_null,
parseDateTime64BestEffortOrZero(syslog_before) - time_shift AS res64_zero,
parseDateTime64BestEffortUS(syslog_before) - time_shift AS res64_us,
parseDateTime64BestEffortUSOrNull(syslog_before) - time_shift AS res64_us_null,
parseDateTime64BestEffortUSOrZero(syslog_before) - time_shift AS res64_us_zero
FORMAT Vertical;
SELECT '──────────────────────────────────────────────';
SELECT 'The argument is after the reference time point';
SELECT '──────────────────────────────────────────────';
WITH
toDateTime('2023-06-30 23:59:30') AS dt_ref,
now() AS dt_now,
date_add(MINUTE, 1, dt_now) as dt_after,
dateDiff(SECOND, dt_ref, dt_now) AS time_shift,
formatDateTime(dt_after, '%b %e %T') AS syslog_after
SELECT
formatDateTime(dt_after - time_shift, '%b %e %T') AS syslog_arg,
parseDateTimeBestEffort(syslog_after) - time_shift AS res,
parseDateTimeBestEffortOrNull(syslog_after) - time_shift AS res_null,
parseDateTimeBestEffortOrZero(syslog_after) - time_shift AS res_zero,
parseDateTimeBestEffortUS(syslog_after) - time_shift AS res_us,
parseDateTimeBestEffortUSOrNull(syslog_after) - time_shift AS res_us_null,
parseDateTimeBestEffortUSOrZero(syslog_after) - time_shift AS res_us_zero,
parseDateTime64BestEffort(syslog_after) - time_shift AS res64,
parseDateTime64BestEffortOrNull(syslog_after) - time_shift AS res64_null,
parseDateTime64BestEffortOrZero(syslog_after) - time_shift AS res64_zero,
parseDateTime64BestEffortUS(syslog_after) - time_shift AS res64_us,
parseDateTime64BestEffortUSOrNull(syslog_after) - time_shift AS res64_us_null,
parseDateTime64BestEffortUSOrZero(syslog_after) - time_shift AS res64_us_zero
FORMAT Vertical;

View File

@ -14,13 +14,13 @@ Expression ((Projection + Before ORDER BY))
Parts: 1/1
Granules: 1/1
Expression ((Project names + Projection))
Filter ((WHERE + (Change column names to column identifiers + Compute alias columns)))
Filter ((WHERE + Change column names to column identifiers))
ReadFromMergeTree (02911_support_alias_column_in_indices.test1)
Indexes:
PrimaryKey
Keys:
c
Condition: (plus(c, 1) in [11, +Inf))
Condition: (_CAST(plus(c, \'UInt64\'), 1) in [11, +Inf))
Parts: 1/2
Granules: 1/2
Skip
@ -44,17 +44,12 @@ Expression ((Projection + Before ORDER BY))
Parts: 1/1
Granules: 1/1
Expression ((Project names + Projection))
Filter ((WHERE + (Change column names to column identifiers + Compute alias columns)))
Filter ((WHERE + Change column names to column identifiers))
ReadFromMergeTree (02911_support_alias_column_in_indices.test2)
Indexes:
PrimaryKey
Keys:
c
Condition: (plus(plus(c, 1), 1) in [16, +Inf))
Condition: (_CAST(plus(_CAST(plus(c, \'UInt64\'), 1), \'UInt64\'), 1) in [16, +Inf))
Parts: 1/2
Granules: 1/2
Skip
Name: i
Description: minmax GRANULARITY 1
Parts: 1/1
Granules: 1/1

View File

@ -0,0 +1,100 @@
2023-01-01 00:00:00
2023-01-01 00:00:00
2023-01-01 01:01:01
2023-01-01 01:01:01
2023-01-02 02:02:02
2023-01-02 02:02:02
2023-01-03 03:03:03
2023-01-03 03:03:03
2023-01-04 04:04:04
2023-01-04 04:04:04
2023-01-05 05:05:05
2023-01-05 05:05:05
2023-01-06 06:06:06
2023-01-06 06:06:06
2023-01-07 07:07:07
2023-01-07 07:07:07
2023-01-08 08:08:08
2023-01-08 08:08:08
2023-01-09 09:09:09
2023-01-09 09:09:09
2023-01-01 00:00:00.00
2023-01-01 00:00:00.00
2023-01-01 01:01:01.00
2023-01-01 01:01:01.10
2023-01-02 02:02:02.00
2023-01-02 02:02:02.12
2023-01-03 03:03:03.00
2023-01-03 03:03:03.12
2023-01-04 04:04:04.00
2023-01-04 04:04:04.12
2023-01-05 05:05:05.00
2023-01-05 05:05:05.12
2023-01-06 06:06:06.00
2023-01-06 06:06:06.12
2023-01-07 07:07:07.00
2023-01-07 07:07:07.12
2023-01-08 08:08:08.00
2023-01-08 08:08:08.12
2023-01-09 09:09:09.00
2023-01-09 09:09:09.12
2023-01-01 00:00:00.000
2023-01-01 00:00:00.000
2023-01-01 01:01:01.000
2023-01-01 01:01:01.100
2023-01-02 02:02:02.000
2023-01-02 02:02:02.120
2023-01-03 03:03:03.000
2023-01-03 03:03:03.123
2023-01-04 04:04:04.000
2023-01-04 04:04:04.123
2023-01-05 05:05:05.000
2023-01-05 05:05:05.123
2023-01-06 06:06:06.000
2023-01-06 06:06:06.123
2023-01-07 07:07:07.000
2023-01-07 07:07:07.123
2023-01-08 08:08:08.000
2023-01-08 08:08:08.123
2023-01-09 09:09:09.000
2023-01-09 09:09:09.123
2023-01-01 00:00:00.000000
2023-01-01 00:00:00.000000
2023-01-01 01:01:01.000000
2023-01-01 01:01:01.100000
2023-01-02 02:02:02.000000
2023-01-02 02:02:02.120000
2023-01-03 03:03:03.000000
2023-01-03 03:03:03.123000
2023-01-04 04:04:04.000000
2023-01-04 04:04:04.123400
2023-01-05 05:05:05.000000
2023-01-05 05:05:05.123450
2023-01-06 06:06:06.000000
2023-01-06 06:06:06.123456
2023-01-07 07:07:07.000000
2023-01-07 07:07:07.123456
2023-01-08 08:08:08.000000
2023-01-08 08:08:08.123456
2023-01-09 09:09:09.000000
2023-01-09 09:09:09.123456
2023-01-01 00:00:00.000000
2023-01-01 00:00:00.000000
2023-01-01 01:01:01.000000
2023-01-01 01:01:01.100000
2023-01-02 02:02:02.000000
2023-01-02 02:02:02.120000
2023-01-03 03:03:03.000000
2023-01-03 03:03:03.123000
2023-01-04 04:04:04.000000
2023-01-04 04:04:04.123400
2023-01-05 05:05:05.000000
2023-01-05 05:05:05.123450
2023-01-06 06:06:06.000000
2023-01-06 06:06:06.123456
2023-01-07 07:07:07.000000
2023-01-07 07:07:07.123456
2023-01-08 08:08:08.000000
2023-01-08 08:08:08.123456
2023-01-09 09:09:09.000000
2023-01-09 09:09:09.123456

View File

@ -0,0 +1,124 @@
DROP TABLE IF EXISTS test_0;
CREATE TABLE IF NOT EXISTS test_0 (a DateTime64(0)) engine = MergeTree order by a;
INSERT INTO test_0 VALUES (toDateTime64('2023-01-01 00:00:00', 0));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-01 00:00:00.123456789', 0));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-01 01:01:01', 1));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-01 01:01:01.123456789', 1));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-02 02:02:02', 2));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-02 02:02:02.123456789', 2));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-03 03:03:03', 3));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-03 03:03:03.123456789', 3));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-04 04:04:04', 4));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-04 04:04:04.123456789', 4));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-05 05:05:05', 5));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-05 05:05:05.123456789', 5));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-06 06:06:06', 6));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-06 06:06:06.123456789', 6));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-07 07:07:07', 7));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-07 07:07:07.123456789', 7));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-08 08:08:08', 8));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-08 08:08:08.123456789', 8));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-09 09:09:09', 9));
INSERT INTO test_0 VALUES (toDateTime64('2023-01-09 09:09:09.123456789', 9));
SELECT * FROM test_0 ORDER BY a;
DROP TABLE test_0;
DROP TABLE IF EXISTS test_2;
CREATE TABLE IF NOT EXISTS test_2 (a DateTime64(2)) engine = MergeTree order by a;
INSERT INTO test_2 VALUES (toDateTime64('2023-01-01 00:00:00', 0));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-01 00:00:00.123456789', 0));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-01 01:01:01', 1));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-01 01:01:01.123456789', 1));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-02 02:02:02', 2));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-02 02:02:02.123456789', 2));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-03 03:03:03', 3));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-03 03:03:03.123456789', 3));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-04 04:04:04', 4));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-04 04:04:04.123456789', 4));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-05 05:05:05', 5));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-05 05:05:05.123456789', 5));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-06 06:06:06', 6));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-06 06:06:06.123456789', 6));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-07 07:07:07', 7));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-07 07:07:07.123456789', 7));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-08 08:08:08', 8));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-08 08:08:08.123456789', 8));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-09 09:09:09', 9));
INSERT INTO test_2 VALUES (toDateTime64('2023-01-09 09:09:09.123456789', 9));
SELECT * FROM test_2 ORDER BY a;
DROP TABLE test_2;
DROP TABLE IF EXISTS test_3;
CREATE TABLE IF NOT EXISTS test_3 (a DateTime64(3)) engine = MergeTree order by a;
INSERT INTO test_3 VALUES (toDateTime64('2023-01-01 00:00:00', 0));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-01 00:00:00.123456789', 0));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-01 01:01:01', 1));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-01 01:01:01.123456789', 1));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-02 02:02:02', 2));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-02 02:02:02.123456789', 2));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-03 03:03:03', 3));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-03 03:03:03.123456789', 3));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-04 04:04:04', 4));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-04 04:04:04.123456789', 4));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-05 05:05:05', 5));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-05 05:05:05.123456789', 5));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-06 06:06:06', 6));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-06 06:06:06.123456789', 6));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-07 07:07:07', 7));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-07 07:07:07.123456789', 7));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-08 08:08:08', 8));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-08 08:08:08.123456789', 8));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-09 09:09:09', 9));
INSERT INTO test_3 VALUES (toDateTime64('2023-01-09 09:09:09.123456789', 9));
SELECT * FROM test_3 ORDER BY a;
DROP TABLE test_3;
DROP TABLE IF EXISTS test_6;
CREATE TABLE IF NOT EXISTS test_6 (a DateTime64(6)) engine = MergeTree order by a;
INSERT INTO test_6 VALUES (toDateTime64('2023-01-01 00:00:00', 0));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-01 00:00:00.123456789', 0));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-01 01:01:01', 1));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-01 01:01:01.123456789', 1));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-02 02:02:02', 2));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-02 02:02:02.123456789', 2));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-03 03:03:03', 3));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-03 03:03:03.123456789', 3));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-04 04:04:04', 4));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-04 04:04:04.123456789', 4));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-05 05:05:05', 5));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-05 05:05:05.123456789', 5));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-06 06:06:06', 6));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-06 06:06:06.123456789', 6));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-07 07:07:07', 7));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-07 07:07:07.123456789', 7));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-08 08:08:08', 8));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-08 08:08:08.123456789', 8));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-09 09:09:09', 9));
INSERT INTO test_6 VALUES (toDateTime64('2023-01-09 09:09:09.123456789', 9));
SELECT * FROM test_6 ORDER BY a;
DROP TABLE test_6;
DROP TABLE IF EXISTS test_9;
CREATE TABLE IF NOT EXISTS test_9 (a DateTime64(6)) engine = MergeTree order by a;
INSERT INTO test_9 VALUES (toDateTime64('2023-01-01 00:00:00', 0));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-01 00:00:00.123456789', 0));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-01 01:01:01', 1));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-01 01:01:01.123456789', 1));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-02 02:02:02', 2));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-02 02:02:02.123456789', 2));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-03 03:03:03', 3));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-03 03:03:03.123456789', 3));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-04 04:04:04', 4));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-04 04:04:04.123456789', 4));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-05 05:05:05', 5));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-05 05:05:05.123456789', 5));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-06 06:06:06', 6));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-06 06:06:06.123456789', 6));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-07 07:07:07', 7));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-07 07:07:07.123456789', 7));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-08 08:08:08', 8));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-08 08:08:08.123456789', 8));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-09 09:09:09', 9));
INSERT INTO test_9 VALUES (toDateTime64('2023-01-09 09:09:09.123456789', 9));
SELECT * FROM test_9 ORDER BY a;
DROP TABLE test_9;

View File

@ -0,0 +1,21 @@
CREATE TABLE attach_partition_t7 (
a UInt32,
b UInt32
)
ENGINE = MergeTree
PARTITION BY a ORDER BY a;
ALTER TABLE attach_partition_t7
ADD COLUMN mat_column
UInt32 MATERIALIZED a+b;
insert into attach_partition_t7 values (1, 2);
CREATE TABLE attach_partition_t8 (
a UInt32,
b UInt32
)
ENGINE = MergeTree
PARTITION BY a ORDER BY a;
ALTER TABLE attach_partition_t8 ATTACH PARTITION ID '1' FROM attach_partition_t7; -- {serverError INCOMPATIBLE_COLUMNS};