Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,113 @@
/*
* 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.flink.table.planner.plan.rules.logical;

import org.apache.flink.table.api.TableException;

import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.plan.RelRule;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.logical.LogicalCorrelate;
import org.apache.calcite.rel.logical.LogicalFilter;
import org.apache.calcite.rel.logical.LogicalSnapshot;
import org.immutables.value.Value;

/**
* Rules that reject temporal table joins ({@code FOR SYSTEM_TIME AS OF}) in batch mode with a clear
* error message.
*
* <p>In the batch {@code EXPAND_PLAN_RULES}, lookup join rules run first and rewrite valid lookup
* joins (processing-time + {@code LookupTableSource}) into {@code TemporalJoin} nodes. Any
* remaining {@link LogicalCorrelate} + {@link LogicalSnapshot} pattern therefore represents an
* unsupported temporal join. These rules catch it and throw a {@link TableException} rather than
* letting the correlate survive into {@code FlinkDecorrelateProgram}, where it would cause a
* confusing "unexpected correlate variable" internal error.
*/
@Value.Enclosing
public class RejectTemporalJoinInBatchRule
extends RelRule<RejectTemporalJoinInBatchRule.RejectTemporalJoinInBatchRuleConfig> {

private static final String MESSAGE =
"Temporal joins (FOR SYSTEM_TIME AS OF) on regular tables are not supported in "
+ "batch mode. Use a lookup join or switch to streaming mode.";

/**
* Matches temporal joins where the right side of the Correlate is a Filter wrapping a Snapshot
* (non-trivial join condition).
*/
public static final RejectTemporalJoinInBatchRule WITH_FILTER =
RejectTemporalJoinInBatchRuleConfig.WITH_FILTER.toRule();

/**
* Matches temporal joins where the right side of the Correlate is a Snapshot directly (trivial
* join condition).
*/
public static final RejectTemporalJoinInBatchRule WITHOUT_FILTER =
RejectTemporalJoinInBatchRuleConfig.WITHOUT_FILTER.toRule();

private RejectTemporalJoinInBatchRule(RejectTemporalJoinInBatchRuleConfig config) {
super(config);
}

@Override
public void onMatch(RelOptRuleCall call) {
throw new TableException(MESSAGE);
}

/** Rule configuration. */
@Value.Immutable(singleton = false)
public interface RejectTemporalJoinInBatchRuleConfig extends RelRule.Config {

RejectTemporalJoinInBatchRuleConfig WITH_FILTER =
ImmutableRejectTemporalJoinInBatchRule.RejectTemporalJoinInBatchRuleConfig.builder()
.operandSupplier(
b0 ->
b0.operand(LogicalCorrelate.class)
.inputs(
b1 -> b1.operand(RelNode.class).anyInputs(),
b2 ->
b2.operand(LogicalFilter.class)
.oneInput(
b3 ->
b3.operand(
LogicalSnapshot
.class)
.anyInputs())))
.description("RejectTemporalJoinInBatchRule_WithFilter")
.build();

RejectTemporalJoinInBatchRuleConfig WITHOUT_FILTER =
ImmutableRejectTemporalJoinInBatchRule.RejectTemporalJoinInBatchRuleConfig.builder()
.operandSupplier(
b0 ->
b0.operand(LogicalCorrelate.class)
.inputs(
b1 -> b1.operand(RelNode.class).anyInputs(),
b2 ->
b2.operand(LogicalSnapshot.class)
.anyInputs()))
.description("RejectTemporalJoinInBatchRule_WithoutFilter")
.build();

@Override
default RejectTemporalJoinInBatchRule toRule() {
return new RejectTemporalJoinInBatchRule(this);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,10 +50,16 @@ object FlinkBatchRuleSets {
/**
* Expand plan by replacing references to tables into a proper plan sub trees. Those rules can
* create new plan nodes.
*
* The lookup join rules run first and rewrite supported temporal joins. The rejection rules then
* catch any remaining temporal joins (unsupported in batch) with a clear error message.
*/
val EXPAND_PLAN_RULES: RuleSet = RuleSets.ofList(
LogicalCorrelateToJoinFromTemporalTableRule.LOOKUP_JOIN_WITH_FILTER,
LogicalCorrelateToJoinFromTemporalTableRule.LOOKUP_JOIN_WITHOUT_FILTER)
LogicalCorrelateToJoinFromTemporalTableRule.LOOKUP_JOIN_WITHOUT_FILTER,
RejectTemporalJoinInBatchRule.WITH_FILTER,
RejectTemporalJoinInBatchRule.WITHOUT_FILTER
)

val POST_EXPAND_CLEAN_UP_RULES: RuleSet = RuleSets.ofList(EnumerableToLogicalTableScan.INSTANCE)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,11 @@ import org.assertj.core.api.Assertions.assertThatExceptionOfType
import org.junit.jupiter.api.{BeforeEach, Test}

/**
* Test temporal join in batch mode.
* Tests that temporal joins on non-lookup tables are rejected in batch mode with a clear error.
*
* <p> Flink only supports lookup join in batch mode, the others Temporal join is not supported yet.
* <p>Batch mode only supports lookup joins. General temporal joins (event-time or processing-time
* on non-lookup sources) are caught by [[RejectTemporalJoinInBatchRule]] during the
* TEMPORAL_JOIN_REWRITE phase.
*/
class TemporalJoinTest extends TableTestBase {

Expand Down Expand Up @@ -105,6 +107,10 @@ class TemporalJoinTest extends TableTestBase {
"GROUP BY currency ")
}

private val expectedMessage =
"Temporal joins (FOR SYSTEM_TIME AS OF) on regular tables are not supported in " +
"batch mode. Use a lookup join or switch to streaming mode."

@Test
def testSimpleJoin(): Unit = {
val sqlQuery = "SELECT " +
Expand All @@ -115,6 +121,7 @@ class TemporalJoinTest extends TableTestBase {

assertThatExceptionOfType(classOf[TableException])
.isThrownBy(() => util.verifyExecPlan(sqlQuery))
.withMessage(expectedMessage)
}

@Test
Expand All @@ -128,6 +135,7 @@ class TemporalJoinTest extends TableTestBase {

assertThatExceptionOfType(classOf[TableException])
.isThrownBy(() => util.verifyExecPlan(sqlQuery))
.withMessage(expectedMessage)
}

@Test
Expand All @@ -141,11 +149,21 @@ class TemporalJoinTest extends TableTestBase {

assertThatExceptionOfType(classOf[TableException])
.isThrownBy(() => util.verifyExecPlan(sqlQuery))
.withMessage(expectedMessage)
}

@Test
def testSimpleViewProcTimeJoin(): Unit = {
def testSimpleJoinOnTrue(): Unit = {
val sqlQuery = "SELECT o_amount FROM Orders AS o JOIN " +
"RatesHistoryWithPK FOR SYSTEM_TIME AS OF o.o_rowtime AS r ON TRUE"

assertThatExceptionOfType(classOf[TableException])
.isThrownBy(() => util.verifyExecPlan(sqlQuery))
.withMessage(expectedMessage)
}

@Test
def testSimpleViewProcTimeJoin(): Unit = {
val sqlQuery = "SELECT " +
"o_amount * rate as rate " +
"FROM Orders AS o JOIN " +
Expand All @@ -155,5 +173,6 @@ class TemporalJoinTest extends TableTestBase {

assertThatExceptionOfType(classOf[TableException])
.isThrownBy(() => util.verifyExecPlan(sqlQuery))
.withMessage(expectedMessage)
Comment thread
jnh5y marked this conversation as resolved.
}
}