Re: [PR] [FLINK-34162][table] Migrate LogicalUnnestRule to java [flink]

2024-04-08 Thread via GitHub


snuyanzin merged PR #24144:
URL: https://github.com/apache/flink/pull/24144


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-34162][table] Migrate LogicalUnnestRule to java [flink]

2024-04-03 Thread via GitHub


snuyanzin commented on code in PR #24144:
URL: https://github.com/apache/flink/pull/24144#discussion_r1549420987


##
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java:
##
@@ -0,0 +1,180 @@
+/*
+ * 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.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.functions.table.UnnestRowsFunction;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import org.apache.flink.shaded.guava32.com.google.common.collect.ImmutableList;

Review Comment:
   I'm going to rebase the PR to be on the safe side and be sure that tests are 
assing



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-34162][table] Migrate LogicalUnnestRule to java [flink]

2024-04-03 Thread via GitHub


snuyanzin commented on code in PR #24144:
URL: https://github.com/apache/flink/pull/24144#discussion_r1549420099


##
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java:
##
@@ -0,0 +1,180 @@
+/*
+ * 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.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.functions.table.UnnestRowsFunction;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import org.apache.flink.shaded.guava32.com.google.common.collect.ImmutableList;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Correlate;
+import org.apache.calcite.rel.core.Uncollect;
+import org.apache.calcite.rel.logical.LogicalCorrelate;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.immutables.value.Value;
+
+import java.util.Collections;
+import java.util.Map;
+
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeUtils.toRowType;
+
+/**
+ * Planner rule that rewrites UNNEST to explode function.
+ *
+ * Note: This class can only be used in HepPlanner.
+ */
+@Value.Enclosing
+public class LogicalUnnestRule extends 
RelRule {
+
+public static final LogicalUnnestRule INSTANCE = 
LogicalUnnestRuleConfig.DEFAULT.toRule();
+
+public LogicalUnnestRule(LogicalUnnestRule.LogicalUnnestRuleConfig config) 
{
+super(config);
+}
+
+public boolean matches(RelOptRuleCall call) {
+LogicalCorrelate join = call.rel(0);
+RelNode right = getRel(join.getRight());
+if (right instanceof LogicalFilter) {
+LogicalFilter logicalFilter = (LogicalFilter) right;
+RelNode relNode = getRel(logicalFilter.getInput());
+if (relNode instanceof Uncollect) {
+return !((Uncollect) relNode).withOrdinality;
+} else if (relNode instanceof LogicalProject) {
+LogicalProject logicalProject = (LogicalProject) relNode;
+relNode = getRel(logicalProject.getInput());
+if (relNode instanceof Uncollect) {
+return !((Uncollect) relNode).withOrdinality;
+}
+return false;
+}
+} else if (right instanceof LogicalProject) {
+LogicalProject logicalProject = (LogicalProject) right;
+RelNode relNode = getRel(logicalProject.getInput());
+if (relNode instanceof Uncollect) {
+Uncollect uncollect = (Uncollect) relNode;
+return !uncollect.withOrdinality;
+}
+return false;
+} else if (right instanceof Uncollect) {
+Uncollect uncollect = (Uncollect) right;
+return !uncollect.withOrdinality;
+}
+return false;
+}
+
+public void onMatch(RelOptRuleCall call) {
+LogicalCorrelate correlate = call.rel(0);
+RelNode outer = getRel(correlate.getLeft());
+RelNode array = getRel(correlate.getRight());
+
+// convert unnest into table function scan
+RelNode tableFunctionScan = convert(array, correlate);
+// create correlate with table function scan as input
+Correlate newCorrelate =
+correlate.copy(correlate.getTraitSet(), 
ImmutableList.of(outer, tableFunctionScan));
+call.transformTo(newCorrelate);
+}
+
+private RelNode convert(RelNode relNode, LogicalCorrelate correlate) {
+if (relNode instanceof HepRelVertex) {
+  

Re: [PR] [FLINK-34162][table] Migrate LogicalUnnestRule to java [flink]

2024-04-02 Thread via GitHub


RyanSkraba commented on code in PR #24144:
URL: https://github.com/apache/flink/pull/24144#discussion_r1547947032


##
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java:
##
@@ -0,0 +1,180 @@
+/*
+ * 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.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.functions.table.UnnestRowsFunction;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import org.apache.flink.shaded.guava32.com.google.common.collect.ImmutableList;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Correlate;
+import org.apache.calcite.rel.core.Uncollect;
+import org.apache.calcite.rel.logical.LogicalCorrelate;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.immutables.value.Value;
+
+import java.util.Collections;
+import java.util.Map;
+
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeUtils.toRowType;
+
+/**
+ * Planner rule that rewrites UNNEST to explode function.
+ *
+ * Note: This class can only be used in HepPlanner.
+ */
+@Value.Enclosing
+public class LogicalUnnestRule extends 
RelRule {
+
+public static final LogicalUnnestRule INSTANCE = 
LogicalUnnestRuleConfig.DEFAULT.toRule();
+
+public LogicalUnnestRule(LogicalUnnestRule.LogicalUnnestRuleConfig config) 
{
+super(config);
+}
+
+public boolean matches(RelOptRuleCall call) {
+LogicalCorrelate join = call.rel(0);
+RelNode right = getRel(join.getRight());
+if (right instanceof LogicalFilter) {
+LogicalFilter logicalFilter = (LogicalFilter) right;
+RelNode relNode = getRel(logicalFilter.getInput());
+if (relNode instanceof Uncollect) {
+return !((Uncollect) relNode).withOrdinality;
+} else if (relNode instanceof LogicalProject) {
+LogicalProject logicalProject = (LogicalProject) relNode;
+relNode = getRel(logicalProject.getInput());
+if (relNode instanceof Uncollect) {
+return !((Uncollect) relNode).withOrdinality;
+}
+return false;
+}
+} else if (right instanceof LogicalProject) {
+LogicalProject logicalProject = (LogicalProject) right;
+RelNode relNode = getRel(logicalProject.getInput());
+if (relNode instanceof Uncollect) {
+Uncollect uncollect = (Uncollect) relNode;
+return !uncollect.withOrdinality;
+}
+return false;
+} else if (right instanceof Uncollect) {
+Uncollect uncollect = (Uncollect) right;
+return !uncollect.withOrdinality;
+}
+return false;
+}
+
+public void onMatch(RelOptRuleCall call) {
+LogicalCorrelate correlate = call.rel(0);
+RelNode outer = getRel(correlate.getLeft());
+RelNode array = getRel(correlate.getRight());
+
+// convert unnest into table function scan
+RelNode tableFunctionScan = convert(array, correlate);
+// create correlate with table function scan as input
+Correlate newCorrelate =
+correlate.copy(correlate.getTraitSet(), 
ImmutableList.of(outer, tableFunctionScan));
+call.transformTo(newCorrelate);
+}
+
+private RelNode convert(RelNode relNode, LogicalCorrelate correlate) {
+if (relNode instanceof HepRelVertex) {
+ 

Re: [PR] [FLINK-34162][table] Migrate LogicalUnnestRule to java [flink]

2024-01-18 Thread via GitHub


flinkbot commented on PR #24144:
URL: https://github.com/apache/flink/pull/24144#issuecomment-1899370307

   
   ## CI report:
   
   * fb9cb9e1dbcc55e715f9345e7a676f04dd3aab44 UNKNOWN
   
   
   Bot commands
 The @flinkbot bot supports the following commands:
   
- `@flinkbot run azure` re-run the last Azure build
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[PR] [FLINK-34162][table] Migrate LogicalUnnestRule to java [flink]

2024-01-18 Thread via GitHub


snuyanzin opened a new pull request, #24144:
URL: https://github.com/apache/flink/pull/24144

   
   ## What is the purpose of the change
   
   The PR migrates `LogicalUnnestRule` to java
   it doesn't touch `LogicalUnnestRuleTest` to be sure that java version 
continues passing it
   
   
   ## Verifying this change
   
   This change is already covered by existing tests
   ## Does this pull request potentially affect one of the following parts:
   
 - Dependencies (does it add or upgrade a dependency): ( no)
 - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: ( no)
 - The serializers: ( no)
 - The runtime per-record code paths (performance sensitive): ( no)
 - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Kubernetes/Yarn, ZooKeeper: ( no )
 - The S3 file system connector: ( no )
   
   ## Documentation
   
 - Does this pull request introduce a new feature? ( no)
 - If yes, how is the feature documented? (not applicable )
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org