mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Fixing Not-ready set in constraint.
This commit is contained in:
parent
5943d60f26
commit
0174a43d17
@ -10,6 +10,7 @@
|
|||||||
#include <Common/FieldVisitorToString.h>
|
#include <Common/FieldVisitorToString.h>
|
||||||
#include <Common/assert_cast.h>
|
#include <Common/assert_cast.h>
|
||||||
#include <Common/quoteString.h>
|
#include <Common/quoteString.h>
|
||||||
|
#include <Storages/VirtualColumnUtils.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -31,6 +32,7 @@ CheckConstraintsTransform::CheckConstraintsTransform(
|
|||||||
, table_id(table_id_)
|
, table_id(table_id_)
|
||||||
, constraints_to_check(constraints_.filterConstraints(ConstraintsDescription::ConstraintType::CHECK))
|
, constraints_to_check(constraints_.filterConstraints(ConstraintsDescription::ConstraintType::CHECK))
|
||||||
, expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList()))
|
, expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList()))
|
||||||
|
, context(std::move(context_))
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -39,6 +41,10 @@ void CheckConstraintsTransform::onConsume(Chunk chunk)
|
|||||||
{
|
{
|
||||||
if (chunk.getNumRows() > 0)
|
if (chunk.getNumRows() > 0)
|
||||||
{
|
{
|
||||||
|
if (rows_written == 0)
|
||||||
|
for (const auto & expression : expressions)
|
||||||
|
VirtualColumnUtils::buildSetsForDAG(expression->getActionsDAG(), context);
|
||||||
|
|
||||||
Block block_to_calculate = getInputPort().getHeader().cloneWithColumns(chunk.getColumns());
|
Block block_to_calculate = getInputPort().getHeader().cloneWithColumns(chunk.getColumns());
|
||||||
for (size_t i = 0; i < expressions.size(); ++i)
|
for (size_t i = 0; i < expressions.size(); ++i)
|
||||||
{
|
{
|
||||||
|
@ -35,6 +35,7 @@ private:
|
|||||||
StorageID table_id;
|
StorageID table_id;
|
||||||
const ASTs constraints_to_check;
|
const ASTs constraints_to_check;
|
||||||
const ConstraintsExpressions expressions;
|
const ConstraintsExpressions expressions;
|
||||||
|
ContextPtr context;
|
||||||
size_t rows_written = 0;
|
size_t rows_written = 0;
|
||||||
Chunk cur_chunk;
|
Chunk cur_chunk;
|
||||||
};
|
};
|
||||||
|
@ -328,7 +328,7 @@ void ReadFromSystemDetachedParts::applyFilters(ActionDAGNodes added_filter_nodes
|
|||||||
|
|
||||||
filter = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &block);
|
filter = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &block);
|
||||||
if (filter)
|
if (filter)
|
||||||
VirtualColumnUtils::buildSetsForDAG(filter, context);
|
VirtualColumnUtils::buildSetsForDAG(*filter, context);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -274,7 +274,7 @@ void ReadFromSystemPartsBase::applyFilters(ActionDAGNodes added_filter_nodes)
|
|||||||
|
|
||||||
filter_by_database = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &block);
|
filter_by_database = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &block);
|
||||||
if (filter_by_database)
|
if (filter_by_database)
|
||||||
VirtualColumnUtils::buildSetsForDAG(filter_by_database, context);
|
VirtualColumnUtils::buildSetsForDAG(*filter_by_database, context);
|
||||||
|
|
||||||
block.insert(ColumnWithTypeAndName({}, std::make_shared<DataTypeString>(), table_column_name));
|
block.insert(ColumnWithTypeAndName({}, std::make_shared<DataTypeString>(), table_column_name));
|
||||||
block.insert(ColumnWithTypeAndName({}, std::make_shared<DataTypeString>(), engine_column_name));
|
block.insert(ColumnWithTypeAndName({}, std::make_shared<DataTypeString>(), engine_column_name));
|
||||||
@ -283,7 +283,7 @@ void ReadFromSystemPartsBase::applyFilters(ActionDAGNodes added_filter_nodes)
|
|||||||
|
|
||||||
filter_by_other_columns = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &block);
|
filter_by_other_columns = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &block);
|
||||||
if (filter_by_other_columns)
|
if (filter_by_other_columns)
|
||||||
VirtualColumnUtils::buildSetsForDAG(filter_by_other_columns, context);
|
VirtualColumnUtils::buildSetsForDAG(*filter_by_other_columns, context);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -54,9 +54,9 @@ namespace DB
|
|||||||
namespace VirtualColumnUtils
|
namespace VirtualColumnUtils
|
||||||
{
|
{
|
||||||
|
|
||||||
void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context)
|
void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context)
|
||||||
{
|
{
|
||||||
for (const auto & node : dag->getNodes())
|
for (const auto & node : dag.getNodes())
|
||||||
{
|
{
|
||||||
if (node.type == ActionsDAG::ActionType::COLUMN)
|
if (node.type == ActionsDAG::ActionType::COLUMN)
|
||||||
{
|
{
|
||||||
@ -79,7 +79,7 @@ void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context)
|
|||||||
|
|
||||||
void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context)
|
void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context)
|
||||||
{
|
{
|
||||||
buildSetsForDAG(dag, context);
|
buildSetsForDAG(*dag, context);
|
||||||
auto actions = std::make_shared<ExpressionActions>(dag);
|
auto actions = std::make_shared<ExpressionActions>(dag);
|
||||||
Block block_with_filter = block;
|
Block block_with_filter = block;
|
||||||
actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true);
|
actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true);
|
||||||
|
@ -26,7 +26,7 @@ void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block,
|
|||||||
void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context);
|
void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context);
|
||||||
|
|
||||||
/// Builds sets used by ActionsDAG inplace.
|
/// Builds sets used by ActionsDAG inplace.
|
||||||
void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context);
|
void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context);
|
||||||
|
|
||||||
/// Recursively checks if all functions used in DAG are deterministic in scope of query.
|
/// Recursively checks if all functions used in DAG are deterministic in scope of query.
|
||||||
bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node);
|
bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node);
|
||||||
|
@ -0,0 +1 @@
|
|||||||
|
42
|
@ -0,0 +1,19 @@
|
|||||||
|
DROP TABLE IF EXISTS t1;
|
||||||
|
DROP TABLE IF EXISTS t2;
|
||||||
|
|
||||||
|
CREATE TABLE t1 (
|
||||||
|
`id` UInt64
|
||||||
|
)
|
||||||
|
ENGINE = MergeTree ORDER BY id;
|
||||||
|
|
||||||
|
INSERT INTO t1(id) VALUES (42);
|
||||||
|
|
||||||
|
CREATE TABLE t2 (
|
||||||
|
`conversation` UInt64,
|
||||||
|
CONSTRAINT constraint_conversation CHECK conversation IN (SELECT id FROM t1)
|
||||||
|
)
|
||||||
|
ENGINE = MergeTree ORDER BY conversation;
|
||||||
|
|
||||||
|
INSERT INTO t2(conversation) VALUES (42);
|
||||||
|
|
||||||
|
select * from t2;
|
Loading…
Reference in New Issue
Block a user