-
Notifications
You must be signed in to change notification settings - Fork 13.5k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-29138][table-planner] fix project can not be pushed into looku…
…p source This closes #20729
- Loading branch information
1 parent
d9ac4c7
commit 54b6b69
Showing
9 changed files
with
517 additions
and
28 deletions.
There are no files selected for viewing
77 changes: 77 additions & 0 deletions
77
.../java/org/apache/flink/table/planner/plan/rules/logical/ProjectSnapshotTransposeRule.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,77 @@ | ||
/* | ||
* 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.calcite.plan.RelOptRule; | ||
import org.apache.calcite.plan.RelOptRuleCall; | ||
import org.apache.calcite.plan.RelRule; | ||
import org.apache.calcite.rel.RelNode; | ||
import org.apache.calcite.rel.logical.LogicalProject; | ||
import org.apache.calcite.rel.logical.LogicalSnapshot; | ||
|
||
/** Transpose {@link LogicalProject} past into {@link LogicalSnapshot}. */ | ||
public class ProjectSnapshotTransposeRule extends RelRule<ProjectSnapshotTransposeRule.Config> { | ||
|
||
public static final RelOptRule INSTANCE = | ||
ProjectSnapshotTransposeRule.Config.EMPTY.as(Config.class).withOperator().toRule(); | ||
|
||
public ProjectSnapshotTransposeRule(Config config) { | ||
super(config); | ||
} | ||
|
||
@Override | ||
public boolean matches(RelOptRuleCall call) { | ||
LogicalProject project = call.rel(0); | ||
// Don't push a project which contains over into a snapshot, snapshot on window aggregate is | ||
// unsupported for now. | ||
return !project.containsOver(); | ||
} | ||
|
||
@Override | ||
public void onMatch(RelOptRuleCall call) { | ||
LogicalProject project = call.rel(0); | ||
LogicalSnapshot snapshot = call.rel(1); | ||
RelNode newProject = project.copy(project.getTraitSet(), snapshot.getInputs()); | ||
RelNode newSnapshot = | ||
snapshot.copy(snapshot.getTraitSet(), newProject, snapshot.getPeriod()); | ||
call.transformTo(newSnapshot); | ||
} | ||
|
||
/** Configuration for {@link ProjectSnapshotTransposeRule}. */ | ||
public interface Config extends RelRule.Config { | ||
|
||
@Override | ||
default RelOptRule toRule() { | ||
return new ProjectSnapshotTransposeRule(this); | ||
} | ||
|
||
default ProjectSnapshotTransposeRule.Config withOperator() { | ||
final RelRule.OperandTransform snapshotTransform = | ||
operandBuilder -> operandBuilder.operand(LogicalSnapshot.class).noInputs(); | ||
|
||
final RelRule.OperandTransform projectTransform = | ||
operandBuilder -> | ||
operandBuilder | ||
.operand(LogicalProject.class) | ||
.oneInput(snapshotTransform); | ||
|
||
return withOperandSupplier(projectTransform).as(Config.class); | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
136 changes: 136 additions & 0 deletions
136
...a/org/apache/flink/table/planner/plan/rules/logical/ProjectSnapshotTransposeRuleTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,136 @@ | ||
/* | ||
* 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.TableConfig; | ||
import org.apache.flink.table.api.TableEnvironment; | ||
import org.apache.flink.table.planner.plan.optimize.program.FlinkBatchProgram; | ||
import org.apache.flink.table.planner.plan.optimize.program.FlinkStreamProgram; | ||
import org.apache.flink.table.planner.utils.BatchTableTestUtil; | ||
import org.apache.flink.table.planner.utils.StreamTableTestUtil; | ||
import org.apache.flink.table.planner.utils.TableTestBase; | ||
import org.apache.flink.table.planner.utils.TableTestUtil; | ||
|
||
import org.junit.Before; | ||
import org.junit.Test; | ||
import org.junit.runner.RunWith; | ||
import org.junit.runners.Parameterized; | ||
|
||
import java.util.Arrays; | ||
import java.util.Collection; | ||
|
||
/** Test rule {@link ProjectSnapshotTransposeRule}. */ | ||
@RunWith(Parameterized.class) | ||
public class ProjectSnapshotTransposeRuleTest extends TableTestBase { | ||
|
||
private static final String STREAM = "stream"; | ||
private static final String BATCH = "batch"; | ||
|
||
@Parameterized.Parameter public String mode; | ||
|
||
@Parameterized.Parameters(name = "mode = {0}") | ||
public static Collection<String> parameters() { | ||
return Arrays.asList(STREAM, BATCH); | ||
} | ||
|
||
private TableTestUtil util; | ||
|
||
@Before | ||
public void setup() { | ||
boolean isStreaming = STREAM.equals(mode); | ||
if (isStreaming) { | ||
util = streamTestUtil(TableConfig.getDefault()); | ||
((StreamTableTestUtil) util).buildStreamProgram(FlinkStreamProgram.LOGICAL_REWRITE()); | ||
} else { | ||
util = batchTestUtil(TableConfig.getDefault()); | ||
((BatchTableTestUtil) util).buildBatchProgram(FlinkBatchProgram.LOGICAL_REWRITE()); | ||
} | ||
|
||
TableEnvironment tEnv = util.getTableEnv(); | ||
String src = | ||
String.format( | ||
"CREATE TABLE MyTable (\n" | ||
+ " a int,\n" | ||
+ " b varchar,\n" | ||
+ " c bigint,\n" | ||
+ " proctime as PROCTIME(),\n" | ||
+ " rowtime as TO_TIMESTAMP(FROM_UNIXTIME(c)),\n" | ||
+ " watermark for rowtime as rowtime - INTERVAL '1' second \n" | ||
+ ") with (\n" | ||
+ " 'connector' = 'values',\n" | ||
+ " 'bounded' = '%s')", | ||
!isStreaming); | ||
String lookup = | ||
String.format( | ||
"CREATE TABLE LookupTable (\n" | ||
+ " id int,\n" | ||
+ " name varchar,\n" | ||
+ " age int \n" | ||
+ ") with (\n" | ||
+ " 'connector' = 'values',\n" | ||
+ " 'bounded' = '%s')", | ||
!isStreaming); | ||
tEnv.executeSql(src); | ||
tEnv.executeSql(lookup); | ||
} | ||
|
||
@Test | ||
public void testJoinTemporalTableWithProjectionPushDown() { | ||
String sql = | ||
"SELECT T.*, D.id\n" | ||
+ "FROM MyTable AS T\n" | ||
+ "JOIN LookupTable FOR SYSTEM_TIME AS OF T.proctime AS D\n" | ||
+ "ON T.a = D.id"; | ||
|
||
util.verifyRelPlan(sql); | ||
} | ||
|
||
@Test | ||
public void testJoinTemporalTableNotProjectable() { | ||
String sql = | ||
"SELECT T.*, D.*\n" | ||
+ "FROM MyTable AS T\n" | ||
+ "JOIN LookupTable FOR SYSTEM_TIME AS OF T.proctime AS D\n" | ||
+ "ON T.a = D.id"; | ||
|
||
util.verifyRelPlan(sql); | ||
} | ||
|
||
@Test | ||
public void testJoinTemporalTableWithReorderedProject() { | ||
String sql = | ||
"SELECT T.*, D.age, D.name, D.id\n" | ||
+ "FROM MyTable AS T\n" | ||
+ "JOIN LookupTable FOR SYSTEM_TIME AS OF T.proctime AS D\n" | ||
+ "ON T.a = D.id"; | ||
|
||
util.verifyRelPlan(sql); | ||
} | ||
|
||
@Test | ||
public void testJoinTemporalTableWithProjectAndFilter() { | ||
String sql = | ||
"SELECT T.*, D.id\n" | ||
+ "FROM MyTable AS T\n" | ||
+ "JOIN LookupTable FOR SYSTEM_TIME AS OF T.proctime AS D\n" | ||
+ "ON T.a = D.id WHERE D.age > 20"; | ||
|
||
util.verifyRelPlan(sql); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.