From d4c1df7c03b48d13c17151e8b09250ee98daf15e Mon Sep 17 00:00:00 2001 From: Weihao Li <18110526956@163.com> Date: Thu, 10 Apr 2025 08:59:44 +0800 Subject: [PATCH 01/11] implement Signed-off-by: Weihao Li <18110526956@163.com> --- .../plan/planner/TableOperatorGenerator.java | 78 ++++++- .../planner/plan/node/PlanGraphPrinter.java | 7 +- .../analyzer/StatementAnalyzer.java | 105 ++++++--- .../relational/planner/RelationPlanner.java | 137 ++++++++---- .../iterative/rule/PruneJoinColumns.java | 1 + ...atedDistinctAggregationWithProjection.java | 1 + ...dDistinctAggregationWithoutProjection.java | 1 + ...elatedGlobalAggregationWithProjection.java | 1 + ...tedGlobalAggregationWithoutProjection.java | 1 + ...latedGroupedAggregationWithProjection.java | 1 + ...edGroupedAggregationWithoutProjection.java | 1 + .../rule/TransformCorrelatedJoinToJoin.java | 1 + ...TransformFilteringSemiJoinToInnerJoin.java | 1 + .../TransformUncorrelatedSubqueryToJoin.java | 1 + .../relational/planner/node/JoinNode.java | 99 ++++++++- .../PushPredicateIntoTableScan.java | 15 ++ .../UnaliasSymbolReferences.java | 1 + .../plan/relational/sql/ast/AsofJoinOn.java | 167 ++++++++++++++ .../sql/ast/DefaultTraversalVisitor.java | 8 +- .../plan/relational/sql/ast/JoinOn.java | 11 +- .../relational/sql/parser/AstBuilder.java | 42 +++- .../relational/analyzer/AsofJoinTest.java | 210 ++++++++++++++++++ .../assertions/AsofJoinClauseProvider.java | 55 +++++ .../planner/assertions/JoinMatcher.java | 32 ++- .../planner/assertions/PlanMatchPattern.java | 6 + .../relational/grammar/sql/RelationalSql.g4 | 7 + 26 files changed, 911 insertions(+), 79 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/AsofJoinOn.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofJoinTest.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/AsofJoinClauseProvider.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java index 40a204aa9f30b..91bf1aa25e076 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java @@ -1442,6 +1442,82 @@ public Operator visitJoin(JoinNode node, LocalExecutionPlanContext context) { joinKeyTypes.add(leftJoinKeyType); } + JoinNode.AsofJoinClause asofJoinClause = node.getAsofCriteria().orElse(null); + if (asofJoinClause != null) { + throw new UnsupportedOperationException("BE of ASOF Join is not supported!"); + /*BiPredicate asofPredicate; + switch (asofJoinClause.getOperator()) { + case LESS_THAN: + asofPredicate = (left, right) -> left < right; + break; + case LESS_THAN_OR_EQUAL: + asofPredicate = (left, right) -> left <= right; + break; + case GREATER_THAN: + asofPredicate = (left, right) -> left > right; + break; + case GREATER_THAN_OR_EQUAL: + asofPredicate = (left, right) -> left >= right; + break; + default: + throw new IllegalStateException(String.format("Unexpected ASOF Join criteria Operator: %s",asofJoinClause.getOperator())); + } + + Integer leftAsofJoinKeyIndex = leftColumnNamesMap.get(asofJoinClause.getLeft()); + if (leftAsofJoinKeyIndex == null) { + throw new IllegalStateException("Left child of JoinNode doesn't contain left ASOF main join key."); + } + Integer rightAsofJoinKeyIndex = rightColumnNamesMap.get(asofJoinClause.getRight()); + if (rightAsofJoinKeyIndex == null) { + throw new IllegalStateException("Right child of JoinNode doesn't contain right ASOF main join key."); + } + if (requireNonNull(node.getJoinType()) == JoinNode.JoinType.INNER) { + OperatorContext operatorContext = + context + .getDriverContext() + .addOperatorContext( + context.getNextOperatorId(), + node.getPlanNodeId(), + AsofMergeSortInnerJoinOperator.class.getSimpleName()); + return new AsofMergeSortInnerJoinOperator( + operatorContext, + asofPredicate, + leftChild, + leftJoinKeyPositions, + leftAsofJoinKeyIndex, + leftOutputSymbolIdx, + rightChild, + rightJoinKeyPositions, + rightAsofJoinKeyIndex, + rightOutputSymbolIdx, + JoinKeyComparatorFactory.getComparators(joinKeyTypes, true), + dataTypes); + } else if(requireNonNull(node.getJoinType()) == JoinNode.JoinType.LEFT) { + OperatorContext operatorContext = + context + .getDriverContext() + .addOperatorContext( + context.getNextOperatorId(), + node.getPlanNodeId(), + AsofMergeSortLeftJoinOperator.class.getSimpleName()); + return new AsofMergeSortInnerJoinOperator( + operatorContext, + asofPredicate, + leftChild, + leftJoinKeyPositions, + leftAsofJoinKeyIndex, + leftOutputSymbolIdx, + rightChild, + rightJoinKeyPositions, + rightAsofJoinKeyIndex, + rightOutputSymbolIdx, + JoinKeyComparatorFactory.getComparators(joinKeyTypes, true), + dataTypes); + } else { + throw new IllegalStateException("Unsupported ASOF join type: " + node.getJoinType()); + }*/ + } + if (requireNonNull(node.getJoinType()) == JoinNode.JoinType.INNER) { OperatorContext operatorContext = context @@ -1510,7 +1586,7 @@ private void semanticCheckForJoin(JoinNode node) { "Filter is not supported in %s. Filter is %s.", node.getJoinType(), node.getFilter().map(Expression::toString).orElse("null"))); checkArgument( - !node.getCriteria().isEmpty(), + !node.getCriteria().isEmpty() || node.getAsofCriteria().isPresent(), String.format("%s must have join keys.", node.getJoinType())); } catch (IllegalArgumentException e) { throw new SemanticException(e.getMessage()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java index c49e3de9004dc..e372296f1d0b7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java @@ -974,7 +974,12 @@ public List visitJoin( GraphContext context) { List boxValue = new ArrayList<>(); boxValue.add(String.format("Join-%s", node.getPlanNodeId().getId())); - boxValue.add(String.format("JoinType: %s", node.getJoinType())); + if (node.getAsofCriteria().isPresent()) { + boxValue.add(String.format("ASOF-JoinType: %s", node.getJoinType())); + boxValue.add(String.format("ASOF-JoinCriteria: %s", node.getAsofCriteria().get())); + } else { + boxValue.add(String.format("JoinType: %s", node.getJoinType())); + } boxValue.add(String.format("JoinCriteria: %s", node.getCriteria())); boxValue.add(String.format("LeftOutputSymbols: %s", node.getLeftOutputSymbols())); boxValue.add(String.format("RightOutputSymbols: %s", node.getRightOutputSymbols())); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java index f28bae6a210e0..de184008e995a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java @@ -59,6 +59,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AlterPipe; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ArithmeticBinaryExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ArithmeticUnaryExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AsofJoinOn; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AstVisitor; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BetweenPredicate; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Cast; @@ -199,6 +200,7 @@ import com.google.common.collect.Streams; import org.apache.tsfile.common.conf.TSFileConfig; import org.apache.tsfile.read.common.type.RowType; +import org.apache.tsfile.read.common.type.TimestampType; import org.apache.tsfile.read.common.type.Type; import org.apache.tsfile.utils.Binary; @@ -2971,11 +2973,19 @@ protected Scope visitJoin(Join node, Optional scope) { return analyzeJoinUsing(node, ((JoinUsing) criteria).getColumns(), scope, left, right); } + if (criteria instanceof AsofJoinOn) { + if (node.getType() != LEFT && node.getType() != INNER) { + throw new SemanticException( + String.format( + "ASOF [%s] JOIN is not supported, only support INNER and LEFT.", node.getType())); + } + } + Scope output = createAndAssignScope( node, scope, left.getRelationType().joinWith(right.getRelationType())); - if (node.getType() == LEFT || node.getType() == RIGHT) { + if (node.getType() == RIGHT) { throw new SemanticException( String.format( "%s JOIN is not supported, only support INNER JOIN in current version.", @@ -2984,36 +2994,77 @@ protected Scope visitJoin(Join node, Optional scope) { return output; } if (criteria instanceof JoinOn) { + boolean isAsofJoin = criteria instanceof AsofJoinOn; + Expression expression = ((JoinOn) criteria).getExpression(); - verifyNoAggregateWindowOrGroupingFunctions(expression, "JOIN clause"); + if (expression != null) { + verifyNoAggregateWindowOrGroupingFunctions(expression, "JOIN clause"); + + // Need to register coercions in case when join criteria requires coercion (e.g. join on + // char(1) = char(2)) + // Correlations are only currently support in the join criteria for INNER joins + ExpressionAnalysis expressionAnalysis = + analyzeExpression( + expression, + output, + node.getType() == INNER && !isAsofJoin + ? CorrelationSupport.ALLOWED + : CorrelationSupport.DISALLOWED); + Type clauseType = expressionAnalysis.getType(expression); + if (!clauseType.equals(BOOLEAN)) { + // if (!clauseType.equals(UNKNOWN)) { + // throw semanticException( + // TYPE_MISMATCH, + // expression, + // "JOIN ON clause must evaluate to a boolean: actual type %s", + // clauseType); + // } + throw new SemanticException( + String.format( + "JOIN ON clause must evaluate to a boolean: actual type %s", clauseType)); + // coerce expression to boolean + // analysis.addCoercion(expression, BOOLEAN, false); + } - // Need to register coercions in case when join criteria requires coercion (e.g. join on - // char(1) = char(2)) - // Correlations are only currently support in the join criteria for INNER joins - ExpressionAnalysis expressionAnalysis = - analyzeExpression( - expression, - output, - node.getType() == INNER - ? CorrelationSupport.ALLOWED - : CorrelationSupport.DISALLOWED); - Type clauseType = expressionAnalysis.getType(expression); - if (!clauseType.equals(BOOLEAN)) { - // if (!clauseType.equals(UNKNOWN)) { - // throw semanticException( - // TYPE_MISMATCH, - // expression, - // "JOIN ON clause must evaluate to a boolean: actual type %s", - // clauseType); - // } - throw new SemanticException( - String.format( - "JOIN ON clause must evaluate to a boolean: actual type %s", clauseType)); - // coerce expression to boolean - // analysis.addCoercion(expression, BOOLEAN, false); + if (!isAsofJoin) { + analysis.recordSubqueries(node, expressionAnalysis); + } } - analysis.recordSubqueries(node, expressionAnalysis); + if (isAsofJoin) { + // The asofExpression must be ComparisonExpression, it has been checked in AstBuilder + ComparisonExpression asofExpression = + (ComparisonExpression) ((AsofJoinOn) criteria).getAsofExpression(); + + verifyNoAggregateWindowOrGroupingFunctions(asofExpression, "JOIN clause"); + + // ASOF Join does not support Correlation + ExpressionAnalysis expressionAnalysis = + analyzeExpression(asofExpression, output, CorrelationSupport.DISALLOWED); + Type clauseType = expressionAnalysis.getType(asofExpression); + if (!clauseType.equals(BOOLEAN)) { + throw new SemanticException( + String.format( + "ASOF main JOIN expression must evaluate to a boolean: actual type %s", + clauseType)); + } + + clauseType = expressionAnalysis.getType(asofExpression.getLeft()); + if (!clauseType.equals(TimestampType.TIMESTAMP)) { + throw new SemanticException( + String.format( + "left child type of ASOF main JOIN expression must be TIMESTAMP: actual type %s", + clauseType)); + } + + clauseType = expressionAnalysis.getType(asofExpression.getRight()); + if (!clauseType.equals(TimestampType.TIMESTAMP)) { + throw new SemanticException( + String.format( + "right child type of ASOF main JOIN expression must be TIMESTAMP: actual type %s", + clauseType)); + } + } analysis.setJoinCriteria(node, expression); } else { throw new UnsupportedOperationException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java index 8d1b1576ea60a..a1259fa586d17 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.planner; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; +import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.common.SessionInfo; @@ -56,6 +57,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AliasedRelation; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AsofJoinOn; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AstVisitor; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.CoalesceExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; @@ -68,6 +70,8 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.InsertTablet; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Intersect; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Join; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.JoinCriteria; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.JoinOn; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.JoinUsing; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LoadTsFile; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LogicalExpression; @@ -104,6 +108,8 @@ import java.util.stream.IntStream; import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.gson.internal.$Gson$Preconditions.checkArgument; +import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static org.apache.iotdb.commons.schema.table.InformationSchema.INFORMATION_DATABASE; import static org.apache.iotdb.db.queryengine.plan.relational.planner.PlanBuilder.newPlanBuilder; @@ -277,8 +283,17 @@ protected RelationPlan visitJoin(final Join node, final Void context) { return planJoinUsing(node, leftPlan, rightPlan); } + Expression asofCriteria = null; + if (node.getCriteria().isPresent()) { + JoinCriteria criteria = node.getCriteria().get(); + checkArgument(criteria instanceof JoinOn); + if (criteria instanceof AsofJoinOn) { + asofCriteria = ((AsofJoinOn) criteria).getAsofExpression(); + } + } return planJoin( analysis.getJoinCriteria(node), + asofCriteria, node.getType(), analysis.getScope(node), leftPlan, @@ -364,6 +379,7 @@ If casts are redundant (due to column type and common type being equal), leftCoercion, rightCoercion, clauses.build(), + Optional.empty(), leftCoercion.getOutputSymbols(), rightCoercion.getOutputSymbols(), Optional.empty(), @@ -409,6 +425,7 @@ If casts are redundant (due to column type and common type being equal), public RelationPlan planJoin( Expression criteria, + Expression asofCriteria, Join.Type type, Scope scope, RelationPlan leftPlan, @@ -427,6 +444,7 @@ public RelationPlan planJoin( newPlanBuilder(rightPlan, analysis).withScope(scope, outputSymbols); ImmutableList.Builder equiClauses = ImmutableList.builder(); + Optional asofJoinClause = Optional.empty(); List complexJoinExpressions = new ArrayList<>(); List postInnerJoinConditions = new ArrayList<>(); @@ -438,49 +456,79 @@ public RelationPlan planJoin( List rightComparisonExpressions = new ArrayList<>(); List joinConditionComparisonOperators = new ArrayList<>(); - for (Expression conjunct : extractPredicates(LogicalExpression.Operator.AND, criteria)) { - if (!isEqualComparisonExpression(conjunct) && type != INNER) { - complexJoinExpressions.add(conjunct); - continue; + if (asofCriteria != null) { + Expression firstExpression = ((ComparisonExpression) asofCriteria).getLeft(); + Expression secondExpression = ((ComparisonExpression) asofCriteria).getRight(); + ComparisonExpression.Operator comparisonOperator = + ((ComparisonExpression) asofCriteria).getOperator(); + Set firstDependencies = + SymbolsExtractor.extractNames(firstExpression, analysis.getColumnReferences()); + Set secondDependencies = + SymbolsExtractor.extractNames(secondExpression, analysis.getColumnReferences()); + + if (firstDependencies.stream().allMatch(left::canResolve) + && secondDependencies.stream().allMatch(right::canResolve)) { + leftComparisonExpressions.add(firstExpression); + rightComparisonExpressions.add(secondExpression); + joinConditionComparisonOperators.add(comparisonOperator); + } else if (firstDependencies.stream().allMatch(right::canResolve) + && secondDependencies.stream().allMatch(left::canResolve)) { + leftComparisonExpressions.add(secondExpression); + rightComparisonExpressions.add(firstExpression); + joinConditionComparisonOperators.add(comparisonOperator.flip()); + } else { + // the case when we mix symbols from both left and right join side on either side of + // condition. + throw new SemanticException( + format("Complex ASOF main join expression [%s] is not supported", asofCriteria)); } + } - Set dependencies = - SymbolsExtractor.extractNames(conjunct, analysis.getColumnReferences()); - - if (dependencies.stream().allMatch(left::canResolve) - || dependencies.stream().allMatch(right::canResolve)) { - // If the conjunct can be evaluated entirely with the inputs on either side of the join, - // add - // it to the list complex expressions and let the optimizers figure out how to push it - // down later. - complexJoinExpressions.add(conjunct); - } else if (conjunct instanceof ComparisonExpression) { - Expression firstExpression = ((ComparisonExpression) conjunct).getLeft(); - Expression secondExpression = ((ComparisonExpression) conjunct).getRight(); - ComparisonExpression.Operator comparisonOperator = - ((ComparisonExpression) conjunct).getOperator(); - Set firstDependencies = - SymbolsExtractor.extractNames(firstExpression, analysis.getColumnReferences()); - Set secondDependencies = - SymbolsExtractor.extractNames(secondExpression, analysis.getColumnReferences()); - - if (firstDependencies.stream().allMatch(left::canResolve) - && secondDependencies.stream().allMatch(right::canResolve)) { - leftComparisonExpressions.add(firstExpression); - rightComparisonExpressions.add(secondExpression); - joinConditionComparisonOperators.add(comparisonOperator); - } else if (firstDependencies.stream().allMatch(right::canResolve) - && secondDependencies.stream().allMatch(left::canResolve)) { - leftComparisonExpressions.add(secondExpression); - rightComparisonExpressions.add(firstExpression); - joinConditionComparisonOperators.add(comparisonOperator.flip()); + if (criteria != null) { + for (Expression conjunct : extractPredicates(LogicalExpression.Operator.AND, criteria)) { + if (!isEqualComparisonExpression(conjunct) && type != INNER) { + complexJoinExpressions.add(conjunct); + continue; + } + + Set dependencies = + SymbolsExtractor.extractNames(conjunct, analysis.getColumnReferences()); + + if (dependencies.stream().allMatch(left::canResolve) + || dependencies.stream().allMatch(right::canResolve)) { + // If the conjunct can be evaluated entirely with the inputs on either side of the join, + // add + // it to the list complex expressions and let the optimizers figure out how to push it + // down later. + complexJoinExpressions.add(conjunct); + } else if (conjunct instanceof ComparisonExpression) { + Expression firstExpression = ((ComparisonExpression) conjunct).getLeft(); + Expression secondExpression = ((ComparisonExpression) conjunct).getRight(); + ComparisonExpression.Operator comparisonOperator = + ((ComparisonExpression) conjunct).getOperator(); + Set firstDependencies = + SymbolsExtractor.extractNames(firstExpression, analysis.getColumnReferences()); + Set secondDependencies = + SymbolsExtractor.extractNames(secondExpression, analysis.getColumnReferences()); + + if (firstDependencies.stream().allMatch(left::canResolve) + && secondDependencies.stream().allMatch(right::canResolve)) { + leftComparisonExpressions.add(firstExpression); + rightComparisonExpressions.add(secondExpression); + joinConditionComparisonOperators.add(comparisonOperator); + } else if (firstDependencies.stream().allMatch(right::canResolve) + && secondDependencies.stream().allMatch(left::canResolve)) { + leftComparisonExpressions.add(secondExpression); + rightComparisonExpressions.add(firstExpression); + joinConditionComparisonOperators.add(comparisonOperator.flip()); + } else { + // the case when we mix symbols from both left and right join side on either side of + // condition. + complexJoinExpressions.add(conjunct); + } } else { - // the case when we mix symbols from both left and right join side on either side of - // condition. complexJoinExpressions.add(conjunct); } - } else { - complexJoinExpressions.add(conjunct); } } @@ -507,6 +555,17 @@ public RelationPlan planJoin( rightPlanBuilder = rightCoercions.getSubPlan(); for (int i = 0; i < leftComparisonExpressions.size(); i++) { + if (asofCriteria != null && i == 0) { + Symbol leftSymbol = leftCoercions.get(leftComparisonExpressions.get(i)); + Symbol rightSymbol = rightCoercions.get(rightComparisonExpressions.get(i)); + + asofJoinClause = + Optional.of( + new JoinNode.AsofJoinClause( + joinConditionComparisonOperators.get(i), leftSymbol, rightSymbol)); + continue; + } + if (joinConditionComparisonOperators.get(i) == ComparisonExpression.Operator.EQUAL) { Symbol leftSymbol = leftCoercions.get(leftComparisonExpressions.get(i)); Symbol rightSymbol = rightCoercions.get(rightComparisonExpressions.get(i)); @@ -529,6 +588,7 @@ public RelationPlan planJoin( leftPlanBuilder.getRoot(), rightPlanBuilder.getRoot(), equiClauses.build(), + asofJoinClause, leftPlanBuilder.getRoot().getOutputSymbols(), rightPlanBuilder.getRoot().getOutputSymbols(), Optional.empty(), @@ -576,6 +636,7 @@ public RelationPlan planJoin( leftPlanBuilder.getRoot(), rightPlanBuilder.getRoot(), equiClauses.build(), + asofJoinClause, leftPlanBuilder.getRoot().getOutputSymbols(), rightPlanBuilder.getRoot().getOutputSymbols(), Optional.of( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneJoinColumns.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneJoinColumns.java index a93db7f0b489d..c09710346ed09 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneJoinColumns.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneJoinColumns.java @@ -44,6 +44,7 @@ protected Optional pushDownProjectOff( joinNode.getLeftChild(), joinNode.getRightChild(), joinNode.getCriteria(), + joinNode.getAsofCriteria(), filteredCopy(joinNode.getLeftOutputSymbols(), referencedOutputs::contains), filteredCopy(joinNode.getRightOutputSymbols(), referencedOutputs::contains), joinNode.getFilter(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithProjection.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithProjection.java index 9e9423fea4e36..a79f86dec91e8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithProjection.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithProjection.java @@ -145,6 +145,7 @@ public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Co inputWithUniqueId, source, ImmutableList.of(), + Optional.empty(), inputWithUniqueId.getOutputSymbols(), source.getOutputSymbols(), decorrelatedSource.get().getCorrelatedPredicates(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithoutProjection.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithoutProjection.java index 9625736a8f3a3..6af314ec33946 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithoutProjection.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithoutProjection.java @@ -134,6 +134,7 @@ public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Co inputWithUniqueId, source, ImmutableList.of(), + Optional.empty(), inputWithUniqueId.getOutputSymbols(), source.getOutputSymbols(), decorrelatedSource.get().getCorrelatedPredicates(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithProjection.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithProjection.java index 021de9fe9bdf4..ffae9eb5c8e02 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithProjection.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithProjection.java @@ -225,6 +225,7 @@ public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Co inputWithUniqueId, source, ImmutableList.of(), + Optional.empty(), inputWithUniqueId.getOutputSymbols(), source.getOutputSymbols(), decorrelatedSource.get().getCorrelatedPredicates(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithoutProjection.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithoutProjection.java index 12c65363ba813..5e85f39454b6b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithoutProjection.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithoutProjection.java @@ -216,6 +216,7 @@ public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Co inputWithUniqueId, source, ImmutableList.of(), + Optional.empty(), inputWithUniqueId.getOutputSymbols(), source.getOutputSymbols(), decorrelatedSource.get().getCorrelatedPredicates(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithProjection.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithProjection.java index 70a1a4d7bc23c..cae6088c8784f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithProjection.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithProjection.java @@ -200,6 +200,7 @@ public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Co inputWithUniqueId, source, ImmutableList.of(), + Optional.empty(), inputWithUniqueId.getOutputSymbols(), source.getOutputSymbols(), decorrelatedSource.get().getCorrelatedPredicates(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithoutProjection.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithoutProjection.java index 2674632a61928..208e3ed5e0019 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithoutProjection.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithoutProjection.java @@ -187,6 +187,7 @@ public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Co inputWithUniqueId, source, ImmutableList.of(), + Optional.empty(), inputWithUniqueId.getOutputSymbols(), source.getOutputSymbols(), decorrelatedSource.get().getCorrelatedPredicates(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedJoinToJoin.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedJoinToJoin.java index cfd22fe4f120f..f24de2d1b0d76 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedJoinToJoin.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformCorrelatedJoinToJoin.java @@ -91,6 +91,7 @@ public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Co correlatedJoinNode.getInput(), decorrelatedSubquery.getNode(), ImmutableList.of(), + Optional.empty(), correlatedJoinNode.getInput().getOutputSymbols(), correlatedJoinNode.getSubquery().getOutputSymbols(), filter.equals(TRUE_LITERAL) ? Optional.empty() : Optional.of(filter), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformFilteringSemiJoinToInnerJoin.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformFilteringSemiJoinToInnerJoin.java index 67f814af66168..1f32fdbffe157 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformFilteringSemiJoinToInnerJoin.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformFilteringSemiJoinToInnerJoin.java @@ -131,6 +131,7 @@ public Result apply(FilterNode filterNode, Captures captures, Context context) { ImmutableList.of( new JoinNode.EquiJoinClause( semiJoin.getSourceJoinSymbol(), semiJoin.getFilteringSourceJoinSymbol())), + Optional.empty(), semiJoin.getSource().getOutputSymbols(), ImmutableList.of(), joinFilter, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformUncorrelatedSubqueryToJoin.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformUncorrelatedSubqueryToJoin.java index 1c9765ff3c8a6..6d96843e89302 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformUncorrelatedSubqueryToJoin.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/TransformUncorrelatedSubqueryToJoin.java @@ -131,6 +131,7 @@ && extractCardinality(parent.getSubquery(), lookup).isAtLeastScalar() parent.getInput(), parent.getSubquery(), ImmutableList.of(), + Optional.empty(), parent.getInput().getOutputSymbols(), parent.getSubquery().getOutputSymbols(), filter.equals(TRUE_LITERAL) ? Optional.empty() : Optional.of(filter), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/JoinNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/JoinNode.java index 0326eb71a60f5..ca4807c064804 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/JoinNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/JoinNode.java @@ -51,6 +51,7 @@ public class JoinNode extends TwoChildProcessNode { private final JoinType joinType; private final List criteria; + private final Optional asofCriteria; private final List leftOutputSymbols; private final List rightOutputSymbols; // some filter like 'a.xx_column < b.yy_column' @@ -69,6 +70,7 @@ public JoinNode( PlanNode leftChild, PlanNode rightChild, List criteria, + Optional asofCriteria, List leftOutputSymbols, List rightOutputSymbols, Optional filter, @@ -96,6 +98,7 @@ public JoinNode( this.leftChild = leftChild; this.rightChild = rightChild; this.criteria = ImmutableList.copyOf(criteria); + this.asofCriteria = asofCriteria; this.leftOutputSymbols = ImmutableList.copyOf(leftOutputSymbols); this.rightOutputSymbols = ImmutableList.copyOf(rightOutputSymbols); this.filter = filter; @@ -135,6 +138,7 @@ public JoinNode( PlanNodeId id, JoinType joinType, List criteria, + Optional asofCriteria, List leftOutputSymbols, List rightOutputSymbols) { super(id); @@ -148,6 +152,7 @@ public JoinNode( this.joinType = joinType; this.criteria = criteria; + this.asofCriteria = asofCriteria; } @Override @@ -164,6 +169,7 @@ public PlanNode replaceChildren(List newChildren) { newChildren.get(0), newChildren.get(1), criteria, + asofCriteria, leftOutputSymbols, rightOutputSymbols, filter, @@ -187,6 +193,7 @@ public PlanNode clone() { getLeftChild(), getRightChild(), criteria, + asofCriteria, leftOutputSymbols, rightOutputSymbols, filter, @@ -235,6 +242,16 @@ protected void serializeAttributes(DataOutputStream stream) throws IOException { Symbol.serialize(equiJoinClause.getRight(), stream); } + if (asofCriteria.isPresent()) { + ReadWriteIOUtils.write(true, stream); + AsofJoinClause asofJoinClause = asofCriteria.get(); + ReadWriteIOUtils.write(asofJoinClause.getOperator().ordinal(), stream); + Symbol.serialize(asofJoinClause.getLeft(), stream); + Symbol.serialize(asofJoinClause.getRight(), stream); + } else { + ReadWriteIOUtils.write(false, stream); + } + ReadWriteIOUtils.write(leftOutputSymbols.size(), stream); for (Symbol leftOutputSymbol : leftOutputSymbols) { Symbol.serialize(leftOutputSymbol, stream); @@ -254,6 +271,16 @@ public static JoinNode deserialize(ByteBuffer byteBuffer) { new EquiJoinClause(Symbol.deserialize(byteBuffer), Symbol.deserialize(byteBuffer))); } + Optional asofJoinClause = Optional.empty(); + if (ReadWriteIOUtils.readBool(byteBuffer)) { + asofJoinClause = + Optional.of( + new AsofJoinClause( + ComparisonExpression.Operator.values()[ReadWriteIOUtils.readInt(byteBuffer)], + Symbol.deserialize(byteBuffer), + Symbol.deserialize(byteBuffer))); + } + size = ReadWriteIOUtils.readInt(byteBuffer); List leftOutputSymbols = new ArrayList<>(size); while (size-- > 0) { @@ -267,7 +294,8 @@ public static JoinNode deserialize(ByteBuffer byteBuffer) { } PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer); - return new JoinNode(planNodeId, joinType, criteria, leftOutputSymbols, rightOutputSymbols); + return new JoinNode( + planNodeId, joinType, criteria, asofJoinClause, leftOutputSymbols, rightOutputSymbols); } public JoinType getJoinType() { @@ -278,6 +306,10 @@ public List getCriteria() { return criteria; } + public Optional getAsofCriteria() { + return asofCriteria; + } + public List getLeftOutputSymbols() { return leftOutputSymbols; } @@ -295,7 +327,10 @@ public Optional isSpillable() { } public boolean isCrossJoin() { - return criteria.isEmpty() && !filter.isPresent() && joinType == INNER; + return !asofCriteria.isPresent() + && criteria.isEmpty() + && !filter.isPresent() + && joinType == INNER; } @Override @@ -355,6 +390,66 @@ public String toString() { } } + public static class AsofJoinClause { + private final Symbol left; + private final Symbol right; + private final ComparisonExpression.Operator operator; + + public AsofJoinClause(ComparisonExpression.Operator operator, Symbol left, Symbol right) { + this.operator = operator; + this.left = requireNonNull(left, "left is null"); + this.right = requireNonNull(right, "right is null"); + } + + public Symbol getLeft() { + return left; + } + + public Symbol getRight() { + return right; + } + + public ComparisonExpression.Operator getOperator() { + return operator; + } + + public ComparisonExpression toExpression() { + return new ComparisonExpression( + operator, left.toSymbolReference(), right.toSymbolReference()); + } + + public AsofJoinClause flip() { + return new AsofJoinClause(operator.flip(), right, left); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || !this.getClass().equals(obj.getClass())) { + return false; + } + + AsofJoinClause other = (AsofJoinClause) obj; + + return Objects.equals(this.operator, other.operator) + && Objects.equals(this.left, other.left) + && Objects.equals(this.right, other.right); + } + + @Override + public int hashCode() { + return Objects.hash(operator, left, right); + } + + @Override + public String toString() { + return format("%s %s %s", left, operator.getValue(), right); + } + } + public enum JoinType { INNER("InnerJoin"), LEFT("LeftJoin"), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java index 4bfcd500098a3..5d238341de570 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java @@ -811,6 +811,7 @@ public PlanNode visitJoin(JoinNode node, RewriteContext context) { leftSource, rightSource, equiJoinClauses, + node.getAsofCriteria(), leftSource.getOutputSymbols(), rightSource.getOutputSymbols(), newJoinFilter, @@ -874,6 +875,10 @@ private Symbol symbolForExpression(Expression expression) { private void appendSortNodeForMergeSortJoin(JoinNode joinNode) { int size = joinNode.getCriteria().size(); + JoinNode.AsofJoinClause asofJoinClause = joinNode.getAsofCriteria().orElse(null); + if (asofJoinClause != null) { + size++; + } List leftOrderBy = new ArrayList<>(size); List rightOrderBy = new ArrayList<>(size); Map leftOrderings = new HashMap<>(size); @@ -884,6 +889,13 @@ private void appendSortNodeForMergeSortJoin(JoinNode joinNode) { rightOrderBy.add(equiJoinClause.getRight()); rightOrderings.put(equiJoinClause.getRight(), ASC_NULLS_LAST); } + if (asofJoinClause != null) { + leftOrderBy.add(asofJoinClause.getLeft()); + leftOrderings.put(asofJoinClause.getLeft(), ASC_NULLS_LAST); + rightOrderBy.add(asofJoinClause.getRight()); + rightOrderings.put(asofJoinClause.getRight(), ASC_NULLS_LAST); + ; + } OrderingScheme leftOrderingScheme = new OrderingScheme(leftOrderBy, leftOrderings); OrderingScheme rightOrderingScheme = new OrderingScheme(rightOrderBy, rightOrderings); SortNode leftSortNode = @@ -1174,6 +1186,7 @@ private JoinNode tryNormalizeToOuterToInnerJoin(JoinNode node, Expression inheri node.getLeftChild(), node.getRightChild(), node.getCriteria(), + node.getAsofCriteria(), node.getLeftOutputSymbols(), node.getRightOutputSymbols(), node.getFilter(), @@ -1186,6 +1199,7 @@ private JoinNode tryNormalizeToOuterToInnerJoin(JoinNode node, Expression inheri node.getLeftChild(), node.getRightChild(), node.getCriteria(), + node.getAsofCriteria(), node.getLeftOutputSymbols(), node.getRightOutputSymbols(), node.getFilter(), @@ -1220,6 +1234,7 @@ private JoinNode tryNormalizeToOuterToInnerJoin(JoinNode node, Expression inheri node.getLeftChild(), node.getRightChild(), node.getCriteria(), + node.getAsofCriteria(), node.getLeftOutputSymbols(), node.getRightOutputSymbols(), node.getFilter(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/UnaliasSymbolReferences.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/UnaliasSymbolReferences.java index abda59b797ad9..9a50138ed972a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/UnaliasSymbolReferences.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/UnaliasSymbolReferences.java @@ -762,6 +762,7 @@ public PlanAndMappings visitJoin(JoinNode node, UnaliasContext context) { rewrittenLeft.getRoot(), rewrittenRight.getRoot(), newCriteria, + node.getAsofCriteria(), newLeftOutputSymbols, newRightOutputSymbols, newFilter, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/AsofJoinOn.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/AsofJoinOn.java new file mode 100644 index 0000000000000..8b7229af32aa3 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/AsofJoinOn.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.sql.ast; + +import org.apache.iotdb.db.exception.sql.SemanticException; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import org.apache.tsfile.utils.TimeDuration; + +import java.util.List; +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.ir.IrUtils.and; +import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression.Operator.LESS_THAN_OR_EQUAL; + +public class AsofJoinOn extends JoinOn { + + // record main expression of ASOF join + // .e.g 'ASOF (tolerance 1) JOIN ON t1.device = t2.device and t1.time > t2.time' => + // asofExpression: + // 't1.time > t2.time', expression in super Class: 't1.device = t2.device and t1.time <= t2.time + + // 1' + private final Expression asofExpression; + + private static final String joinErrMsg = + "The join expression of ASOF should be single Comparison or Logical 'AND'"; + private static final String asofErrMsg = + "The main join expression of ASOF should only be Comparison '>, >=, <, <=', actual is %s"; + + public AsofJoinOn(Expression otherExpression, Expression asofExpression) { + super(otherExpression); + this.asofExpression = asofExpression; + } + + public Expression getAsofExpression() { + return asofExpression; + } + + public static JoinCriteria constructAsofJoinOn( + Expression joinExpression, TimeDuration timeDuration) { + ImmutableList.Builder newTerms = ImmutableList.builder(); + ComparisonExpression asofExpression; + if (joinExpression instanceof ComparisonExpression) { + asofExpression = (ComparisonExpression) joinExpression; + } else if (joinExpression instanceof LogicalExpression) { + LogicalExpression logicalExpression = (LogicalExpression) joinExpression; + if (logicalExpression.getOperator() != LogicalExpression.Operator.AND) { + throw new SemanticException(joinErrMsg); + } + + List terms = logicalExpression.getTerms(); + Expression lastExpression = Iterables.getLast(terms); + if (!(lastExpression instanceof ComparisonExpression)) { + throw new SemanticException(String.format(asofErrMsg, lastExpression)); + } + asofExpression = (ComparisonExpression) lastExpression; + + int size = terms.size() - 1; + for (int i = 0; i < size; i++) { + newTerms.add(terms.get(i)); + } + } else { + throw new SemanticException(joinErrMsg); + } + + // add tolerance condition to expression in super Class, it will be extracted to post-Join + // filter in later process + if (timeDuration != null) { + Expression timeInterval = new LongLiteral(String.valueOf(timeDuration.nonMonthDuration)); + switch (asofExpression.getOperator()) { + case GREATER_THAN: + case GREATER_THAN_OR_EQUAL: + newTerms.add( + new ComparisonExpression( + LESS_THAN_OR_EQUAL, + asofExpression.getLeft(), + new ArithmeticBinaryExpression( + ArithmeticBinaryExpression.Operator.ADD, + asofExpression.getRight(), + timeInterval))); + break; + case LESS_THAN: + case LESS_THAN_OR_EQUAL: + newTerms.add( + new ComparisonExpression( + LESS_THAN_OR_EQUAL, + asofExpression.getRight(), + new ArithmeticBinaryExpression( + ArithmeticBinaryExpression.Operator.ADD, + asofExpression.getLeft(), + timeInterval))); + break; + default: + throw new SemanticException(String.format(asofErrMsg, asofExpression)); + } + } else { + // also check Comparison type + switch (asofExpression.getOperator()) { + case GREATER_THAN: + case GREATER_THAN_OR_EQUAL: + case LESS_THAN: + case LESS_THAN_OR_EQUAL: + break; + default: + throw new SemanticException(String.format(asofErrMsg, asofExpression)); + } + } + + List newTermList = newTerms.build(); + if (newTermList.isEmpty()) { + return new AsofJoinOn(null, asofExpression); + } else { + return new AsofJoinOn(and(newTermList), asofExpression); + } + } + + private JoinCriteria constructAsofJoinOn(Expression joinExpression, long timeInterval) { + return null; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + AsofJoinOn o = (AsofJoinOn) obj; + return Objects.equals(expression, o.expression) + && Objects.equals(asofExpression, o.asofExpression); + } + + @Override + public int hashCode() { + return Objects.hash(expression, asofExpression); + } + + @Override + public String toString() { + return toStringHelper(this).addValue(expression).addValue(asofExpression).toString(); + } + + @Override + public List getNodes() { + return ImmutableList.of(expression, asofExpression); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/DefaultTraversalVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/DefaultTraversalVisitor.java index cf5590d2ce4a8..573482295be7d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/DefaultTraversalVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/DefaultTraversalVisitor.java @@ -366,7 +366,13 @@ protected Void visitJoin(Join node, C context) { node.getCriteria() .filter(JoinOn.class::isInstance) - .ifPresent(criteria -> process(((JoinOn) criteria).getExpression(), context)); + .ifPresent( + criteria -> { + Expression expression = ((JoinOn) criteria).getExpression(); + if (expression != null) { + process(expression, context); + } + }); return null; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/JoinOn.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/JoinOn.java index 16f8fb95fe313..5c86dfaa1aeab 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/JoinOn.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/JoinOn.java @@ -21,20 +21,23 @@ import com.google.common.collect.ImmutableList; +import javax.annotation.Nullable; + import java.util.List; import java.util.Objects; import static com.google.common.base.MoreObjects.toStringHelper; -import static java.util.Objects.requireNonNull; public class JoinOn extends JoinCriteria { - private final Expression expression; + // this can be null when it is AsofJoinOn + @Nullable protected final Expression expression; - public JoinOn(Expression expression) { - this.expression = requireNonNull(expression, "expression is null"); + public JoinOn(@Nullable Expression expression) { + this.expression = expression; } + @Nullable public Expression getExpression() { return expression; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java index a1c4aba7ab497..9b4f8e9d43b0e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java @@ -250,6 +250,7 @@ import java.util.stream.Collectors; import static com.google.common.collect.ImmutableList.toImmutableList; +import static java.lang.Long.parseLong; import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toList; import static org.apache.iotdb.commons.schema.table.TsTable.TIME_COLUMN_NAME; @@ -260,6 +261,7 @@ import static org.apache.iotdb.commons.udf.builtin.relational.TableBuiltinScalarFunction.DATE_BIN; import static org.apache.iotdb.db.queryengine.plan.execution.config.TableConfigTaskVisitor.DATABASE_NOT_SPECIFIED; import static org.apache.iotdb.db.queryengine.plan.parser.ASTVisitor.parseDateTimeFormat; +import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AsofJoinOn.constructAsofJoinOn; import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.GroupingSets.Type.CUBE; import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.GroupingSets.Type.EXPLICIT; import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.GroupingSets.Type.ROLLUP; @@ -2161,13 +2163,43 @@ public Node visitJoinRelation(RelationalSqlParser.JoinRelationContext ctx) { } JoinCriteria criteria; + + // This will be true if criteria is ASOF join criteria with tolerance + boolean hasTolerance = false; + if (ctx.NATURAL() != null) { right = (Relation) visit(ctx.right); criteria = new NaturalJoin(); } else { right = (Relation) visit(ctx.rightRelation); if (ctx.joinCriteria().ON() != null) { - criteria = new JoinOn((Expression) visit(ctx.joinCriteria().booleanExpression())); + if (ctx.ASOF() != null) { + TimeDuration timeDuration = null; + RelationalSqlParser.ToleranceParameterContext toleranceContext = ctx.toleranceParameter(); + if (toleranceContext != null) { + if (toleranceContext.timeDuration() != null) { + timeDuration = + DateTimeUtils.constructTimeDuration(toleranceContext.timeDuration().getText()); + + if (timeDuration.monthDuration != 0) { + throw new SemanticException( + "Month or year interval in tolerance is not supported now."); + } + } else if (toleranceContext.INTEGER_VALUE() != null) { + timeDuration = + new TimeDuration(0, parseLong(toleranceContext.INTEGER_VALUE().getText())); + } else { + throw new IllegalStateException("No time duration or time interval value appears!"); + } + + hasTolerance = true; + } + criteria = + constructAsofJoinOn( + (Expression) visit(ctx.joinCriteria().booleanExpression()), timeDuration); + } else { + criteria = new JoinOn((Expression) visit(ctx.joinCriteria().booleanExpression())); + } } else if (ctx.joinCriteria().USING() != null) { criteria = new JoinUsing(visit(ctx.joinCriteria().identifier(), Identifier.class)); } else { @@ -2186,6 +2218,10 @@ public Node visitJoinRelation(RelationalSqlParser.JoinRelationContext ctx) { joinType = Join.Type.INNER; } + if (joinType != Join.Type.INNER && hasTolerance) { + throw new SemanticException("Tolerance is only supported in ASOF INNER JOIN"); + } + return new Join(getLocation(ctx), joinType, left, right, criteria); } @@ -2800,9 +2836,9 @@ private long parseTimeValue(RelationalSqlParser.TimeValueContext ctx, long curre if (ctx.INTEGER_VALUE() != null) { try { if (ctx.MINUS() != null) { - return -Long.parseLong(ctx.INTEGER_VALUE().getText()); + return -parseLong(ctx.INTEGER_VALUE().getText()); } - return Long.parseLong(ctx.INTEGER_VALUE().getText()); + return parseLong(ctx.INTEGER_VALUE().getText()); } catch (NumberFormatException e) { throw new SemanticException( String.format("Can not parse %s to long value", ctx.INTEGER_VALUE().getText())); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofJoinTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofJoinTest.java new file mode 100644 index 0000000000000..bf220dca9c673 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofJoinTest.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.analyzer; + +import org.apache.iotdb.db.queryengine.plan.relational.planner.PlanTester; +import org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.JoinNode; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.junit.Test; + +import static org.apache.iotdb.db.queryengine.plan.relational.analyzer.TestUtils.assertAnalyzeSemanticException; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanAssert.assertPlan; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.equiJoinClause; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.exchange; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.filter; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.join; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.mergeSort; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.output; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.project; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.sort; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.tableScan; +import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SortItem.NullOrdering.LAST; +import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SortItem.Ordering.ASCENDING; + +public class AsofJoinTest { + @Test + public void simpleTest() { + PlanTester planTester = new PlanTester(); + + assertPlan( + planTester.createPlan( + "select table1.time,table1.s1,table2.time,table2.s1 from table1 asof join table2 on table1.time > table2.time"), + output( + join( + JoinNode.JoinType.INNER, + builder -> + builder + .asofCriteria(ComparisonExpression.Operator.GREATER_THAN, "time", "time_0") + .left( + sort( + ImmutableList.of(sort("time", ASCENDING, LAST)), + tableScan( + "testdb.table1", + ImmutableList.of("time", "s1"), + ImmutableSet.of("time", "s1")))) + .right( + sort( + ImmutableList.of(sort("time_0", ASCENDING, LAST)), + tableScan( + "testdb.table2", + ImmutableMap.of("time_0", "time", "s1_6", "s1"))))))); + } + + @Test + public void toleranceTest() { + PlanTester planTester = new PlanTester(); + + // tolerance will be converted to Filter after Join, predicate is 'table1.time <= table2.time + + // 1' + assertPlan( + planTester.createPlan( + "select table1.time,table1.s1,table2.time,table2.s1 from table1 asof (tolerance 1ms) join table2 on table1.time > table2.time"), + output( + project( + filter( + join( + JoinNode.JoinType.INNER, + builder -> + builder + .asofCriteria( + ComparisonExpression.Operator.GREATER_THAN, "time", "time_0") + .left( + sort( + ImmutableList.of(sort("time", ASCENDING, LAST)), + tableScan( + "testdb.table1", + ImmutableList.of("time", "s1"), + ImmutableSet.of("time", "s1")))) + .right( + sort( + ImmutableList.of(sort("time_0", ASCENDING, LAST)), + project( + tableScan( + "testdb.table2", + ImmutableMap.of( + "time_0", "time", "s1_6", "s1")))))))))); + } + + @Test + // could make use of sort properties of TableScanNode + public void sortEliminateTest() { + PlanTester planTester = new PlanTester(); + + PlanMatchPattern table1 = + tableScan( + "testdb.table1", + ImmutableList.of("time", "tag1", "tag2", "tag3", "s1"), + ImmutableSet.of("time", "tag1", "tag2", "tag3", "s1")); + PlanMatchPattern table2 = + tableScan( + "testdb.table2", + ImmutableMap.of( + "time_0", "time", "tag1_1", "tag1", "tag2_2", "tag2", "tag3_3", "tag3", "s1_6", + "s1")); + + assertPlan( + planTester.createPlan( + "select table1.time,table1.s1,table2.time,table2.s1 from table1 asof join table2 on table1.tag1 = table2.tag1 and table1.tag2 = table2.tag2 and table1.tag3 = table2.tag3 and table1.time > table2.time"), + output( + join( + JoinNode.JoinType.INNER, + builder -> + builder + .asofCriteria(ComparisonExpression.Operator.GREATER_THAN, "time", "time_0") + .equiCriteria( + ImmutableList.of( + equiJoinClause("tag1", "tag1_1"), + equiJoinClause("tag2", "tag2_2"), + equiJoinClause("tag3", "tag3_3"))) + .left(sort(table1)) + .right(sort(table2))))); + + // DistributionPlan + assertPlan( + planTester.getFragmentPlan(0), + output( + join( + JoinNode.JoinType.INNER, + builder -> + builder + .asofCriteria(ComparisonExpression.Operator.GREATER_THAN, "time", "time_0") + .equiCriteria( + ImmutableList.of( + equiJoinClause("tag1", "tag1_1"), + equiJoinClause("tag2", "tag2_2"), + equiJoinClause("tag3", "tag3_3"))) + .left(exchange()) + .right(exchange())))); + + assertPlan(planTester.getFragmentPlan(1), mergeSort(exchange(), exchange(), exchange())); + + // SortNode has been eliminated + assertPlan(planTester.getFragmentPlan(2), table1); + } + + @Test + public void exceptionTest() { + String errMsg = "The join expression of ASOF should be single Comparison or Logical 'AND'"; + // Wrong ASOF main join expression type + assertAnalyzeSemanticException( + "select * from table1 asof join table2 on table1.time is not null", errMsg); + // Wrong concat Logical type + assertAnalyzeSemanticException( + "select * from table1 asof join table2 on table1.tag1 = table2.tag1 or table1.time > table2.time", + errMsg); + + errMsg = + "The main join expression of ASOF should only be Comparison '>, >=, <, <=', actual is (table1.time = table2.time)"; + // Wrong ASOF main join expression operator + assertAnalyzeSemanticException( + "select * from table1 asof join table2 on table1.time = table2.time", errMsg); + assertAnalyzeSemanticException( + "select * from table1 asof (tolerance 1ms) join table2 on table1.time = table2.time", + errMsg); + assertAnalyzeSemanticException( + "select * from table1 asof join table2 on table1.tag1 = table2.tag1 and table1.time = table2.time", + errMsg); + assertAnalyzeSemanticException( + "select * from table1 asof (tolerance 1ms) join table2 on table1.tag1 = table2.tag1 and table1.time = table2.time", + errMsg); + + // Wrong left or right Type of ASOF main join expression + assertAnalyzeSemanticException( + "select * from table1 asof (tolerance 1ms) join table2 on table1.tag1 = table2.tag1 and table1.s1 > table2.s1", + "left child type of ASOF main JOIN expression must be TIMESTAMP: actual type INT64"); + assertAnalyzeSemanticException( + "select * from table1 asof (tolerance 1ms) join table2 on table1.tag1 = table2.tag1 and table1.s1 > table2.time", + "left child type of ASOF main JOIN expression must be TIMESTAMP: actual type INT64"); + assertAnalyzeSemanticException( + "select * from table1 asof (tolerance 1ms) join table2 on table1.tag1 = table2.tag1 and table1.time > table2.s1", + "right child type of ASOF main JOIN expression must be TIMESTAMP: actual type INT64"); + + // ASOF left join doesn't support tolerance, because 'For now, FullOuterJoin and LeftJoin only + // support EquiJoinClauses' + assertAnalyzeSemanticException( + "select * from table1 asof (tolerance 1ms) left join table2 on table1.time > table2.time", + "Tolerance is only supported in ASOF INNER JOIN"); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/AsofJoinClauseProvider.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/AsofJoinClauseProvider.java new file mode 100644 index 0000000000000..c02f41bc8ae06 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/AsofJoinClauseProvider.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.planner.assertions; + +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.JoinNode; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SymbolReference; + +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class AsofJoinClauseProvider implements ExpectedValueProvider { + private final SymbolAlias left; + private final SymbolAlias right; + private final ComparisonExpression.Operator operator; + + public AsofJoinClauseProvider( + ComparisonExpression.Operator operator, SymbolAlias left, SymbolAlias right) { + this.operator = requireNonNull(operator, "operator is null"); + this.left = requireNonNull(left, "left is null"); + this.right = requireNonNull(right, "right is null"); + } + + public ComparisonExpression toExpression() { + return new ComparisonExpression( + operator, new SymbolReference(left.toString()), new SymbolReference(right.toString())); + } + + @Override + public JoinNode.AsofJoinClause getExpectedValue(SymbolAliases aliases) { + return new JoinNode.AsofJoinClause(operator, left.toSymbol(aliases), right.toSymbol(aliases)); + } + + @Override + public String toString() { + return format("%s %s %s", left, operator.getValue(), right); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/JoinMatcher.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/JoinMatcher.java index 90deddbf35b35..090e4a0bc3bc9 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/JoinMatcher.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/JoinMatcher.java @@ -23,6 +23,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.JoinNode; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; import com.google.common.collect.ImmutableList; @@ -38,6 +39,7 @@ import static com.google.common.collect.ImmutableSet.toImmutableSet; import static java.util.Objects.requireNonNull; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.MatchResult.NO_MATCH; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.asofJoinClause; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.equiJoinClause; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.node; @@ -46,12 +48,14 @@ public final class JoinMatcher implements Matcher { private final List> equiCriteria; private final boolean ignoreEquiCriteria; private final Optional filter; + private final Optional asofCriteria; JoinMatcher( JoinNode.JoinType joinType, List> equiCriteria, boolean ignoreEquiCriteria, - Optional filter) { + Optional filter, + Optional asofCriteria) { this.joinType = requireNonNull(joinType, "joinType is null"); this.equiCriteria = requireNonNull(equiCriteria, "equiCriteria is null"); if (ignoreEquiCriteria && !equiCriteria.isEmpty()) { @@ -59,6 +63,7 @@ public final class JoinMatcher implements Matcher { } this.ignoreEquiCriteria = ignoreEquiCriteria; this.filter = requireNonNull(filter, "filter cannot be null"); + this.asofCriteria = requireNonNull(asofCriteria, "asofCriteria cannot be null"); } @Override @@ -114,6 +119,17 @@ public MatchResult detailMatches( } } + if (asofCriteria.isPresent()) { + if (!joinNode.getAsofCriteria().isPresent()) { + return NO_MATCH; + } + if (!new ExpressionVerifier(symbolAliases) + .process( + joinNode.getAsofCriteria().get().toExpression(), asofCriteria.get().toExpression())) { + return NO_MATCH; + } + } + return MatchResult.match(); } @@ -135,6 +151,7 @@ public static class Builder { private PlanMatchPattern right; private Optional filter = Optional.empty(); private boolean ignoreEquiCriteria; + private Optional asofJoinCriteria = Optional.empty(); public Builder(JoinNode.JoinType joinType) { this.joinType = joinType; @@ -155,6 +172,13 @@ public Builder equiCriteria(String left, String right) { return this; } + @CanIgnoreReturnValue + public Builder asofCriteria(ComparisonExpression.Operator operator, String left, String right) { + this.asofJoinCriteria = Optional.of(asofJoinClause(operator, left, right)); + + return this; + } + @CanIgnoreReturnValue public Builder filter(Expression expectedFilter) { this.filter = Optional.of(expectedFilter); @@ -185,7 +209,11 @@ public PlanMatchPattern build() { return node(JoinNode.class, left, right) .with( new JoinMatcher( - joinType, equiCriteria.orElse(ImmutableList.of()), ignoreEquiCriteria, filter)); + joinType, + equiCriteria.orElse(ImmutableList.of()), + ignoreEquiCriteria, + filter, + asofJoinCriteria)); } } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/PlanMatchPattern.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/PlanMatchPattern.java index 7206b1792ec21..e1ba62641a2b8 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/PlanMatchPattern.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/assertions/PlanMatchPattern.java @@ -51,6 +51,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeAlignedDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeNonAlignedDeviceViewScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DataType; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SortItem; @@ -612,6 +613,11 @@ public static ExpectedValueProvider equiJoinClause( return new EquiJoinClauseProvider(new SymbolAlias(left), new SymbolAlias(right)); } + public static AsofJoinClauseProvider asofJoinClause( + ComparisonExpression.Operator operator, String left, String right) { + return new AsofJoinClauseProvider(operator, new SymbolAlias(left), new SymbolAlias(right)); + } + public static SymbolAlias symbol(String alias) { return new SymbolAlias(alias); } diff --git a/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 b/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 index 1776a6ac88f5c..0808ce9be4fdb 100644 --- a/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 +++ b/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 @@ -899,6 +899,7 @@ relation ( CROSS JOIN right=aliasedRelation | joinType JOIN rightRelation=relation joinCriteria | NATURAL joinType JOIN right=aliasedRelation + | ASOF toleranceParameter? joinType JOIN rightRelation=relation joinCriteria ) #joinRelation | aliasedRelation #relationDefault ; @@ -919,6 +920,10 @@ aliasedRelation : relationPrimary (AS? identifier columnAliases?)? ; +toleranceParameter + : '(' TOLERANCE (timeDuration | INTEGER_VALUE) ')' + ; + columnAliases : '(' identifier (',' identifier)* ')' ; @@ -1203,6 +1208,7 @@ ANY: 'ANY'; ARRAY: 'ARRAY'; AS: 'AS'; ASC: 'ASC'; +ASOF: 'ASOF'; AT: 'AT'; ATTRIBUTE: 'ATTRIBUTE'; AUTHORIZATION: 'AUTHORIZATION'; @@ -1527,6 +1533,7 @@ TIMESERIES: 'TIMESERIES'; TIMESLOTID: 'TIMESLOTID'; TIMESTAMP: 'TIMESTAMP'; TO: 'TO'; +TOLERANCE: 'TOLERANCE'; TOPIC: 'TOPIC'; TOPICS: 'TOPICS'; TRAILING: 'TRAILING'; From 93ba4480d9a6329332e46a8cc8d0495a3b2a9f53 Mon Sep 17 00:00:00 2001 From: Weihao Li <18110526956@163.com> Date: Thu, 10 Apr 2025 09:29:23 +0800 Subject: [PATCH 02/11] extra test Signed-off-by: Weihao Li <18110526956@163.com> --- .../relational/analyzer/AsofJoinTest.java | 50 +++++++++++++++++++ .../relational/analyzer/TestMatadata.java | 9 ++++ 2 files changed, 59 insertions(+) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofJoinTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofJoinTest.java index bf220dca9c673..5b8aac75b31c1 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofJoinTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofJoinTest.java @@ -22,7 +22,10 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.PlanTester; import org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.JoinNode; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ArithmeticBinaryExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LongLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SymbolReference; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -33,6 +36,7 @@ import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanAssert.assertPlan; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.equiJoinClause; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.exchange; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.expression; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.filter; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.join; import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.mergeSort; @@ -107,6 +111,52 @@ public void toleranceTest() { "time_0", "time", "s1_6", "s1")))))))))); } + @Test + public void projectInAsofCriteriaTest() { + PlanTester planTester = new PlanTester(); + + // 'table1.tag1 = table2.tag1 and table1.time + 1 > table2.time' + // => left: order by tag1, expr; right: order by tag1_1, time_0 + assertPlan( + planTester.createPlan( + "select table1.time,table1.s1,table2.time,table2.s1 from table1 asof (tolerance 1ms) join table2 on table1.tag1=table2.tag1 and table1.time+1 > table2.time"), + output( + project( + filter( + join( + JoinNode.JoinType.INNER, + builder -> + builder + .asofCriteria( + ComparisonExpression.Operator.GREATER_THAN, "expr", "time_0") + .equiCriteria("tag1", "tag1_1") + .left( + sort( + ImmutableList.of( + sort("tag1", ASCENDING, LAST), + sort("expr", ASCENDING, LAST)), + project( + ImmutableMap.of( + "expr", + expression( + new ArithmeticBinaryExpression( + ArithmeticBinaryExpression.Operator.ADD, + new SymbolReference("time"), + new LongLiteral("1")))), + tableScan( + "testdb.table1", + ImmutableList.of("time", "tag1", "s1"), + ImmutableSet.of("time", "tag1", "s1"))))) + .right( + sort( + project( + tableScan( + "testdb.table2", + ImmutableMap.of( + "time_0", "time", "tag1_1", "tag1", "s1_6", + "s1")))))))))); + } + @Test // could make use of sort properties of TableScanNode public void sortEliminateTest() { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/TestMatadata.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/TestMatadata.java index b5bc2c66f5c35..f0d8f37031b4a 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/TestMatadata.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/TestMatadata.java @@ -222,6 +222,15 @@ public Type getOperatorReturnType( argumentTypes, new IllegalArgumentException("Should have two numeric operands.")); } + if (argumentTypes.get(0).equals(TIMESTAMP) && argumentTypes.get(1).equals(TIMESTAMP)) { + throw new OperatorNotFoundException( + operatorType, + argumentTypes, + new IllegalArgumentException("Cannot apply operator: TIMESTAMP - TIMESTAMP")); + } + if (argumentTypes.get(0).equals(TIMESTAMP) || argumentTypes.get(1).equals(TIMESTAMP)) { + return TIMESTAMP; + } return DOUBLE; case NEGATION: if (!isOneNumericType(argumentTypes)) { From 4f3ab4675eba13f5f87698fcc23733991b978fdb Mon Sep 17 00:00:00 2001 From: Weihao Li <18110526956@163.com> Date: Thu, 10 Apr 2025 12:21:58 +0800 Subject: [PATCH 03/11] fix ci Signed-off-by: Weihao Li <18110526956@163.com> --- .../db/queryengine/plan/relational/analyzer/JoinTest.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/JoinTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/JoinTest.java index f13500196c6c8..56708f246f773 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/JoinTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/JoinTest.java @@ -624,11 +624,6 @@ public void otherInnerJoinTests() { // ========== unsupported test =============== @Test public void unsupportedJoinTest() { - // LEFT JOIN - assertAnalyzeSemanticException( - "SELECT * FROM table1 t1 LEFT JOIN table1 t2 ON t1.time=t2.time", - "LEFT JOIN is not supported, only support INNER JOIN in current version"); - // RIGHT JOIN assertAnalyzeSemanticException( "SELECT * FROM table1 t1 RIGHT JOIN table1 t2 ON t1.time=t2.time", From ba9096d8445ce6b31163766ee6f441443f0b7b5c Mon Sep 17 00:00:00 2001 From: Weihao Li <18110526956@163.com> Date: Wed, 23 Apr 2025 11:38:41 +0800 Subject: [PATCH 04/11] operator finish Signed-off-by: Weihao Li <18110526956@163.com> --- ...cLongTypeIgnoreEqualJoinKeyComparator.java | 95 +++++++ .../comparator/JoinKeyComparatorFactory.java | 13 + .../AsofMergeSortInnerJoinOperator.java | 237 ++++++++++++++++++ .../plan/planner/TableOperatorGenerator.java | 114 ++++----- 4 files changed, 392 insertions(+), 67 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/AscLongTypeIgnoreEqualJoinKeyComparator.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AsofMergeSortInnerJoinOperator.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/AscLongTypeIgnoreEqualJoinKeyComparator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/AscLongTypeIgnoreEqualJoinKeyComparator.java new file mode 100644 index 0000000000000..8b94dde815bfc --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/AscLongTypeIgnoreEqualJoinKeyComparator.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.execution.operator.process.join.merge.comparator; + +import org.apache.tsfile.read.common.block.TsBlock; + +import java.util.Optional; + +// This Comparator is used to handle the case where the join condition is l<=r, making its interface +// consistent with the case that l lessThan( + TsBlock left, + int leftColumnIndex, + int leftRowIndex, + TsBlock right, + int rightColumnIndex, + int rightRowIndex) { + if (left.getColumn(leftColumnIndex).isNull(leftRowIndex) + || right.getColumn(rightColumnIndex).isNull(rightRowIndex)) { + return Optional.empty(); + } + + return Optional.of( + left.getColumn(leftColumnIndex).getLong(leftRowIndex) + <= right.getColumn(rightColumnIndex).getLong(rightRowIndex)); + } + + @Override + public Optional equalsTo( + TsBlock left, + int leftColumnIndex, + int leftRowIndex, + TsBlock right, + int rightColumnIndex, + int rightRowIndex) { + if (left.getColumn(leftColumnIndex).isNull(leftRowIndex) + || right.getColumn(rightColumnIndex).isNull(rightRowIndex)) { + return Optional.empty(); + } + + return Optional.of( + left.getColumn(leftColumnIndex).getLong(leftRowIndex) + == right.getColumn(rightColumnIndex).getLong(rightRowIndex)); + } + + @Override + public Optional lessThanOrEqual( + TsBlock left, + int leftColumnIndex, + int leftRowIndex, + TsBlock right, + int rightColumnIndex, + int rightRowIndex) { + if (left.getColumn(leftColumnIndex).isNull(leftRowIndex) + || right.getColumn(rightColumnIndex).isNull(rightRowIndex)) { + return Optional.empty(); + } + + return Optional.of( + left.getColumn(leftColumnIndex).getLong(leftRowIndex) + < right.getColumn(rightColumnIndex).getLong(rightRowIndex)); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/JoinKeyComparatorFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/JoinKeyComparatorFactory.java index 33620690fed63..728cd5e64ce05 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/JoinKeyComparatorFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/JoinKeyComparatorFactory.java @@ -36,6 +36,19 @@ public static List getComparators( return comparators; } + public static List getAsofComparators( + List joinKeyTypes, boolean ignoreEqual) { + List comparators = new ArrayList<>(joinKeyTypes.size()); + for (Type joinKeyType : joinKeyTypes) { + comparators.add(getComparator(joinKeyType, true)); + } + comparators.add( + ignoreEqual + ? AscLongTypeIgnoreEqualJoinKeyComparator.getInstance() + : AscLongTypeJoinKeyComparator.getInstance()); + return comparators; + } + public static JoinKeyComparator getComparator(Type type, boolean isAscending) { switch (type.getTypeEnum()) { case INT32: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AsofMergeSortInnerJoinOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AsofMergeSortInnerJoinOperator.java new file mode 100644 index 0000000000000..b8fc88b3a2ce5 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AsofMergeSortInnerJoinOperator.java @@ -0,0 +1,237 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.execution.operator.source.relational; + +import org.apache.iotdb.db.queryengine.execution.operator.Operator; +import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.execution.operator.process.join.merge.comparator.JoinKeyComparator; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; + +import java.util.List; + +public class AsofMergeSortInnerJoinOperator extends MergeSortInnerJoinOperator { + private final JoinKeyComparator asofComparator; + private final int leftAsofJoinKeyIndex; + private final int rightAsofJoinKeyIndex; + + public AsofMergeSortInnerJoinOperator( + OperatorContext operatorContext, + Operator leftChild, + int[] leftJoinKeyPositions, + int[] leftOutputSymbolIdx, + Operator rightChild, + int[] rightJoinKeyPositions, + int[] rightOutputSymbolIdx, + List joinKeyComparators, + List dataTypes) { + super( + operatorContext, + leftChild, + leftJoinKeyPositions, + leftOutputSymbolIdx, + rightChild, + rightJoinKeyPositions, + rightOutputSymbolIdx, + joinKeyComparators, + dataTypes); + this.asofComparator = joinKeyComparators.get(joinKeyComparators.size() - 1); + this.leftAsofJoinKeyIndex = leftJoinKeyPositions[leftJoinKeyPositions.length - 1]; + this.rightAsofJoinKeyIndex = rightJoinKeyPositions[rightJoinKeyPositions.length - 1]; + } + + @Override + protected boolean processFinished() { + // skip all NULL values in left, because NULL value can not appear in the inner join result + while (currentLeftHasNullValue()) { + if (leftFinishedWithIncIndex()) { + return true; + } + } + + // all the join keys in rightTsBlock are less or equal than leftTsBlock, just skip right + if (allRightLessOrEqualThanLeft()) { + resetRightBlockList(); + return true; + } + + // skip all NULL values in right, because NULL value can not appear in the inner join result + while (currentRightHasNullValue()) { + if (rightFinishedWithIncIndex()) { + return true; + } + } + + // find first candidate of right meets the conditions + while (lessThanOrEqual( + rightBlockList.get(rightBlockListIdx), + rightJoinKeyPositions, + rightIndex, + leftBlock, + leftJoinKeyPositions, + leftIndex)) { + if (rightFinishedWithIncIndex()) { + return true; + } + } + if (currentRoundNeedStop()) { + return true; + } + + // has right values meet condition, append to join result + hasMatchedRightValueToProbeLeft(); + // always inc leftIndex after current left result appended + return leftFinishedWithIncIndex(); + } + + // check if the last value of the right is less or equal than left + protected boolean allRightLessOrEqualThanLeft() { + return lessThanOrEqual( + rightBlockList.get(rightBlockList.size() - 1), + rightJoinKeyPositions, + rightBlockList.get(rightBlockList.size() - 1).getPositionCount() - 1, + leftBlock, + leftJoinKeyPositions, + leftIndex); + } + + private boolean lessThanOrEqual( + TsBlock leftBlock, + int[] leftPositions, + int lIndex, + TsBlock rightBlock, + int[] rightPositions, + int rIndex) { + // if join key size equals to 1, can return true in inner join + if (rightPositions.length == 1 && rightBlock.getColumn(rightPositions[0]).isNull(rIndex)) { + return true; + } + + int lastIndex = comparators.size() - 1; + for (int i = 0; i < lastIndex; i++) { + if (comparators + .get(i) + .lessThan(leftBlock, leftPositions[i], lIndex, rightBlock, rightPositions[i], rIndex) + .orElse(false)) { + return true; + } else if (!comparators + .get(i) + .equalsTo(leftBlock, leftPositions[i], lIndex, rightBlock, rightPositions[i], rIndex) + .orElse(false)) { + return false; + } + } + + return comparators + .get(lastIndex) + .lessThanOrEqual( + leftBlock, + leftPositions[lastIndex], + lIndex, + rightBlock, + rightPositions[lastIndex], + rIndex) + .orElse(false); + } + + /** + * Examine if stop this round and rebuild rightBlockLists. + * + * @return true if rightBlockListIdx more than zero. + */ + protected boolean currentRoundNeedStop() { + if (rightBlockListIdx > 0) { + for (int i = 0; i < rightBlockListIdx; i++) { + long size = rightBlockList.get(i).getRetainedSizeInBytes(); + usedMemory -= size; + memoryReservationManager.releaseMemoryCumulatively(size); + } + rightBlockList = rightBlockList.subList(rightBlockListIdx, rightBlockList.size()); + rightBlockListIdx = 0; + return true; + } + + return false; + } + + public boolean hasMatchedRightValueToProbeLeft() { + int tmpBlockIdx = rightBlockListIdx; + int tmpIdx = rightIndex; + boolean hasMatched = false; + long matchedTime = Long.MIN_VALUE; + while (equalsTo( + leftBlock, + leftJoinKeyPositions, + leftIndex, + rightBlockList.get(tmpBlockIdx), + rightJoinKeyPositions, + tmpIdx) + && asofComparator + .lessThan( + leftBlock, + leftAsofJoinKeyIndex, + leftIndex, + rightBlockList.get(rightBlockListIdx), + rightAsofJoinKeyIndex, + rightIndex) + .orElse(false)) { + long currentTime = + rightBlockList.get(tmpBlockIdx).getColumn(rightAsofJoinKeyIndex).getLong(tmpIdx); + if (matchedTime == Long.MIN_VALUE) { + matchedTime = currentTime; + } else if (currentTime != matchedTime) { + break; + } + + hasMatched = true; + appendValueToResultWhenMatches(tmpBlockIdx, tmpIdx); + + tmpIdx++; + if (tmpIdx >= rightBlockList.get(tmpBlockIdx).getPositionCount()) { + tmpIdx = 0; + tmpBlockIdx++; + } + + if (tmpBlockIdx >= rightBlockList.size()) { + break; + } + } + return hasMatched; + } + + protected boolean equalsTo( + TsBlock leftBlock, + int[] leftPositions, + int lIndex, + TsBlock rightBlock, + int[] rightPositions, + int rIndex) { + for (int i = 0; i < comparators.size() - 1; i++) { + if (!comparators + .get(i) + .equalsTo(leftBlock, leftPositions[i], lIndex, rightBlock, rightPositions[i], rIndex) + .orElse(false)) { + return false; + } + } + return true; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java index 19ad9a15e0f80..8aabc4556bd1b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java @@ -83,6 +83,7 @@ import org.apache.iotdb.db.queryengine.execution.operator.source.ExchangeOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.AbstractAggTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.AbstractTableScanOperator; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.AsofMergeSortInnerJoinOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.DefaultAggTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.InformationSchemaTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.LastQueryAggTableScanOperator; @@ -166,6 +167,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.TableDeviceFetchNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.TableDeviceQueryCountNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.TableDeviceQueryScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.FunctionCall; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Literal; @@ -1429,9 +1431,11 @@ public Operator visitJoin(JoinNode node, LocalExecutionPlanContext context) { semanticCheckForJoin(node); - int size = node.getCriteria().size(); + JoinNode.AsofJoinClause asofJoinClause = node.getAsofCriteria().orElse(null); + int equiSize = node.getCriteria().size(); + int size = equiSize + (asofJoinClause == null ? 0 : 1); int[] leftJoinKeyPositions = new int[size]; - for (int i = 0; i < size; i++) { + for (int i = 0; i < equiSize; i++) { Integer leftJoinKeyPosition = leftColumnNamesMap.get(node.getCriteria().get(i).getLeft()); if (leftJoinKeyPosition == null) { throw new IllegalStateException("Left child of JoinNode doesn't contain left join key."); @@ -1441,7 +1445,7 @@ public Operator visitJoin(JoinNode node, LocalExecutionPlanContext context) { List joinKeyTypes = new ArrayList<>(size); int[] rightJoinKeyPositions = new int[size]; - for (int i = 0; i < size; i++) { + for (int i = 0; i < equiSize; i++) { Integer rightJoinKeyPosition = rightColumnNamesMap.get(node.getCriteria().get(i).getRight()); if (rightJoinKeyPosition == null) { throw new IllegalStateException("Right child of JoinNode doesn't contain right join key."); @@ -1456,80 +1460,56 @@ public Operator visitJoin(JoinNode node, LocalExecutionPlanContext context) { joinKeyTypes.add(leftJoinKeyType); } - JoinNode.AsofJoinClause asofJoinClause = node.getAsofCriteria().orElse(null); if (asofJoinClause != null) { - throw new UnsupportedOperationException("BE of ASOF Join is not supported!"); - /*BiPredicate asofPredicate; - switch (asofJoinClause.getOperator()) { - case LESS_THAN: - asofPredicate = (left, right) -> left < right; - break; - case LESS_THAN_OR_EQUAL: - asofPredicate = (left, right) -> left <= right; - break; - case GREATER_THAN: - asofPredicate = (left, right) -> left > right; - break; - case GREATER_THAN_OR_EQUAL: - asofPredicate = (left, right) -> left >= right; - break; - default: - throw new IllegalStateException(String.format("Unexpected ASOF Join criteria Operator: %s",asofJoinClause.getOperator())); + Integer leftAsofJoinKeyPosition = leftColumnNamesMap.get(asofJoinClause.getLeft()); + if (leftAsofJoinKeyPosition == null) { + throw new IllegalStateException( + "Left child of JoinNode doesn't contain left ASOF main join key."); + } + leftJoinKeyPositions[equiSize] = leftAsofJoinKeyPosition; + Integer rightAsofJoinKeyPosition = rightColumnNamesMap.get(asofJoinClause.getRight()); + if (rightAsofJoinKeyPosition == null) { + throw new IllegalStateException( + "Right child of JoinNode doesn't contain right ASOF main join key."); } + rightJoinKeyPositions[equiSize] = rightAsofJoinKeyPosition; - Integer leftAsofJoinKeyIndex = leftColumnNamesMap.get(asofJoinClause.getLeft()); - if (leftAsofJoinKeyIndex == null) { - throw new IllegalStateException("Left child of JoinNode doesn't contain left ASOF main join key."); + if (context.getTypeProvider().getTableModelType(asofJoinClause.getLeft()) != TIMESTAMP) { + throw new IllegalStateException("Type of left ASOF Join key is not TIMESTAMP"); } - Integer rightAsofJoinKeyIndex = rightColumnNamesMap.get(asofJoinClause.getRight()); - if (rightAsofJoinKeyIndex == null) { - throw new IllegalStateException("Right child of JoinNode doesn't contain right ASOF main join key."); + if (context.getTypeProvider().getTableModelType(asofJoinClause.getRight()) != TIMESTAMP) { + throw new IllegalStateException("Type of right ASOF Join key is not TIMESTAMP"); } + + ComparisonExpression.Operator asofOperator = asofJoinClause.getOperator(); + if (asofOperator != ComparisonExpression.Operator.LESS_THAN + && asofOperator != ComparisonExpression.Operator.LESS_THAN_OR_EQUAL) { + throw new IllegalStateException( + String.format("Unexpected asofOperator here: %s", asofOperator)); + } + if (requireNonNull(node.getJoinType()) == JoinNode.JoinType.INNER) { OperatorContext operatorContext = - context - .getDriverContext() - .addOperatorContext( - context.getNextOperatorId(), - node.getPlanNodeId(), - AsofMergeSortInnerJoinOperator.class.getSimpleName()); - return new AsofMergeSortInnerJoinOperator( - operatorContext, - asofPredicate, - leftChild, - leftJoinKeyPositions, - leftAsofJoinKeyIndex, - leftOutputSymbolIdx, - rightChild, - rightJoinKeyPositions, - rightAsofJoinKeyIndex, - rightOutputSymbolIdx, - JoinKeyComparatorFactory.getComparators(joinKeyTypes, true), - dataTypes); - } else if(requireNonNull(node.getJoinType()) == JoinNode.JoinType.LEFT) { - OperatorContext operatorContext = - context - .getDriverContext() - .addOperatorContext( - context.getNextOperatorId(), - node.getPlanNodeId(), - AsofMergeSortLeftJoinOperator.class.getSimpleName()); + context + .getDriverContext() + .addOperatorContext( + context.getNextOperatorId(), + node.getPlanNodeId(), + AsofMergeSortInnerJoinOperator.class.getSimpleName()); return new AsofMergeSortInnerJoinOperator( - operatorContext, - asofPredicate, - leftChild, - leftJoinKeyPositions, - leftAsofJoinKeyIndex, - leftOutputSymbolIdx, - rightChild, - rightJoinKeyPositions, - rightAsofJoinKeyIndex, - rightOutputSymbolIdx, - JoinKeyComparatorFactory.getComparators(joinKeyTypes, true), - dataTypes); + operatorContext, + leftChild, + leftJoinKeyPositions, + leftOutputSymbolIdx, + rightChild, + rightJoinKeyPositions, + rightOutputSymbolIdx, + JoinKeyComparatorFactory.getAsofComparators( + joinKeyTypes, asofOperator == ComparisonExpression.Operator.LESS_THAN_OR_EQUAL), + dataTypes); } else { throw new IllegalStateException("Unsupported ASOF join type: " + node.getJoinType()); - }*/ + } } if (requireNonNull(node.getJoinType()) == JoinNode.JoinType.INNER) { From 8b615276c5d93a28c923ffcf948c715cf2d51ee2 Mon Sep 17 00:00:00 2001 From: Weihao Li <18110526956@163.com> Date: Thu, 24 Apr 2025 10:09:15 +0800 Subject: [PATCH 05/11] modify '>' '>=' Signed-off-by: Weihao Li <18110526956@163.com> --- ...cLongTypeIgnoreEqualJoinKeyComparator.java | 95 +++++++++++++++++++ .../comparator/JoinKeyComparatorFactory.java | 17 +++- .../plan/planner/TableOperatorGenerator.java | 10 +- .../relational/planner/node/JoinNode.java | 13 +++ .../PushPredicateIntoTableScan.java | 9 +- .../plan/relational/sql/ast/AsofJoinOn.java | 8 +- .../relational/sql/parser/AstBuilder.java | 9 +- .../relational/analyzer/AsofJoinTest.java | 40 ++++++-- 8 files changed, 165 insertions(+), 36 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/DescLongTypeIgnoreEqualJoinKeyComparator.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/DescLongTypeIgnoreEqualJoinKeyComparator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/DescLongTypeIgnoreEqualJoinKeyComparator.java new file mode 100644 index 0000000000000..59ea63730d016 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/DescLongTypeIgnoreEqualJoinKeyComparator.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.execution.operator.process.join.merge.comparator; + +import org.apache.tsfile.read.common.block.TsBlock; + +import java.util.Optional; + +// This Comparator is used to handle the case where the join condition is l<=r, making its interface +// consistent with the case that l lessThan( + TsBlock left, + int leftColumnIndex, + int leftRowIndex, + TsBlock right, + int rightColumnIndex, + int rightRowIndex) { + if (left.getColumn(leftColumnIndex).isNull(leftRowIndex) + || right.getColumn(rightColumnIndex).isNull(rightRowIndex)) { + return Optional.empty(); + } + + return Optional.of( + left.getColumn(leftColumnIndex).getLong(leftRowIndex) + >= right.getColumn(rightColumnIndex).getLong(rightRowIndex)); + } + + @Override + public Optional equalsTo( + TsBlock left, + int leftColumnIndex, + int leftRowIndex, + TsBlock right, + int rightColumnIndex, + int rightRowIndex) { + if (left.getColumn(leftColumnIndex).isNull(leftRowIndex) + || right.getColumn(rightColumnIndex).isNull(rightRowIndex)) { + return Optional.empty(); + } + + return Optional.of( + left.getColumn(leftColumnIndex).getLong(leftRowIndex) + == right.getColumn(rightColumnIndex).getLong(rightRowIndex)); + } + + @Override + public Optional lessThanOrEqual( + TsBlock left, + int leftColumnIndex, + int leftRowIndex, + TsBlock right, + int rightColumnIndex, + int rightRowIndex) { + if (left.getColumn(leftColumnIndex).isNull(leftRowIndex) + || right.getColumn(rightColumnIndex).isNull(rightRowIndex)) { + return Optional.empty(); + } + + return Optional.of( + left.getColumn(leftColumnIndex).getLong(leftRowIndex) + > right.getColumn(rightColumnIndex).getLong(rightRowIndex)); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/JoinKeyComparatorFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/JoinKeyComparatorFactory.java index 728cd5e64ce05..4ab5f87fb798b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/JoinKeyComparatorFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/merge/comparator/JoinKeyComparatorFactory.java @@ -37,15 +37,22 @@ public static List getComparators( } public static List getAsofComparators( - List joinKeyTypes, boolean ignoreEqual) { + List joinKeyTypes, boolean ignoreEqual, boolean asofAscending) { List comparators = new ArrayList<>(joinKeyTypes.size()); for (Type joinKeyType : joinKeyTypes) { comparators.add(getComparator(joinKeyType, true)); } - comparators.add( - ignoreEqual - ? AscLongTypeIgnoreEqualJoinKeyComparator.getInstance() - : AscLongTypeJoinKeyComparator.getInstance()); + if (asofAscending) { + comparators.add( + ignoreEqual + ? AscLongTypeIgnoreEqualJoinKeyComparator.getInstance() + : AscLongTypeJoinKeyComparator.getInstance()); + } else { + comparators.add( + ignoreEqual + ? DescLongTypeIgnoreEqualJoinKeyComparator.getInstance() + : DescLongTypeJoinKeyComparator.getInstance()); + } return comparators; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java index 8aabc4556bd1b..da2f912429444 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java @@ -1482,11 +1482,6 @@ public Operator visitJoin(JoinNode node, LocalExecutionPlanContext context) { } ComparisonExpression.Operator asofOperator = asofJoinClause.getOperator(); - if (asofOperator != ComparisonExpression.Operator.LESS_THAN - && asofOperator != ComparisonExpression.Operator.LESS_THAN_OR_EQUAL) { - throw new IllegalStateException( - String.format("Unexpected asofOperator here: %s", asofOperator)); - } if (requireNonNull(node.getJoinType()) == JoinNode.JoinType.INNER) { OperatorContext operatorContext = @@ -1505,7 +1500,10 @@ public Operator visitJoin(JoinNode node, LocalExecutionPlanContext context) { rightJoinKeyPositions, rightOutputSymbolIdx, JoinKeyComparatorFactory.getAsofComparators( - joinKeyTypes, asofOperator == ComparisonExpression.Operator.LESS_THAN_OR_EQUAL), + joinKeyTypes, + asofOperator == ComparisonExpression.Operator.LESS_THAN_OR_EQUAL + || asofOperator == ComparisonExpression.Operator.GREATER_THAN_OR_EQUAL, + !asofJoinClause.isOperatorContainsGreater()), dataTypes); } else { throw new IllegalStateException("Unsupported ASOF join type: " + node.getJoinType()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/JoinNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/JoinNode.java index ca4807c064804..50a67915dbbd9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/JoinNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/JoinNode.java @@ -422,6 +422,19 @@ public AsofJoinClause flip() { return new AsofJoinClause(operator.flip(), right, left); } + public boolean isOperatorContainsGreater() { + switch (operator) { + case GREATER_THAN: + case GREATER_THAN_OR_EQUAL: + return true; + case LESS_THAN: + case LESS_THAN_OR_EQUAL: + return false; + default: + throw new IllegalArgumentException("Invalid operator type: " + operator); + } + } + @Override public boolean equals(Object obj) { if (this == obj) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java index 5d238341de570..e392024338da0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java @@ -106,6 +106,7 @@ import static org.apache.iotdb.db.queryengine.plan.relational.planner.ExpressionSymbolInliner.inlineSymbols; import static org.apache.iotdb.db.queryengine.plan.relational.planner.SortOrder.ASC_NULLS_FIRST; import static org.apache.iotdb.db.queryengine.plan.relational.planner.SortOrder.ASC_NULLS_LAST; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.SortOrder.DESC_NULLS_LAST; import static org.apache.iotdb.db.queryengine.plan.relational.planner.SymbolsExtractor.extractUnique; import static org.apache.iotdb.db.queryengine.plan.relational.planner.ir.DeterminismEvaluator.isDeterministic; import static org.apache.iotdb.db.queryengine.plan.relational.planner.ir.GlobalTimePredicateExtractVisitor.extractGlobalTimeFilter; @@ -890,11 +891,13 @@ private void appendSortNodeForMergeSortJoin(JoinNode joinNode) { rightOrderings.put(equiJoinClause.getRight(), ASC_NULLS_LAST); } if (asofJoinClause != null) { + // if operator of AsofJoinClause is '>' or '>=', use DESC ordering for convenience of + // process in BE + boolean needDesc = asofJoinClause.isOperatorContainsGreater(); leftOrderBy.add(asofJoinClause.getLeft()); - leftOrderings.put(asofJoinClause.getLeft(), ASC_NULLS_LAST); + leftOrderings.put(asofJoinClause.getLeft(), needDesc ? DESC_NULLS_LAST : ASC_NULLS_LAST); rightOrderBy.add(asofJoinClause.getRight()); - rightOrderings.put(asofJoinClause.getRight(), ASC_NULLS_LAST); - ; + rightOrderings.put(asofJoinClause.getRight(), needDesc ? DESC_NULLS_LAST : ASC_NULLS_LAST); } OrderingScheme leftOrderingScheme = new OrderingScheme(leftOrderBy, leftOrderings); OrderingScheme rightOrderingScheme = new OrderingScheme(rightOrderBy, rightOrderings); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/AsofJoinOn.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/AsofJoinOn.java index 8b7229af32aa3..c19606c754893 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/AsofJoinOn.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/AsofJoinOn.java @@ -36,8 +36,8 @@ public class AsofJoinOn extends JoinOn { // record main expression of ASOF join // .e.g 'ASOF (tolerance 1) JOIN ON t1.device = t2.device and t1.time > t2.time' => - // asofExpression: - // 't1.time > t2.time', expression in super Class: 't1.device = t2.device and t1.time <= t2.time + + // asofExpression:'t1.time > t2.time', expression in super Class: 't1.device = t2.device and + // t1.time <= t2.time + // 1' private final Expression asofExpression; @@ -133,10 +133,6 @@ public static JoinCriteria constructAsofJoinOn( } } - private JoinCriteria constructAsofJoinOn(Expression joinExpression, long timeInterval) { - return null; - } - @Override public boolean equals(Object obj) { if (this == obj) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java index 9b4f8e9d43b0e..32e0dcd5db5d8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java @@ -2164,9 +2164,6 @@ public Node visitJoinRelation(RelationalSqlParser.JoinRelationContext ctx) { JoinCriteria criteria; - // This will be true if criteria is ASOF join criteria with tolerance - boolean hasTolerance = false; - if (ctx.NATURAL() != null) { right = (Relation) visit(ctx.right); criteria = new NaturalJoin(); @@ -2191,8 +2188,6 @@ public Node visitJoinRelation(RelationalSqlParser.JoinRelationContext ctx) { } else { throw new IllegalStateException("No time duration or time interval value appears!"); } - - hasTolerance = true; } criteria = constructAsofJoinOn( @@ -2218,8 +2213,8 @@ public Node visitJoinRelation(RelationalSqlParser.JoinRelationContext ctx) { joinType = Join.Type.INNER; } - if (joinType != Join.Type.INNER && hasTolerance) { - throw new SemanticException("Tolerance is only supported in ASOF INNER JOIN"); + if (joinType != Join.Type.INNER) { + throw new SemanticException("ASOF JOIN is only support INNER type now"); } return new Join(getLocation(ctx), joinType, left, right, criteria); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofJoinTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofJoinTest.java index 5b8aac75b31c1..6c92220780a4b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofJoinTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofJoinTest.java @@ -46,6 +46,7 @@ import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.tableScan; import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SortItem.NullOrdering.LAST; import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SortItem.Ordering.ASCENDING; +import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SortItem.Ordering.DESCENDING; public class AsofJoinTest { @Test @@ -54,13 +55,13 @@ public void simpleTest() { assertPlan( planTester.createPlan( - "select table1.time,table1.s1,table2.time,table2.s1 from table1 asof join table2 on table1.time > table2.time"), + "select table1.time,table1.s1,table2.time,table2.s1 from table1 asof join table2 on table1.time < table2.time"), output( join( JoinNode.JoinType.INNER, builder -> builder - .asofCriteria(ComparisonExpression.Operator.GREATER_THAN, "time", "time_0") + .asofCriteria(ComparisonExpression.Operator.LESS_THAN, "time", "time_0") .left( sort( ImmutableList.of(sort("time", ASCENDING, LAST)), @@ -74,6 +75,30 @@ public void simpleTest() { tableScan( "testdb.table2", ImmutableMap.of("time_0", "time", "s1_6", "s1"))))))); + + // Asof operator is '>', appended ordering is DESC + assertPlan( + planTester.createPlan( + "select table1.time,table1.s1,table2.time,table2.s1 from table1 asof join table2 on table1.time > table2.time"), + output( + join( + JoinNode.JoinType.INNER, + builder -> + builder + .asofCriteria(ComparisonExpression.Operator.GREATER_THAN, "time", "time_0") + .left( + sort( + ImmutableList.of(sort("time", DESCENDING, LAST)), + tableScan( + "testdb.table1", + ImmutableList.of("time", "s1"), + ImmutableSet.of("time", "s1")))) + .right( + sort( + ImmutableList.of(sort("time_0", DESCENDING, LAST)), + tableScan( + "testdb.table2", + ImmutableMap.of("time_0", "time", "s1_6", "s1"))))))); } @Test @@ -96,14 +121,14 @@ public void toleranceTest() { ComparisonExpression.Operator.GREATER_THAN, "time", "time_0") .left( sort( - ImmutableList.of(sort("time", ASCENDING, LAST)), + ImmutableList.of(sort("time", DESCENDING, LAST)), tableScan( "testdb.table1", ImmutableList.of("time", "s1"), ImmutableSet.of("time", "s1")))) .right( sort( - ImmutableList.of(sort("time_0", ASCENDING, LAST)), + ImmutableList.of(sort("time_0", DESCENDING, LAST)), project( tableScan( "testdb.table2", @@ -134,7 +159,7 @@ public void projectInAsofCriteriaTest() { sort( ImmutableList.of( sort("tag1", ASCENDING, LAST), - sort("expr", ASCENDING, LAST)), + sort("expr", DESCENDING, LAST)), project( ImmutableMap.of( "expr", @@ -250,11 +275,8 @@ public void exceptionTest() { assertAnalyzeSemanticException( "select * from table1 asof (tolerance 1ms) join table2 on table1.tag1 = table2.tag1 and table1.time > table2.s1", "right child type of ASOF main JOIN expression must be TIMESTAMP: actual type INT64"); - - // ASOF left join doesn't support tolerance, because 'For now, FullOuterJoin and LeftJoin only - // support EquiJoinClauses' assertAnalyzeSemanticException( "select * from table1 asof (tolerance 1ms) left join table2 on table1.time > table2.time", - "Tolerance is only supported in ASOF INNER JOIN"); + "ASOF JOIN is only support INNER type now"); } } From 8772c3a0bb6666b70f50b46ca48223dddcda39a9 Mon Sep 17 00:00:00 2001 From: Weihao Li <18110526956@163.com> Date: Thu, 24 Apr 2025 20:31:54 +0800 Subject: [PATCH 06/11] fix add it Signed-off-by: Weihao Li <18110526956@163.com> --- .../IoTDBMultiTAGsWithAttributesTableIT.java | 115 ++++++++++++++++++ .../rule/PruneJoinChildrenColumns.java | 10 ++ .../relational/planner/node/JoinNode.java | 10 ++ 3 files changed, 135 insertions(+) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBMultiTAGsWithAttributesTableIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBMultiTAGsWithAttributesTableIT.java index caeba55e32e44..de1215ee97957 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBMultiTAGsWithAttributesTableIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBMultiTAGsWithAttributesTableIT.java @@ -2213,6 +2213,121 @@ public void lastCacheTest() { repeatTest(sql, expectedHeader, retArray, DATABASE_NAME, 3); } + @Test + public void asofJoinTest() { + expectedHeader = new String[] {"time", "device", "level", "time", "device", "level"}; + retArray = + new String[] { + "1971-01-01T00:00:00.000Z,d1,l1,1970-01-01T00:00:00.100Z,d1,l5,", + "1971-01-01T00:01:40.000Z,d1,l1,1971-01-01T00:00:00.000Z,d1,l1,", + "1971-01-01T00:01:40.000Z,d1,l1,1971-01-01T00:00:00.000Z,d999,null,", + "1971-01-01T00:01:40.000Z,d1,l1,1971-01-01T00:00:00.000Z,null,l999,", + "1970-01-01T00:00:00.020Z,d1,l2,1970-01-01T00:00:00.010Z,d11,l11,", + "1971-01-01T00:00:00.100Z,d1,l2,1971-01-01T00:00:00.000Z,d1,l1,", + "1971-01-01T00:00:00.100Z,d1,l2,1971-01-01T00:00:00.000Z,d999,null,", + "1971-01-01T00:00:00.100Z,d1,l2,1971-01-01T00:00:00.000Z,null,l999,", + "1971-04-26T17:46:40.000Z,d1,l2,1971-01-01T00:00:00.000Z,d1,l1,", + "1971-04-26T17:46:40.000Z,d1,l2,1971-01-01T00:00:00.000Z,d999,null," + }; + // test single join condition + tableResultSetEqualTest( + "select table0.time,table0.device,table0.level,table1.time,table1.device,table1.level from table0 asof join table1 on " + + "table0.time>table1.time " + + "order by table0.device,table0.level,table0.time,table1.device,table1.level limit 10", + expectedHeader, + retArray, + DATABASE_NAME); + // test expr and '>=' in ASOF condition + tableResultSetEqualTest( + "select table0.time,table0.device,table0.level,table1.time,table1.device,table1.level from table0 asof join table1 on " + + "table0.time>=table1.time+1 " + + "order by table0.device,table0.level,table0.time,table1.device,table1.level limit 10", + expectedHeader, + retArray, + DATABASE_NAME); + + retArray = + new String[] { + "1971-01-01T00:00:00.000Z,d1,l1,1970-01-01T00:00:00.000Z,d1,l1,", + "1971-01-01T00:01:40.000Z,d1,l1,1971-01-01T00:00:00.000Z,d1,l1,", + "1971-01-01T00:00:00.100Z,d1,l2,1970-01-01T00:00:00.020Z,d1,l2,", + "1971-04-26T17:46:40.000Z,d1,l2,1970-01-01T00:00:00.020Z,d1,l2,", + "1971-01-01T00:00:00.500Z,d1,l3,1970-01-01T00:00:00.040Z,d1,l3,", + "1971-04-26T17:46:40.020Z,d1,l3,1970-01-01T00:00:00.040Z,d1,l3,", + "1971-01-01T00:00:01.000Z,d1,l4,1970-01-01T00:00:00.080Z,d1,l4,", + "1971-04-26T18:01:40.000Z,d1,l4,1970-01-01T00:00:00.080Z,d1,l4,", + "1971-01-01T00:00:10.000Z,d1,l5,1970-01-01T00:00:00.100Z,d1,l5,", + "1971-08-20T11:33:20.000Z,d1,l5,1970-01-01T00:00:00.100Z,d1,l5," + }; + // test multi join conditions + tableResultSetEqualTest( + "select table0.time,table0.device,table0.level,table1.time,table1.device,table1.level from table0 asof join table1 on " + + "table0.device=table1.device and table1.level=table0.level and table0.time>table1.time " + + "order by table0.device,table0.level,table0.time,table1.device,table1.level", + expectedHeader, + retArray, + DATABASE_NAME); + // test expr and '>=' in ASOF condition + tableResultSetEqualTest( + "select table0.time,table0.device,table0.level,table1.time,table1.device,table1.level from table0 asof join table1 on " + + "table0.device=table1.device and table1.level=table0.level and table0.time>=table1.time+1 " + + "order by table0.device,table0.level,table0.time,table1.device,table1.level", + expectedHeader, + retArray, + DATABASE_NAME); + + retArray = + new String[] { + "1970-01-01T00:00:00.000Z,d1,l1,1970-01-01T00:00:00.010Z,d11,l11,", + "1970-01-01T00:00:00.020Z,d1,l2,1970-01-01T00:00:00.030Z,d11,l11,", + "1970-01-01T00:00:00.040Z,d1,l3,1970-01-01T00:00:00.080Z,d1,l4,", + "1970-01-01T00:00:00.080Z,d1,l4,1970-01-01T00:00:00.100Z,d1,l5,", + "1970-01-01T00:00:00.100Z,d1,l5,1971-01-01T00:00:00.000Z,d1,l1,", + "1970-01-01T00:00:00.100Z,d1,l5,1971-01-01T00:00:00.000Z,d999,null,", + "1970-01-01T00:00:00.100Z,d1,l5,1971-01-01T00:00:00.000Z,null,l999,", + "1970-01-01T00:00:00.000Z,d2,l1,1970-01-01T00:00:00.010Z,d11,l11,", + "1970-01-01T00:00:00.020Z,d2,l2,1970-01-01T00:00:00.030Z,d11,l11,", + "1970-01-01T00:00:00.040Z,d2,l3,1970-01-01T00:00:00.080Z,d1,l4," + }; + // test single join condition + tableResultSetEqualTest( + "select table0.time,table0.device,table0.level,table1.time,table1.device,table1.level from table0 asof join table1 on " + + "table0.time=' in ASOF condition + tableResultSetEqualTest( + "select table0.time,table0.device,table0.level,table1.time,table1.device,table1.level from table0 asof join table1 on " + + "table0.device=table1.device and table1.level=table0.level and table0.time<=table1.time-1 " + + "order by table0.device,table0.level,table0.time,table1.device,table1.level", + expectedHeader, + retArray, + DATABASE_NAME); + } + @Test public void exceptionTest() { String errMsg = TSStatusCode.SEMANTIC_ERROR.getStatusCode() + ": " + ONLY_SUPPORT_EQUI_JOIN; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneJoinChildrenColumns.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneJoinChildrenColumns.java index 660231820670c..d52dca46efe83 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneJoinChildrenColumns.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneJoinChildrenColumns.java @@ -58,6 +58,11 @@ public Result apply(JoinNode joinNode, Captures captures, Context context) { .addAll(globallyUsableInputs) .addAll( joinNode.getCriteria().stream().map(JoinNode.EquiJoinClause::getLeft).iterator()) + .addAll( + joinNode + .getAsofCriteria() + .map(criteria -> ImmutableSet.of(criteria.getLeft())) + .orElse(ImmutableSet.of())) // .addAll(joinNode.getLeftHashSymbol().map(ImmutableSet::of).orElse(ImmutableSet.of())) .build(); @@ -66,6 +71,11 @@ public Result apply(JoinNode joinNode, Captures captures, Context context) { .addAll(globallyUsableInputs) .addAll( joinNode.getCriteria().stream().map(JoinNode.EquiJoinClause::getRight).iterator()) + .addAll( + joinNode + .getAsofCriteria() + .map(criteria -> ImmutableSet.of(criteria.getRight())) + .orElse(ImmutableSet.of())) // .addAll(joinNode.getRightHashSymbol().map(ImmutableSet::of).orElse(ImmutableSet.of())) .build(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/JoinNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/JoinNode.java index 50a67915dbbd9..3de9076865bd5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/JoinNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/JoinNode.java @@ -220,6 +220,16 @@ protected void serializeAttributes(ByteBuffer byteBuffer) { Symbol.serialize(equiJoinClause.getRight(), byteBuffer); } + if (asofCriteria.isPresent()) { + ReadWriteIOUtils.write(true, byteBuffer); + AsofJoinClause asofJoinClause = asofCriteria.get(); + ReadWriteIOUtils.write(asofJoinClause.getOperator().ordinal(), byteBuffer); + Symbol.serialize(asofJoinClause.getLeft(), byteBuffer); + Symbol.serialize(asofJoinClause.getRight(), byteBuffer); + } else { + ReadWriteIOUtils.write(false, byteBuffer); + } + ReadWriteIOUtils.write(leftOutputSymbols.size(), byteBuffer); for (Symbol leftOutputSymbol : leftOutputSymbols) { Symbol.serialize(leftOutputSymbol, byteBuffer); From 0fe34e54fd55af14409c425deb646b3982e97576 Mon Sep 17 00:00:00 2001 From: Weihao Li <18110526956@163.com> Date: Thu, 24 Apr 2025 22:02:32 +0800 Subject: [PATCH 07/11] fix Signed-off-by: Weihao Li <18110526956@163.com> --- .../plan/relational/analyzer/StatementAnalyzer.java | 10 +--------- .../plan/relational/sql/parser/AstBuilder.java | 3 ++- .../queryengine/plan/relational/analyzer/JoinTest.java | 5 +++++ 3 files changed, 8 insertions(+), 10 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java index f6b9d944f30cd..1f716c2012215 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java @@ -2974,19 +2974,11 @@ protected Scope visitJoin(Join node, Optional scope) { return analyzeJoinUsing(node, ((JoinUsing) criteria).getColumns(), scope, left, right); } - if (criteria instanceof AsofJoinOn) { - if (node.getType() != LEFT && node.getType() != INNER) { - throw new SemanticException( - String.format( - "ASOF [%s] JOIN is not supported, only support INNER and LEFT.", node.getType())); - } - } - Scope output = createAndAssignScope( node, scope, left.getRelationType().joinWith(right.getRelationType())); - if (node.getType() == RIGHT) { + if (node.getType() == LEFT || node.getType() == RIGHT) { throw new SemanticException( String.format( "%s JOIN is not supported, only support INNER JOIN in current version.", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java index 21cbd76a4f41a..3806edb805c6b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java @@ -42,6 +42,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AlterPipe; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ArithmeticBinaryExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ArithmeticUnaryExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AsofJoinOn; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BetweenPredicate; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BinaryLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BooleanLiteral; @@ -2213,7 +2214,7 @@ public Node visitJoinRelation(RelationalSqlParser.JoinRelationContext ctx) { joinType = Join.Type.INNER; } - if (joinType != Join.Type.INNER) { + if (criteria instanceof AsofJoinOn && joinType != Join.Type.INNER) { throw new SemanticException("ASOF JOIN is only support INNER type now"); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/JoinTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/JoinTest.java index 56708f246f773..f13500196c6c8 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/JoinTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/JoinTest.java @@ -624,6 +624,11 @@ public void otherInnerJoinTests() { // ========== unsupported test =============== @Test public void unsupportedJoinTest() { + // LEFT JOIN + assertAnalyzeSemanticException( + "SELECT * FROM table1 t1 LEFT JOIN table1 t2 ON t1.time=t2.time", + "LEFT JOIN is not supported, only support INNER JOIN in current version"); + // RIGHT JOIN assertAnalyzeSemanticException( "SELECT * FROM table1 t1 RIGHT JOIN table1 t2 ON t1.time=t2.time", From 4d8ad06498771d5047c69b491a093b8a90228752 Mon Sep 17 00:00:00 2001 From: Weihao Li <18110526956@163.com> Date: Fri, 25 Apr 2025 09:14:57 +0800 Subject: [PATCH 08/11] fix Signed-off-by: Weihao Li <18110526956@163.com> --- .../source/relational/AsofMergeSortInnerJoinOperator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AsofMergeSortInnerJoinOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AsofMergeSortInnerJoinOperator.java index b8fc88b3a2ce5..9d7acecf03ec2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AsofMergeSortInnerJoinOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AsofMergeSortInnerJoinOperator.java @@ -177,7 +177,7 @@ public boolean hasMatchedRightValueToProbeLeft() { int tmpIdx = rightIndex; boolean hasMatched = false; long matchedTime = Long.MIN_VALUE; - while (equalsTo( + while (nonAsofEquals( leftBlock, leftJoinKeyPositions, leftIndex, @@ -217,7 +217,7 @@ public boolean hasMatchedRightValueToProbeLeft() { return hasMatched; } - protected boolean equalsTo( + protected boolean nonAsofEquals( TsBlock leftBlock, int[] leftPositions, int lIndex, From 3b443918f6aaa85ccadd2d6021e0d33d8258e143 Mon Sep 17 00:00:00 2001 From: Weihao Li <18110526956@163.com> Date: Fri, 25 Apr 2025 16:53:32 +0800 Subject: [PATCH 09/11] modify according review & add more IT Signed-off-by: Weihao Li <18110526956@163.com> --- .../it/db/it/IoTDBAsofJoinTableIT.java | 146 ++++++++++++++++++ .../relational/sql/parser/AstBuilder.java | 21 +-- .../relational/grammar/sql/RelationalSql.g4 | 6 +- 3 files changed, 153 insertions(+), 20 deletions(-) create mode 100644 integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBAsofJoinTableIT.java diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBAsofJoinTableIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBAsofJoinTableIT.java new file mode 100644 index 0000000000000..10f510c50015c --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBAsofJoinTableIT.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.relational.it.db.it; + +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.TableClusterIT; +import org.apache.iotdb.itbase.category.TableLocalStandaloneIT; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.sql.Connection; +import java.sql.Statement; + +import static org.apache.iotdb.db.it.utils.TestUtils.tableResultSetEqualTest; +import static org.junit.Assert.fail; + +@RunWith(IoTDBTestRunner.class) +@Category({TableLocalStandaloneIT.class, TableClusterIT.class}) +public class IoTDBAsofJoinTableIT { + private static final String DATABASE_NAME = "test"; + + private static final String[] sql = + new String[] { + "create database test", + "use test", + "create table table1(device string tag, value int32 field)", + "insert into table1(time,device,value) values(2020-01-01 00:00:01.000,'d1',1)", + "insert into table1(time,device,value) values(2020-01-01 00:00:03.000,'d1',3)", + "insert into table1(time,device,value) values(2020-01-01 00:00:05.000,'d1',5)", + "insert into table1(time,device,value) values(2020-01-01 00:00:08.000,'d2',8)", + "create table table2(device string tag, value int32 field)", + "insert into table2(time,device,value) values(2020-01-01 00:00:02.000,'d1',20)", + "insert into table2(time,device,value) values(2020-01-01 00:00:03.000,'d1',30)", + "insert into table2(time,device,value) values(2020-01-01 00:00:04.000,'d2',40)", + "insert into table2(time,device,value) values(2020-01-01 00:00:05.000,'d2',50)" + }; + String[] expectedHeader = + new String[] {"time1", "device1", "value1", "time2", "device2", "value2"}; + ; + String[] retArray; + + @BeforeClass + public static void setUp() throws Exception { + EnvFactory.getEnv().initClusterEnvironment(); + EnvFactory.getEnv() + .getConfig() + .getCommonConfig() + .setMaxTsBlockLineNumber(2) + .setMaxNumberOfPointsInPage(5); + insertData(); + } + + @AfterClass + public static void tearDown() throws Exception { + EnvFactory.getEnv().cleanClusterEnvironment(); + } + + private static void insertData() { + try (Connection connection = EnvFactory.getEnv().getTableConnection(); + Statement statement = connection.createStatement()) { + for (String sql : sql) { + statement.execute(sql); + } + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void innerJoinTest() { + retArray = + new String[] { + "2020-01-01T00:00:03.000Z,d1,3,2020-01-01T00:00:03.000Z,d1,30,", + "2020-01-01T00:00:05.000Z,d1,5,2020-01-01T00:00:05.000Z,d2,50,", + "2020-01-01T00:00:08.000Z,d2,8,2020-01-01T00:00:05.000Z,d2,50," + }; + tableResultSetEqualTest( + "SELECT t1.time as time1, t1.device as device1, t1.value as value1, \n" + + " t2.time as time2, t2.device as device2, t2.value as value2 \n" + + "FROM \n" + + "table1 t1 ASOF INNER JOIN table2 t2\n" + + "ON\n" + + "t1.time>=t2.time\n" + + "order by time1", + expectedHeader, + retArray, + DATABASE_NAME); + + retArray = + new String[] { + "2020-01-01T00:00:03.000Z,d1,3,2020-01-01T00:00:03.000Z,d1,30,", + "2020-01-01T00:00:05.000Z,d1,5,2020-01-01T00:00:05.000Z,d2,50,", + }; + tableResultSetEqualTest( + "SELECT t1.time as time1, t1.device as device1, t1.value as value1, \n" + + " t2.time as time2, t2.device as device2, t2.value as value2 \n" + + "FROM \n" + + "table1 t1 ASOF(tolerance 2s) INNER JOIN table2 t2\n" + + "ON\n" + + "t1.time>=t2.time\n" + + "order by time1", + expectedHeader, + retArray, + DATABASE_NAME); + + retArray = + new String[] { + "2020-01-01T00:00:03.000Z,d1,3,2020-01-01T00:00:03.000Z,d1,30,", + "2020-01-01T00:00:05.000Z,d1,5,2020-01-01T00:00:03.000Z,d1,30,", + }; + tableResultSetEqualTest( + "SELECT t1.time as time1, t1.device as device1, t1.value as value1, \n" + + " t2.time as time2, t2.device as device2, t2.value as value2 \n" + + "FROM \n" + + "table1 t1 ASOF(tolerance 2s) INNER JOIN table2 t2\n" + + "ON\n" + + "t1.device=t2.device AND t1.time>=t2.time\n" + + "order by time1", + expectedHeader, + retArray, + DATABASE_NAME); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java index 3806edb805c6b..9d2db97afce96 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java @@ -2173,21 +2173,12 @@ public Node visitJoinRelation(RelationalSqlParser.JoinRelationContext ctx) { if (ctx.joinCriteria().ON() != null) { if (ctx.ASOF() != null) { TimeDuration timeDuration = null; - RelationalSqlParser.ToleranceParameterContext toleranceContext = ctx.toleranceParameter(); - if (toleranceContext != null) { - if (toleranceContext.timeDuration() != null) { - timeDuration = - DateTimeUtils.constructTimeDuration(toleranceContext.timeDuration().getText()); - - if (timeDuration.monthDuration != 0) { - throw new SemanticException( - "Month or year interval in tolerance is not supported now."); - } - } else if (toleranceContext.INTEGER_VALUE() != null) { - timeDuration = - new TimeDuration(0, parseLong(toleranceContext.INTEGER_VALUE().getText())); - } else { - throw new IllegalStateException("No time duration or time interval value appears!"); + if (ctx.timeDuration() != null) { + timeDuration = DateTimeUtils.constructTimeDuration(ctx.timeDuration().getText()); + + if (timeDuration.monthDuration != 0) { + throw new SemanticException( + "Month or year interval in tolerance is not supported now."); } } criteria = diff --git a/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 b/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 index 8adea05e9e862..829a7880754f1 100644 --- a/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 +++ b/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 @@ -899,7 +899,7 @@ relation ( CROSS JOIN right=aliasedRelation | joinType JOIN rightRelation=relation joinCriteria | NATURAL joinType JOIN right=aliasedRelation - | ASOF toleranceParameter? joinType JOIN rightRelation=relation joinCriteria + | ASOF ('(' TOLERANCE timeDuration ')')? joinType JOIN rightRelation=relation joinCriteria ) #joinRelation | aliasedRelation #relationDefault ; @@ -920,10 +920,6 @@ aliasedRelation : relationPrimary (AS? identifier columnAliases?)? ; -toleranceParameter - : '(' TOLERANCE (timeDuration | INTEGER_VALUE) ')' - ; - columnAliases : '(' identifier (',' identifier)* ')' ; From 21b200f75b1e9ecb3819a7a8a739562c5e011140 Mon Sep 17 00:00:00 2001 From: Weihao Li <18110526956@163.com> Date: Sun, 27 Apr 2025 10:06:41 +0800 Subject: [PATCH 10/11] refine corner case and name Signed-off-by: Weihao Li <18110526956@163.com> --- .../AsofMergeSortInnerJoinOperator.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AsofMergeSortInnerJoinOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AsofMergeSortInnerJoinOperator.java index 9d7acecf03ec2..5976217693499 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AsofMergeSortInnerJoinOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AsofMergeSortInnerJoinOperator.java @@ -60,6 +60,12 @@ public AsofMergeSortInnerJoinOperator( @Override protected boolean processFinished() { + // all the join keys in rightTsBlock are less or equal than leftTsBlock, just skip right + if (allRightLessOrEqualThanLeft()) { + resetRightBlockList(); + return true; + } + // skip all NULL values in left, because NULL value can not appear in the inner join result while (currentLeftHasNullValue()) { if (leftFinishedWithIncIndex()) { @@ -67,12 +73,6 @@ protected boolean processFinished() { } } - // all the join keys in rightTsBlock are less or equal than leftTsBlock, just skip right - if (allRightLessOrEqualThanLeft()) { - resetRightBlockList(); - return true; - } - // skip all NULL values in right, because NULL value can not appear in the inner join result while (currentRightHasNullValue()) { if (rightFinishedWithIncIndex()) { @@ -177,7 +177,7 @@ public boolean hasMatchedRightValueToProbeLeft() { int tmpIdx = rightIndex; boolean hasMatched = false; long matchedTime = Long.MIN_VALUE; - while (nonAsofEquals( + while (equalsIgnoreAsof( leftBlock, leftJoinKeyPositions, leftIndex, @@ -217,7 +217,7 @@ public boolean hasMatchedRightValueToProbeLeft() { return hasMatched; } - protected boolean nonAsofEquals( + protected boolean equalsIgnoreAsof( TsBlock leftBlock, int[] leftPositions, int lIndex, From 09c10a2e1a88a5daba6d4b0feafe6a3a70d955f9 Mon Sep 17 00:00:00 2001 From: Weihao Li <18110526956@163.com> Date: Tue, 29 Apr 2025 17:54:42 +0800 Subject: [PATCH 11/11] add UT of operator Signed-off-by: Weihao Li <18110526956@163.com> --- .../analyzer/AsofInnerJoinOperatorTest.java | 665 ++++++++++++++++++ 1 file changed, 665 insertions(+) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofInnerJoinOperatorTest.java diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofInnerJoinOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofInnerJoinOperatorTest.java new file mode 100644 index 0000000000000..de54d7faa0270 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofInnerJoinOperatorTest.java @@ -0,0 +1,665 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.analyzer; + +import org.apache.iotdb.db.queryengine.common.QueryId; +import org.apache.iotdb.db.queryengine.execution.driver.DriverContext; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.queryengine.execution.operator.Operator; +import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.execution.operator.process.join.merge.comparator.JoinKeyComparatorFactory; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.AsofMergeSortInnerJoinOperator; +import org.apache.iotdb.db.queryengine.plan.planner.memory.ThreadSafeMemoryReservationManager; + +import com.google.common.util.concurrent.ListenableFuture; +import io.airlift.units.Duration; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.apache.tsfile.read.common.type.BinaryType; +import org.apache.tsfile.utils.Binary; +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.Arrays; +import java.util.Collections; +import java.util.concurrent.TimeUnit; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.AbstractTableScanOperator.TIME_COLUMN_TEMPLATE; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +public class AsofInnerJoinOperatorTest { + private final Binary d1 = new Binary("d1".getBytes()); + private final Binary d2 = new Binary("d2".getBytes()); + + @Test + public void testAsofInnerJoin() { + // left table + // Time, device, s1 + // 4 d1, 4 + // 6 d1, 6 + // 9 d1, 9 + // ----------- TsBlock-1 + // 13 d1, 13 + // 17 d1, 17 + // ----------- TsBlock-2 + // 22 d2, 22 + // 25 d2, 25 + // ----------- TsBlock-3 + + // right table + // Time, device, s2 + // 1 d1, 10 + // 2 d1, 20 + // 3 d1, 30 + // ----------- TsBlock-1 + // 4 d1, 40 + // 5 d1, 50 + // 10 d1, 100 + // ----------- TsBlock-2 + // 13 d2, 130 + // 16 d2, 160 + // ----------- TsBlock-3 + // 26 d2, 260 + // 27 d2, 270 + // ----------- TsBlock-4 + + // join condition1 + // left.time>right.time + + // result table + // Time1,s1, Time2,s2 + // 25, 25, 16, 160 + // 22, 22, 16, 160 + // 17, 17, 16, 160 + // 13 13, 10, 100 + // 9, 9, 5, 50 + // 6, 6, 5, 50 + // 4, 4, 3, 30 + testExecute( + true, + false, + false, + new long[] {25, 22, 17, 13, 9, 6, 4}, + new int[] {25, 22, 17, 13, 9, 6, 4}, + new long[] {16, 16, 16, 10, 5, 5, 3}, + new int[] {160, 160, 160, 100, 50, 50, 30}, + new boolean[][] { + {false, false, false, false, false, false, false}, + {false, false, false, false, false, false, false} + }); + + // join condition2 + // left.time>=right.time + + // result table + // Time1,s1, Time2,s2 + // 25, 25, 16, 160 + // 22, 22, 16, 160 + // 17, 17, 16, 160 + // 13 13, 13, 130 + // 9, 9, 5, 50 + // 6, 6, 5, 50 + // 4, 4, 4, 40 + testExecute( + true, + true, + false, + new long[] {25, 22, 17, 13, 9, 6, 4}, + new int[] {25, 22, 17, 13, 9, 6, 4}, + new long[] {16, 16, 16, 13, 5, 5, 4}, + new int[] {160, 160, 160, 130, 50, 50, 40}, + new boolean[][] { + {false, false, false, false, false, false, false}, + {false, false, false, false, false, false, false} + }); + + // join condition3 + // left.timeright.time + + // result table + // result table + // Time1,s1, Time2,s2 + // 13 13, 10, 100 + // 9, 9, 5, 50 + // 6, 6, 5, 50 + // 4, 4, 3, 30 + // 25, 25, 16, 160 + // 22, 22, 16, 160 + testExecute( + true, + false, + true, + new long[] { + 17, 13, 9, 6, 4, 25, 22, + }, + new int[] { + 17, 13, 9, 6, 4, 25, 22, + }, + new long[] { + 10, 10, 5, 5, 3, 16, 16, + }, + new int[] { + 100, 100, 50, 50, 30, 160, 160, + }, + new boolean[][] { + {false, false, false, false, false, false, false}, + {false, false, false, false, false, false, false} + }); + + // join condition6 + // left.device=right.device and left.time>=right.time + + // result table + // 17, 17, 10, 100 + // 13 13, 10, 100 + // 9, 9, 5, 50 + // 6, 6, 5, 50 + // 4, 4, 4, 40 + // 25, 25, 16, 160 + // 22, 22, 16, 160 + testExecute( + true, + true, + true, + new long[] { + 17, 13, 9, 6, 4, 25, 22, + }, + new int[] { + 17, 13, 9, 6, 4, 25, 22, + }, + new long[] { + 10, 10, 5, 5, 4, 16, 16, + }, + new int[] { + 100, 100, 50, 50, 40, 160, 160, + }, + new boolean[][] { + {false, false, false, false, false, false, false}, + {false, false, false, false, false, false, false} + }); + + // join condition7 + // left.device=right.device and left.timeright.time: containsGreaterThan=true, ignoreEqual=false + // left.time<=right.time: containsGreaterThan=false, ignoreEqual=true + private void testExecute( + boolean containsGreaterThan, + boolean ignoreEqual, + boolean deviceEqual, + long[] time1Array, + int[] column1Array, + long[] time2Array, + int[] column2Array, + boolean[][] columnsIsNull) { + FragmentInstanceContext fragmentInstanceContext = Mockito.mock(FragmentInstanceContext.class); + Mockito.when(fragmentInstanceContext.getMemoryReservationContext()) + .thenReturn(new ThreadSafeMemoryReservationManager(new QueryId("1"), "test")); + DriverContext driverContext = Mockito.mock(DriverContext.class); + Mockito.when(driverContext.getFragmentInstanceContext()).thenReturn(fragmentInstanceContext); + OperatorContext operatorContext = Mockito.mock(OperatorContext.class); + Mockito.when(operatorContext.getMaxRunTime()).thenReturn(new Duration(1, TimeUnit.SECONDS)); + Mockito.when(operatorContext.getDriverContext()).thenReturn(driverContext); + + Operator leftChild = + new Operator() { + private final long[][] timeArray = + containsGreaterThan + ? (deviceEqual + ? new long[][] { + {17L, 13L}, + {9L, 6L, 4L}, + {25L, 22L} + } + : new long[][] { + {25L, 22L}, + {17L, 13L}, + {9L, 6L, 4L} + }) + : new long[][] { + {4L, 6L, 9L}, + {13L, 17L}, + {22L, 25L} + }; + + private final Binary[][] deviceArray = + containsGreaterThan + ? (deviceEqual + ? new Binary[][] { + new Binary[] {d1, d1}, + new Binary[] {d1, d1, d1}, + new Binary[] {d2, d2} + } + : new Binary[][] { + new Binary[] {d2, d2}, new Binary[] {d1, d1}, new Binary[] {d1, d1, d1} + }) + : new Binary[][] { + new Binary[] {d1, d1, d1}, + new Binary[] {d1, d1}, + new Binary[] {d2, d2}, + }; + + private final int[][] valueArray = + containsGreaterThan + ? (deviceEqual + ? new int[][] { + {17, 13}, {9, 6, 4}, {25, 22}, + } + : new int[][] { + {25, 22}, {17, 13}, {9, 6, 4}, + }) + : new int[][] { + {4, 6, 9}, + {13, 17}, + {22, 25} + }; + + private final boolean[][][] valueIsNull = + containsGreaterThan + ? (deviceEqual + ? new boolean[][][] { + { + {false, false}, + {false, false, false}, + {false, false} + } + } + : new boolean[][][] { + { + {false, false}, + {false, false}, + {false, false, false} + } + }) + : new boolean[][][] { + { + {false, false, false}, + {false, false}, + {false, false} + } + }; + + private int index = 0; + + @Override + public OperatorContext getOperatorContext() { + return operatorContext; + } + + @Override + public TsBlock next() { + TsBlockBuilder builder = + new TsBlockBuilder( + timeArray[index].length, + Arrays.asList(TSDataType.TIMESTAMP, TSDataType.TEXT, TSDataType.INT32)); + for (int i = 0, size = timeArray[index].length; i < size; i++) { + builder.getColumnBuilder(0).writeLong(timeArray[index][i]); + builder.getColumnBuilder(1).writeBinary(deviceArray[index][i]); + if (valueIsNull[0][index][i]) { + builder.getColumnBuilder(2).appendNull(); + } else { + builder.getColumnBuilder(2).writeInt(valueArray[index][i]); + } + } + builder.declarePositions(timeArray[index].length); + return builder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, timeArray[index++].length)); + } + + @Override + public boolean hasNext() { + return index < 3; + } + + @Override + public void close() {} + + @Override + public boolean isFinished() { + return index >= 3; + } + + @Override + public long calculateMaxPeekMemory() { + return 64 * 1024; + } + + @Override + public long calculateMaxReturnSize() { + return 64 * 1024; + } + + @Override + public long calculateRetainedSizeAfterCallingNext() { + return 0; + } + + @Override + public long ramBytesUsed() { + return 0; + } + }; + + Operator rightChild = + new Operator() { + private final long[][] timeArray = + containsGreaterThan + ? (deviceEqual + ? new long[][] { + {10L, 5L, 4L}, + {3L, 2L, 1L}, + {27L, 26L}, + {16L, 13L}, + } + : new long[][] { + {27L, 26L}, + {16L, 13L}, + {10L, 5L, 4L}, + {3L, 2L, 1L}, + }) + : new long[][] { + {1L, 2L, 3L}, + {4L, 5L, 10L}, + {13L, 16L}, + {26L, 27L} + }; + + private final Binary[][] deviceArray = + containsGreaterThan && !deviceEqual + ? new Binary[][] { + new Binary[] {d2, d2}, + new Binary[] {d2, d2}, + new Binary[] {d1, d1, d1}, + new Binary[] {d1, d1, d1} + } + : new Binary[][] { + new Binary[] {d1, d1, d1}, + new Binary[] {d1, d1, d1}, + new Binary[] {d2, d2}, + new Binary[] {d2, d2} + }; + + private final int[][] valueArray = + containsGreaterThan + ? (deviceEqual + ? new int[][] { + {100, 50, 40}, + {30, 20, 10}, + {270, 260}, + {160, 130}, + } + : new int[][] { + {270, 260}, + {160, 130}, + {100, 50, 40}, + {30, 20, 10}, + }) + : new int[][] { + {10, 20, 30}, + {40, 50, 100}, + {130, 160}, + {260, 270} + }; + + private final boolean[][][] valueIsNull = + containsGreaterThan && !deviceEqual + ? new boolean[][][] { + { + {false, false}, + {false, false}, + {false, false, false}, + {false, false, false} + } + } + : new boolean[][][] { + { + {false, false, false}, + {false, false, false}, + {false, false}, + {false, false} + } + }; + + private int index = 0; + + @Override + public OperatorContext getOperatorContext() { + return operatorContext; + } + + @Override + public TsBlock next() { + TsBlockBuilder builder = + new TsBlockBuilder( + timeArray[index].length, + Arrays.asList(TSDataType.TIMESTAMP, TSDataType.TEXT, TSDataType.INT32)); + for (int i = 0, size = timeArray[index].length; i < size; i++) { + builder.getColumnBuilder(0).writeLong(timeArray[index][i]); + builder.getColumnBuilder(1).writeBinary(deviceArray[index][i]); + if (valueIsNull[0][index][i]) { + builder.getColumnBuilder(2).appendNull(); + } else { + builder.getColumnBuilder(2).writeInt(valueArray[index][i]); + } + } + builder.declarePositions(timeArray[index].length); + return builder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, timeArray[index++].length)); + } + + @Override + public boolean hasNext() { + return index < 4; + } + + @Override + public void close() {} + + @Override + public boolean isFinished() { + return index >= 4; + } + + @Override + public long calculateMaxPeekMemory() { + return 64 * 1024; + } + + @Override + public long calculateMaxReturnSize() { + return 64 * 1024; + } + + @Override + public long calculateRetainedSizeAfterCallingNext() { + return 0; + } + + @Override + public long ramBytesUsed() { + return 0; + } + }; + + AsofMergeSortInnerJoinOperator asofInnerJoinOperator = + new AsofMergeSortInnerJoinOperator( + operatorContext, + leftChild, + deviceEqual ? new int[] {1, 0} : new int[] {0}, + new int[] {0, 2}, + rightChild, + deviceEqual ? new int[] {1, 0} : new int[] {0}, + new int[] {0, 2}, + JoinKeyComparatorFactory.getAsofComparators( + deviceEqual ? Collections.singletonList(BinaryType.TEXT) : Collections.emptyList(), + ignoreEqual, + !containsGreaterThan), + Arrays.asList( + TSDataType.TIMESTAMP, TSDataType.INT32, TSDataType.TIMESTAMP, TSDataType.INT32)); + + try { + int count = 0; + ListenableFuture listenableFuture = asofInnerJoinOperator.isBlocked(); + listenableFuture.get(); + while (!asofInnerJoinOperator.isFinished() && asofInnerJoinOperator.hasNext()) { + TsBlock tsBlock = asofInnerJoinOperator.next(); + if (tsBlock != null && !tsBlock.isEmpty()) { + for (int i = 0, size = tsBlock.getPositionCount(); i < size; i++, count++) { + assertEquals(time1Array[count], tsBlock.getColumn(0).getLong(i)); + assertEquals(columnsIsNull[0][count], tsBlock.getColumn(1).isNull(i)); + if (!columnsIsNull[1][count]) { + assertEquals(column1Array[count], tsBlock.getColumn(1).getInt(i)); + } + + assertEquals(time2Array[count], tsBlock.getColumn(2).getLong(i)); + assertEquals(columnsIsNull[1][count], tsBlock.getColumn(3).isNull(i)); + if (!columnsIsNull[1][count]) { + assertEquals(column2Array[count], tsBlock.getColumn(3).getLong(i)); + } + } + } + listenableFuture = asofInnerJoinOperator.isBlocked(); + listenableFuture.get(); + } + assertEquals(time1Array.length, count); + assertEquals(time2Array.length, count); + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } +}