You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/06/01 12:53:12 UTC

[GitHub] [flink] godfreyhe commented on a diff in pull request #18408: [FLINK-24865][CEP] Support MATCH_RECOGNIZE in Batch mode

godfreyhe commented on code in PR #18408:
URL: https://github.com/apache/flink/pull/18408#discussion_r886754006


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecMatch.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * 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.nodes.exec.common;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.cep.EventComparator;
+import org.apache.flink.cep.nfa.aftermatch.AfterMatchSkipStrategy;
+import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.cep.operator.CepOperator;
+import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.Quantifier;
+import org.apache.flink.cep.pattern.conditions.BooleanConditions;
+import org.apache.flink.cep.pattern.conditions.IterativeCondition;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.codegen.CodeGenUtils;
+import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
+import org.apache.flink.table.planner.codegen.MatchCodeGenerator;
+import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecMatch;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.MatchSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
+import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.planner.plan.utils.RexDefaultVisitor;
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import org.apache.flink.table.runtime.operators.match.PatternProcessFunctionRunner;
+import org.apache.flink.table.runtime.operators.match.RowDataEventComparator;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.MathUtils;
+
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlMatchRecognize;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.tools.RelBuilder;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Common {@link ExecNode} which matches along with MATCH_RECOGNIZE. */
+public class CommonExecMatch extends ExecNodeBase<RowData>
+        implements ExecNode<RowData>, MultipleTransformationTranslator<RowData> {
+
+    public static final String MATCH_TRANSFORMATION = "match";
+
+    private final MatchSpec matchSpec;
+
+    public CommonExecMatch(
+            ReadableConfig tableConfig,
+            MatchSpec matchSpec,
+            InputProperty inputProperty,
+            RowType outputType,
+            String description) {
+        this(
+                ExecNodeContext.newNodeId(),
+                ExecNodeContext.newContext(BatchExecMatch.class),
+                ExecNodeContext.newPersistedConfig(BatchExecMatch.class, tableConfig),
+                matchSpec,
+                Collections.singletonList(inputProperty),
+                outputType,
+                description);
+    }

Review Comment:
   this contractor can be removed.
   
   note: BatchExecMatch.class should not be here
   



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/batch/sql/validation/MatchRecognizeValidationTest.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.api.batch.sql.validation;
+
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvg;
+import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction;
+import org.apache.flink.table.planner.utils.TableTestBase;
+import org.apache.flink.table.planner.utils.TableTestUtil;
+
+import org.apache.calcite.sql.SqlMatchRecognize;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+/** Validation test for {@link SqlMatchRecognize}. */
+public class MatchRecognizeValidationTest extends TableTestBase {

Review Comment:
   the test cases for this class and stream.sql.validataion.MatchRecognizeValidationTest are almost same, we can merge them into one. or we can extract a base class which holds all test cases, and each sub class holds `setup` and `after` method.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecMatch.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * 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.nodes.exec.common;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.cep.EventComparator;
+import org.apache.flink.cep.nfa.aftermatch.AfterMatchSkipStrategy;
+import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.cep.operator.CepOperator;
+import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.Quantifier;
+import org.apache.flink.cep.pattern.conditions.BooleanConditions;
+import org.apache.flink.cep.pattern.conditions.IterativeCondition;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.codegen.CodeGenUtils;
+import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
+import org.apache.flink.table.planner.codegen.MatchCodeGenerator;
+import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecMatch;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.MatchSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
+import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.planner.plan.utils.RexDefaultVisitor;
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import org.apache.flink.table.runtime.operators.match.PatternProcessFunctionRunner;
+import org.apache.flink.table.runtime.operators.match.RowDataEventComparator;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.MathUtils;
+
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlMatchRecognize;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.tools.RelBuilder;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Common {@link ExecNode} which matches along with MATCH_RECOGNIZE. */
+public class CommonExecMatch extends ExecNodeBase<RowData>
+        implements ExecNode<RowData>, MultipleTransformationTranslator<RowData> {
+
+    public static final String MATCH_TRANSFORMATION = "match";
+
+    private final MatchSpec matchSpec;
+
+    public CommonExecMatch(
+            ReadableConfig tableConfig,
+            MatchSpec matchSpec,
+            InputProperty inputProperty,
+            RowType outputType,
+            String description) {
+        this(
+                ExecNodeContext.newNodeId(),
+                ExecNodeContext.newContext(BatchExecMatch.class),
+                ExecNodeContext.newPersistedConfig(BatchExecMatch.class, tableConfig),
+                matchSpec,
+                Collections.singletonList(inputProperty),
+                outputType,
+                description);
+    }
+
+    public CommonExecMatch(
+            int id,
+            ExecNodeContext context,
+            ReadableConfig persistedConfig,
+            MatchSpec matchSpec,
+            List<InputProperty> inputProperties,
+            LogicalType outputType,
+            String description) {
+        super(id, context, persistedConfig, inputProperties, outputType, description);
+        checkArgument(inputProperties.size() == 1);
+        this.matchSpec = checkNotNull(matchSpec);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected Transformation<RowData> translateToPlanInternal(
+            PlannerBase planner, ExecNodeConfig config) {
+        final ExecEdge inputEdge = getInputEdges().get(0);
+        final Transformation<RowData> inputTransform =
+                (Transformation<RowData>) inputEdge.translateToPlan(planner);
+        final RowType inputRowType = (RowType) inputEdge.getOutputType();
+
+        checkOrderKeys(inputRowType);
+        final EventComparator<RowData> eventComparator =
+                createEventComparator(config, inputRowType);
+        final Transformation<RowData> timestampedInputTransform =
+                translateOrder(inputTransform, inputRowType, config);
+
+        final Tuple2<Pattern<RowData, RowData>, List<String>> cepPatternAndNames =
+                translatePattern(matchSpec, config, planner.createRelBuilder(), inputRowType);
+        final Pattern<RowData, RowData> cepPattern = cepPatternAndNames.f0;
+
+        // TODO remove this once it is supported in CEP library
+        if (NFACompiler.canProduceEmptyMatches(cepPattern)) {
+            throw new TableException(
+                    "Patterns that can produce empty matches are not supported. There must be at least one non-optional state.");
+        }
+
+        // TODO remove this once it is supported in CEP library
+        if (cepPattern.getQuantifier().hasProperty(Quantifier.QuantifierProperty.GREEDY)) {
+            throw new TableException(
+                    "Greedy quantifiers are not allowed as the last element of a Pattern yet. "
+                            + "Finish your pattern with either a simple variable or reluctant quantifier.");
+        }
+
+        if (matchSpec.isAllRows()) {
+            throw new TableException("All rows per match mode is not supported yet.");
+        }
+
+        final int[] partitionKeys = matchSpec.getPartition().getFieldIndices();
+        final InternalTypeInfo<RowData> inputTypeInfo =
+                (InternalTypeInfo<RowData>) inputTransform.getOutputType();
+        final TypeSerializer<RowData> inputSerializer =
+                inputTypeInfo.createSerializer(planner.getExecEnv().getConfig());
+        final NFACompiler.NFAFactory<RowData> nfaFactory =
+                NFACompiler.compileFactory(cepPattern, false);
+        final MatchCodeGenerator generator =
+                new MatchCodeGenerator(
+                        new CodeGeneratorContext(config),
+                        planner.createRelBuilder(),
+                        false, // nullableInput
+                        JavaScalaConversionUtil.toScala(cepPatternAndNames.f1),
+                        JavaScalaConversionUtil.toScala(Optional.empty()),
+                        CodeGenUtils.DEFAULT_COLLECTOR_TERM());
+        generator.bindInput(
+                inputRowType,
+                CodeGenUtils.DEFAULT_INPUT1_TERM(),
+                JavaScalaConversionUtil.toScala(Optional.empty()));
+        final PatternProcessFunctionRunner patternProcessFunction =
+                generator.generateOneRowPerMatchExpression(
+                        (RowType) getOutputType(), partitionKeys, matchSpec.getMeasures());
+        final CepOperator<RowData, RowData, RowData> operator =
+                new CepOperator<>(
+                        inputSerializer,
+                        isProcTime(inputRowType),
+                        nfaFactory,
+                        eventComparator,
+                        cepPattern.getAfterMatchSkipStrategy(),
+                        patternProcessFunction,
+                        null);
+        final OneInputTransformation<RowData, RowData> transform =
+                ExecNodeUtil.createOneInputTransformation(
+                        timestampedInputTransform,
+                        createTransformationMeta(MATCH_TRANSFORMATION, config),
+                        operator,
+                        InternalTypeInfo.of(getOutputType()),
+                        timestampedInputTransform.getParallelism());
+        final RowDataKeySelector selector =
+                KeySelectorUtil.getRowDataSelector(partitionKeys, inputTypeInfo);
+        transform.setStateKeySelector(selector);
+        transform.setStateKeyType(selector.getProducedType());
+
+        if (inputsContainSingleton()) {
+            transform.setParallelism(1);
+            transform.setMaxParallelism(1);
+        }
+        return transform;
+    }
+
+    public void checkOrderKeys(RowType inputRowType) {}
+
+    private EventComparator<RowData> createEventComparator(
+            ExecNodeConfig config, RowType inputRowType) {
+        SortSpec orderKeys = matchSpec.getOrderKeys();
+        if (orderKeys.getFieldIndices().length > 1) {
+            GeneratedRecordComparator rowComparator =
+                    ComparatorCodeGenerator.gen(
+                            config, "RowDataComparator", inputRowType, orderKeys);
+            return new RowDataEventComparator(rowComparator);
+        } else {
+            return null;
+        }
+    }
+
+    public Transformation<RowData> translateOrder(
+            Transformation<RowData> inputTransform, RowType inputRowType, ReadableConfig config) {
+        return inputTransform;
+    }
+
+    @VisibleForTesting
+    public static Tuple2<Pattern<RowData, RowData>, List<String>> translatePattern(
+            MatchSpec matchSpec,
+            ReadableConfig config,
+            RelBuilder relBuilder,
+            RowType inputRowType) {
+        final PatternVisitor patternVisitor =
+                new PatternVisitor(config, relBuilder, inputRowType, matchSpec);
+
+        final Pattern<RowData, RowData> cepPattern;
+        if (matchSpec.getInterval().isPresent()) {
+            Time interval = translateTimeBound(matchSpec.getInterval().get());
+            cepPattern = matchSpec.getPattern().accept(patternVisitor).within(interval);
+        } else {
+            cepPattern = matchSpec.getPattern().accept(patternVisitor);
+        }
+        return new Tuple2<>(cepPattern, new ArrayList<>(patternVisitor.names));
+    }
+
+    private static Time translateTimeBound(RexNode interval) {
+        if (interval instanceof RexLiteral) {
+            final RexLiteral l = (RexLiteral) interval;
+            if (l.getTypeName().getFamily() == SqlTypeFamily.INTERVAL_DAY_TIME) {
+                return Time.milliseconds(l.getValueAs(Long.class));
+            }
+        }
+        throw new TableException(
+                "Only constant intervals with millisecond resolution are supported as time constraints of patterns.");
+    }
+
+    public boolean isProcTime(RowType inputRowType) {

Review Comment:
   no default implementation here.
   
   row time is not supported for batch? 



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/common/CommonPhysicalMatchRule.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.physical.common;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.plan.logical.MatchRecognize;
+import org.apache.flink.table.planner.plan.nodes.FlinkConvention;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalMatch;
+import org.apache.flink.table.planner.plan.trait.FlinkRelDistribution;
+import org.apache.flink.table.planner.plan.utils.MatchUtil.AggregationPatternVariableFinder;
+import org.apache.flink.table.planner.plan.utils.RexDefaultVisitor;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.util.ImmutableBitSet;
+
+import java.util.Arrays;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * The physical rule is responsible for converting {@link FlinkLogicalMatch} to {@link
+ * CommonPhysicalMatch}.

Review Comment:
   to physical Match rel.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecMatch.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * 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.nodes.exec.common;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.cep.EventComparator;
+import org.apache.flink.cep.nfa.aftermatch.AfterMatchSkipStrategy;
+import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.cep.operator.CepOperator;
+import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.Quantifier;
+import org.apache.flink.cep.pattern.conditions.BooleanConditions;
+import org.apache.flink.cep.pattern.conditions.IterativeCondition;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.codegen.CodeGenUtils;
+import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
+import org.apache.flink.table.planner.codegen.MatchCodeGenerator;
+import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecMatch;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.MatchSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
+import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.planner.plan.utils.RexDefaultVisitor;
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import org.apache.flink.table.runtime.operators.match.PatternProcessFunctionRunner;
+import org.apache.flink.table.runtime.operators.match.RowDataEventComparator;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.MathUtils;
+
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlMatchRecognize;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.tools.RelBuilder;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Common {@link ExecNode} which matches along with MATCH_RECOGNIZE. */
+public class CommonExecMatch extends ExecNodeBase<RowData>
+        implements ExecNode<RowData>, MultipleTransformationTranslator<RowData> {
+
+    public static final String MATCH_TRANSFORMATION = "match";
+
+    private final MatchSpec matchSpec;
+
+    public CommonExecMatch(
+            ReadableConfig tableConfig,
+            MatchSpec matchSpec,
+            InputProperty inputProperty,
+            RowType outputType,
+            String description) {
+        this(
+                ExecNodeContext.newNodeId(),
+                ExecNodeContext.newContext(BatchExecMatch.class),
+                ExecNodeContext.newPersistedConfig(BatchExecMatch.class, tableConfig),
+                matchSpec,
+                Collections.singletonList(inputProperty),
+                outputType,
+                description);
+    }
+
+    public CommonExecMatch(
+            int id,
+            ExecNodeContext context,
+            ReadableConfig persistedConfig,
+            MatchSpec matchSpec,
+            List<InputProperty> inputProperties,
+            LogicalType outputType,
+            String description) {
+        super(id, context, persistedConfig, inputProperties, outputType, description);
+        checkArgument(inputProperties.size() == 1);
+        this.matchSpec = checkNotNull(matchSpec);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected Transformation<RowData> translateToPlanInternal(
+            PlannerBase planner, ExecNodeConfig config) {
+        final ExecEdge inputEdge = getInputEdges().get(0);
+        final Transformation<RowData> inputTransform =
+                (Transformation<RowData>) inputEdge.translateToPlan(planner);
+        final RowType inputRowType = (RowType) inputEdge.getOutputType();
+
+        checkOrderKeys(inputRowType);
+        final EventComparator<RowData> eventComparator =
+                createEventComparator(config, inputRowType);
+        final Transformation<RowData> timestampedInputTransform =
+                translateOrder(inputTransform, inputRowType, config);
+
+        final Tuple2<Pattern<RowData, RowData>, List<String>> cepPatternAndNames =
+                translatePattern(matchSpec, config, planner.createRelBuilder(), inputRowType);
+        final Pattern<RowData, RowData> cepPattern = cepPatternAndNames.f0;
+
+        // TODO remove this once it is supported in CEP library
+        if (NFACompiler.canProduceEmptyMatches(cepPattern)) {
+            throw new TableException(
+                    "Patterns that can produce empty matches are not supported. There must be at least one non-optional state.");
+        }
+
+        // TODO remove this once it is supported in CEP library
+        if (cepPattern.getQuantifier().hasProperty(Quantifier.QuantifierProperty.GREEDY)) {
+            throw new TableException(
+                    "Greedy quantifiers are not allowed as the last element of a Pattern yet. "
+                            + "Finish your pattern with either a simple variable or reluctant quantifier.");
+        }
+
+        if (matchSpec.isAllRows()) {
+            throw new TableException("All rows per match mode is not supported yet.");
+        }
+
+        final int[] partitionKeys = matchSpec.getPartition().getFieldIndices();
+        final InternalTypeInfo<RowData> inputTypeInfo =
+                (InternalTypeInfo<RowData>) inputTransform.getOutputType();
+        final TypeSerializer<RowData> inputSerializer =
+                inputTypeInfo.createSerializer(planner.getExecEnv().getConfig());
+        final NFACompiler.NFAFactory<RowData> nfaFactory =
+                NFACompiler.compileFactory(cepPattern, false);
+        final MatchCodeGenerator generator =
+                new MatchCodeGenerator(
+                        new CodeGeneratorContext(config),
+                        planner.createRelBuilder(),
+                        false, // nullableInput
+                        JavaScalaConversionUtil.toScala(cepPatternAndNames.f1),
+                        JavaScalaConversionUtil.toScala(Optional.empty()),
+                        CodeGenUtils.DEFAULT_COLLECTOR_TERM());
+        generator.bindInput(
+                inputRowType,
+                CodeGenUtils.DEFAULT_INPUT1_TERM(),
+                JavaScalaConversionUtil.toScala(Optional.empty()));
+        final PatternProcessFunctionRunner patternProcessFunction =
+                generator.generateOneRowPerMatchExpression(
+                        (RowType) getOutputType(), partitionKeys, matchSpec.getMeasures());
+        final CepOperator<RowData, RowData, RowData> operator =
+                new CepOperator<>(
+                        inputSerializer,
+                        isProcTime(inputRowType),
+                        nfaFactory,
+                        eventComparator,
+                        cepPattern.getAfterMatchSkipStrategy(),
+                        patternProcessFunction,
+                        null);
+        final OneInputTransformation<RowData, RowData> transform =
+                ExecNodeUtil.createOneInputTransformation(
+                        timestampedInputTransform,
+                        createTransformationMeta(MATCH_TRANSFORMATION, config),
+                        operator,
+                        InternalTypeInfo.of(getOutputType()),
+                        timestampedInputTransform.getParallelism());
+        final RowDataKeySelector selector =
+                KeySelectorUtil.getRowDataSelector(partitionKeys, inputTypeInfo);
+        transform.setStateKeySelector(selector);
+        transform.setStateKeyType(selector.getProducedType());
+
+        if (inputsContainSingleton()) {
+            transform.setParallelism(1);
+            transform.setMaxParallelism(1);
+        }
+        return transform;
+    }
+
+    public void checkOrderKeys(RowType inputRowType) {}

Review Comment:
   public -> protected



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/common/CommonPhysicalMatchRule.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.physical.common;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.plan.logical.MatchRecognize;
+import org.apache.flink.table.planner.plan.nodes.FlinkConvention;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalMatch;
+import org.apache.flink.table.planner.plan.trait.FlinkRelDistribution;
+import org.apache.flink.table.planner.plan.utils.MatchUtil.AggregationPatternVariableFinder;
+import org.apache.flink.table.planner.plan.utils.RexDefaultVisitor;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.util.ImmutableBitSet;
+
+import java.util.Arrays;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * The physical rule is responsible for converting {@link FlinkLogicalMatch} to {@link
+ * CommonPhysicalMatch}.
+ */
+public abstract class CommonPhysicalMatchRule extends ConverterRule {
+
+    public CommonPhysicalMatchRule(
+            Class<? extends RelNode> clazz, RelTrait in, RelTrait out, String descriptionPrefix) {
+        super(clazz, in, out, descriptionPrefix);
+    }
+
+    @Override
+    public boolean matches(RelOptRuleCall call) {
+        FlinkLogicalMatch logicalMatch = call.rel(0);
+
+        validateAggregations(logicalMatch.getMeasures().values());
+        validateAggregations(logicalMatch.getPatternDefinitions().values());
+        // This check might be obsolete once CALCITE-2747 is resolved
+        validateAmbiguousColumns(logicalMatch);
+        return true;
+    }
+
+    public RelNode convert(RelNode rel, FlinkConvention convention) {
+        FlinkLogicalMatch logicalMatch = (FlinkLogicalMatch) rel;
+        RelTraitSet traitSet = rel.getTraitSet().replace(convention);
+        ImmutableBitSet partitionKeys = logicalMatch.getPartitionKeys();
+
+        FlinkRelDistribution requiredDistribution =
+                partitionKeys.isEmpty()
+                        ? FlinkRelDistribution.SINGLETON()
+                        : FlinkRelDistribution.hash(logicalMatch.getPartitionKeys().asList(), true);
+        RelTraitSet requiredTraitSet =
+                rel.getCluster()
+                        .getPlanner()
+                        .emptyTraitSet()
+                        .replace(requiredDistribution)
+                        .replace(convention);
+
+        RelNode convertInput = RelOptRule.convert(logicalMatch.getInput(), requiredTraitSet);
+
+        try {
+            Class.forName(
+                    "org.apache.flink.cep.pattern.Pattern",
+                    false,
+                    Thread.currentThread().getContextClassLoader());
+        } catch (ClassNotFoundException e) {
+            throw new TableException(
+                    "MATCH RECOGNIZE clause requires flink-cep dependency to be present on the classpath.",
+                    e);
+        }
+        return convertToPhysicalMatch(
+                rel.getCluster(),
+                traitSet,
+                convertInput,
+                new MatchRecognize(
+                        logicalMatch.getPattern(),
+                        logicalMatch.getPatternDefinitions(),
+                        logicalMatch.getMeasures(),
+                        logicalMatch.getAfter(),
+                        logicalMatch.getSubsets(),
+                        logicalMatch.isAllRows(),
+                        logicalMatch.getPartitionKeys(),
+                        logicalMatch.getOrderKeys(),
+                        logicalMatch.getInterval()),
+                logicalMatch.getRowType());
+    }
+
+    public abstract RelNode convertToPhysicalMatch(

Review Comment:
   protected



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecMatch.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * 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.nodes.exec.common;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.cep.EventComparator;
+import org.apache.flink.cep.nfa.aftermatch.AfterMatchSkipStrategy;
+import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.cep.operator.CepOperator;
+import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.Quantifier;
+import org.apache.flink.cep.pattern.conditions.BooleanConditions;
+import org.apache.flink.cep.pattern.conditions.IterativeCondition;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.codegen.CodeGenUtils;
+import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
+import org.apache.flink.table.planner.codegen.MatchCodeGenerator;
+import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecMatch;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.MatchSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
+import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.planner.plan.utils.RexDefaultVisitor;
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import org.apache.flink.table.runtime.operators.match.PatternProcessFunctionRunner;
+import org.apache.flink.table.runtime.operators.match.RowDataEventComparator;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.MathUtils;
+
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlMatchRecognize;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.tools.RelBuilder;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Common {@link ExecNode} which matches along with MATCH_RECOGNIZE. */
+public class CommonExecMatch extends ExecNodeBase<RowData>
+        implements ExecNode<RowData>, MultipleTransformationTranslator<RowData> {
+
+    public static final String MATCH_TRANSFORMATION = "match";
+
+    private final MatchSpec matchSpec;
+
+    public CommonExecMatch(
+            ReadableConfig tableConfig,
+            MatchSpec matchSpec,
+            InputProperty inputProperty,
+            RowType outputType,
+            String description) {
+        this(
+                ExecNodeContext.newNodeId(),
+                ExecNodeContext.newContext(BatchExecMatch.class),
+                ExecNodeContext.newPersistedConfig(BatchExecMatch.class, tableConfig),
+                matchSpec,
+                Collections.singletonList(inputProperty),
+                outputType,
+                description);
+    }
+
+    public CommonExecMatch(
+            int id,
+            ExecNodeContext context,
+            ReadableConfig persistedConfig,
+            MatchSpec matchSpec,
+            List<InputProperty> inputProperties,
+            LogicalType outputType,
+            String description) {
+        super(id, context, persistedConfig, inputProperties, outputType, description);
+        checkArgument(inputProperties.size() == 1);
+        this.matchSpec = checkNotNull(matchSpec);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected Transformation<RowData> translateToPlanInternal(
+            PlannerBase planner, ExecNodeConfig config) {
+        final ExecEdge inputEdge = getInputEdges().get(0);
+        final Transformation<RowData> inputTransform =
+                (Transformation<RowData>) inputEdge.translateToPlan(planner);
+        final RowType inputRowType = (RowType) inputEdge.getOutputType();
+
+        checkOrderKeys(inputRowType);
+        final EventComparator<RowData> eventComparator =
+                createEventComparator(config, inputRowType);
+        final Transformation<RowData> timestampedInputTransform =
+                translateOrder(inputTransform, inputRowType, config);
+
+        final Tuple2<Pattern<RowData, RowData>, List<String>> cepPatternAndNames =
+                translatePattern(matchSpec, config, planner.createRelBuilder(), inputRowType);
+        final Pattern<RowData, RowData> cepPattern = cepPatternAndNames.f0;
+
+        // TODO remove this once it is supported in CEP library
+        if (NFACompiler.canProduceEmptyMatches(cepPattern)) {
+            throw new TableException(
+                    "Patterns that can produce empty matches are not supported. There must be at least one non-optional state.");
+        }
+
+        // TODO remove this once it is supported in CEP library
+        if (cepPattern.getQuantifier().hasProperty(Quantifier.QuantifierProperty.GREEDY)) {
+            throw new TableException(
+                    "Greedy quantifiers are not allowed as the last element of a Pattern yet. "
+                            + "Finish your pattern with either a simple variable or reluctant quantifier.");
+        }
+
+        if (matchSpec.isAllRows()) {
+            throw new TableException("All rows per match mode is not supported yet.");
+        }
+
+        final int[] partitionKeys = matchSpec.getPartition().getFieldIndices();
+        final InternalTypeInfo<RowData> inputTypeInfo =
+                (InternalTypeInfo<RowData>) inputTransform.getOutputType();
+        final TypeSerializer<RowData> inputSerializer =
+                inputTypeInfo.createSerializer(planner.getExecEnv().getConfig());
+        final NFACompiler.NFAFactory<RowData> nfaFactory =
+                NFACompiler.compileFactory(cepPattern, false);
+        final MatchCodeGenerator generator =
+                new MatchCodeGenerator(
+                        new CodeGeneratorContext(config),
+                        planner.createRelBuilder(),
+                        false, // nullableInput
+                        JavaScalaConversionUtil.toScala(cepPatternAndNames.f1),
+                        JavaScalaConversionUtil.toScala(Optional.empty()),
+                        CodeGenUtils.DEFAULT_COLLECTOR_TERM());
+        generator.bindInput(
+                inputRowType,
+                CodeGenUtils.DEFAULT_INPUT1_TERM(),
+                JavaScalaConversionUtil.toScala(Optional.empty()));
+        final PatternProcessFunctionRunner patternProcessFunction =
+                generator.generateOneRowPerMatchExpression(
+                        (RowType) getOutputType(), partitionKeys, matchSpec.getMeasures());
+        final CepOperator<RowData, RowData, RowData> operator =
+                new CepOperator<>(
+                        inputSerializer,
+                        isProcTime(inputRowType),
+                        nfaFactory,
+                        eventComparator,
+                        cepPattern.getAfterMatchSkipStrategy(),
+                        patternProcessFunction,
+                        null);
+        final OneInputTransformation<RowData, RowData> transform =
+                ExecNodeUtil.createOneInputTransformation(
+                        timestampedInputTransform,
+                        createTransformationMeta(MATCH_TRANSFORMATION, config),
+                        operator,
+                        InternalTypeInfo.of(getOutputType()),
+                        timestampedInputTransform.getParallelism());
+        final RowDataKeySelector selector =
+                KeySelectorUtil.getRowDataSelector(partitionKeys, inputTypeInfo);
+        transform.setStateKeySelector(selector);
+        transform.setStateKeyType(selector.getProducedType());
+
+        if (inputsContainSingleton()) {
+            transform.setParallelism(1);
+            transform.setMaxParallelism(1);
+        }
+        return transform;
+    }
+
+    public void checkOrderKeys(RowType inputRowType) {}
+
+    private EventComparator<RowData> createEventComparator(
+            ExecNodeConfig config, RowType inputRowType) {
+        SortSpec orderKeys = matchSpec.getOrderKeys();
+        if (orderKeys.getFieldIndices().length > 1) {
+            GeneratedRecordComparator rowComparator =
+                    ComparatorCodeGenerator.gen(
+                            config, "RowDataComparator", inputRowType, orderKeys);
+            return new RowDataEventComparator(rowComparator);
+        } else {
+            return null;
+        }
+    }
+
+    public Transformation<RowData> translateOrder(

Review Comment:
   protected



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/MatchRecognizeTest.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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
+ * imitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.batch.sql;
+
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.planner.utils.BatchTableTestUtil;
+import org.apache.flink.table.planner.utils.TableTestBase;
+
+import org.apache.calcite.sql.SqlMatchRecognize;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/** Tests for {@link SqlMatchRecognize}. */
+public class MatchRecognizeTest extends TableTestBase {
+
+    private BatchTableTestUtil util;
+
+    @Before
+    public void before() {
+        util = batchTestUtil(TableConfig.getDefault());
+        util.getTableEnv()
+                .executeSql(
+                        "CREATE TABLE Ticker (\n"
+                                + "  `symbol` VARCHAR,\n"
+                                + "  `price` INT,\n"
+                                + "  `tax` INT\n"
+                                + ") with (\n"
+                                + "  'connector' = 'values',\n"
+                                + "  'bounded' = 'true'\n"
+                                + ")");
+    }
+
+    @After
+    public void after() {
+        util.getTableEnv().executeSql("DROP TABLE Ticker");
+    }
+
+    @Test
+    public void testCascadeMatch() {

Review Comment:
   please add more test to verify the plan, we can extract a base test class for batch and stream MatchRecognizeTest



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalMatchRule.scala:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.physical.batch
+
+import org.apache.flink.table.api.{TableException, ValidationException}
+import org.apache.flink.table.planner.plan.`trait`.FlinkRelDistribution
+import org.apache.flink.table.planner.plan.logical.MatchRecognize
+import org.apache.flink.table.planner.plan.nodes.FlinkConventions
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalMatch
+import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalMatch
+import org.apache.flink.table.planner.plan.utils.{MatchUtil, RexDefaultVisitor}
+
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rex.{RexCall, RexNode}
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.calcite.util.ImmutableBitSet
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+class BatchPhysicalMatchRule

Review Comment:
   use JAVA for new class



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecMatch.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * 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.nodes.exec.common;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.cep.EventComparator;
+import org.apache.flink.cep.nfa.aftermatch.AfterMatchSkipStrategy;
+import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.cep.operator.CepOperator;
+import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.Quantifier;
+import org.apache.flink.cep.pattern.conditions.BooleanConditions;
+import org.apache.flink.cep.pattern.conditions.IterativeCondition;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.codegen.CodeGenUtils;
+import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
+import org.apache.flink.table.planner.codegen.MatchCodeGenerator;
+import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecMatch;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.MatchSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
+import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.planner.plan.utils.RexDefaultVisitor;
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import org.apache.flink.table.runtime.operators.match.PatternProcessFunctionRunner;
+import org.apache.flink.table.runtime.operators.match.RowDataEventComparator;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.MathUtils;
+
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlMatchRecognize;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.tools.RelBuilder;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Common {@link ExecNode} which matches along with MATCH_RECOGNIZE. */
+public class CommonExecMatch extends ExecNodeBase<RowData>
+        implements ExecNode<RowData>, MultipleTransformationTranslator<RowData> {
+
+    public static final String MATCH_TRANSFORMATION = "match";
+
+    private final MatchSpec matchSpec;
+
+    public CommonExecMatch(
+            ReadableConfig tableConfig,
+            MatchSpec matchSpec,
+            InputProperty inputProperty,
+            RowType outputType,
+            String description) {
+        this(
+                ExecNodeContext.newNodeId(),
+                ExecNodeContext.newContext(BatchExecMatch.class),
+                ExecNodeContext.newPersistedConfig(BatchExecMatch.class, tableConfig),
+                matchSpec,
+                Collections.singletonList(inputProperty),
+                outputType,
+                description);
+    }
+
+    public CommonExecMatch(
+            int id,
+            ExecNodeContext context,
+            ReadableConfig persistedConfig,
+            MatchSpec matchSpec,
+            List<InputProperty> inputProperties,
+            LogicalType outputType,
+            String description) {
+        super(id, context, persistedConfig, inputProperties, outputType, description);
+        checkArgument(inputProperties.size() == 1);
+        this.matchSpec = checkNotNull(matchSpec);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected Transformation<RowData> translateToPlanInternal(
+            PlannerBase planner, ExecNodeConfig config) {
+        final ExecEdge inputEdge = getInputEdges().get(0);
+        final Transformation<RowData> inputTransform =
+                (Transformation<RowData>) inputEdge.translateToPlan(planner);
+        final RowType inputRowType = (RowType) inputEdge.getOutputType();
+
+        checkOrderKeys(inputRowType);
+        final EventComparator<RowData> eventComparator =
+                createEventComparator(config, inputRowType);
+        final Transformation<RowData> timestampedInputTransform =
+                translateOrder(inputTransform, inputRowType, config);
+
+        final Tuple2<Pattern<RowData, RowData>, List<String>> cepPatternAndNames =
+                translatePattern(matchSpec, config, planner.createRelBuilder(), inputRowType);
+        final Pattern<RowData, RowData> cepPattern = cepPatternAndNames.f0;
+
+        // TODO remove this once it is supported in CEP library
+        if (NFACompiler.canProduceEmptyMatches(cepPattern)) {
+            throw new TableException(
+                    "Patterns that can produce empty matches are not supported. There must be at least one non-optional state.");
+        }
+
+        // TODO remove this once it is supported in CEP library
+        if (cepPattern.getQuantifier().hasProperty(Quantifier.QuantifierProperty.GREEDY)) {
+            throw new TableException(
+                    "Greedy quantifiers are not allowed as the last element of a Pattern yet. "
+                            + "Finish your pattern with either a simple variable or reluctant quantifier.");
+        }
+
+        if (matchSpec.isAllRows()) {
+            throw new TableException("All rows per match mode is not supported yet.");
+        }
+
+        final int[] partitionKeys = matchSpec.getPartition().getFieldIndices();
+        final InternalTypeInfo<RowData> inputTypeInfo =
+                (InternalTypeInfo<RowData>) inputTransform.getOutputType();
+        final TypeSerializer<RowData> inputSerializer =
+                inputTypeInfo.createSerializer(planner.getExecEnv().getConfig());
+        final NFACompiler.NFAFactory<RowData> nfaFactory =
+                NFACompiler.compileFactory(cepPattern, false);
+        final MatchCodeGenerator generator =
+                new MatchCodeGenerator(
+                        new CodeGeneratorContext(config),
+                        planner.createRelBuilder(),
+                        false, // nullableInput
+                        JavaScalaConversionUtil.toScala(cepPatternAndNames.f1),
+                        JavaScalaConversionUtil.toScala(Optional.empty()),
+                        CodeGenUtils.DEFAULT_COLLECTOR_TERM());
+        generator.bindInput(
+                inputRowType,
+                CodeGenUtils.DEFAULT_INPUT1_TERM(),
+                JavaScalaConversionUtil.toScala(Optional.empty()));
+        final PatternProcessFunctionRunner patternProcessFunction =
+                generator.generateOneRowPerMatchExpression(
+                        (RowType) getOutputType(), partitionKeys, matchSpec.getMeasures());
+        final CepOperator<RowData, RowData, RowData> operator =
+                new CepOperator<>(
+                        inputSerializer,
+                        isProcTime(inputRowType),
+                        nfaFactory,
+                        eventComparator,
+                        cepPattern.getAfterMatchSkipStrategy(),
+                        patternProcessFunction,
+                        null);
+        final OneInputTransformation<RowData, RowData> transform =
+                ExecNodeUtil.createOneInputTransformation(
+                        timestampedInputTransform,
+                        createTransformationMeta(MATCH_TRANSFORMATION, config),
+                        operator,
+                        InternalTypeInfo.of(getOutputType()),
+                        timestampedInputTransform.getParallelism());
+        final RowDataKeySelector selector =
+                KeySelectorUtil.getRowDataSelector(partitionKeys, inputTypeInfo);
+        transform.setStateKeySelector(selector);
+        transform.setStateKeyType(selector.getProducedType());
+
+        if (inputsContainSingleton()) {
+            transform.setParallelism(1);
+            transform.setMaxParallelism(1);
+        }
+        return transform;
+    }
+
+    public void checkOrderKeys(RowType inputRowType) {}
+
+    private EventComparator<RowData> createEventComparator(
+            ExecNodeConfig config, RowType inputRowType) {
+        SortSpec orderKeys = matchSpec.getOrderKeys();
+        if (orderKeys.getFieldIndices().length > 1) {
+            GeneratedRecordComparator rowComparator =
+                    ComparatorCodeGenerator.gen(
+                            config, "RowDataComparator", inputRowType, orderKeys);
+            return new RowDataEventComparator(rowComparator);
+        } else {
+            return null;
+        }
+    }
+
+    public Transformation<RowData> translateOrder(
+            Transformation<RowData> inputTransform, RowType inputRowType, ReadableConfig config) {
+        return inputTransform;
+    }
+
+    @VisibleForTesting
+    public static Tuple2<Pattern<RowData, RowData>, List<String>> translatePattern(

Review Comment:
   ditto



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalMatch.scala:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.nodes.physical.common
+
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.planner.plan.logical.MatchRecognize
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode
+import org.apache.flink.table.planner.plan.nodes.physical.FlinkPhysicalRel
+import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall
+import org.apache.flink.table.planner.plan.utils.RelExplainUtil._
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
+
+import scala.collection.JavaConversions._
+
+/** Base physical RelNode which matches along with MATCH_RECOGNIZE. */
+abstract class CommonPhysicalMatch(

Review Comment:
   ditto



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/common/CommonPhysicalMatchRule.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.physical.common;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.plan.logical.MatchRecognize;
+import org.apache.flink.table.planner.plan.nodes.FlinkConvention;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalMatch;
+import org.apache.flink.table.planner.plan.trait.FlinkRelDistribution;
+import org.apache.flink.table.planner.plan.utils.MatchUtil.AggregationPatternVariableFinder;
+import org.apache.flink.table.planner.plan.utils.RexDefaultVisitor;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.util.ImmutableBitSet;
+
+import java.util.Arrays;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * The physical rule is responsible for converting {@link FlinkLogicalMatch} to {@link
+ * CommonPhysicalMatch}.
+ */
+public abstract class CommonPhysicalMatchRule extends ConverterRule {
+
+    public CommonPhysicalMatchRule(
+            Class<? extends RelNode> clazz, RelTrait in, RelTrait out, String descriptionPrefix) {
+        super(clazz, in, out, descriptionPrefix);
+    }
+
+    @Override
+    public boolean matches(RelOptRuleCall call) {
+        FlinkLogicalMatch logicalMatch = call.rel(0);
+
+        validateAggregations(logicalMatch.getMeasures().values());
+        validateAggregations(logicalMatch.getPatternDefinitions().values());
+        // This check might be obsolete once CALCITE-2747 is resolved
+        validateAmbiguousColumns(logicalMatch);
+        return true;
+    }
+
+    public RelNode convert(RelNode rel, FlinkConvention convention) {
+        FlinkLogicalMatch logicalMatch = (FlinkLogicalMatch) rel;
+        RelTraitSet traitSet = rel.getTraitSet().replace(convention);
+        ImmutableBitSet partitionKeys = logicalMatch.getPartitionKeys();
+
+        FlinkRelDistribution requiredDistribution =
+                partitionKeys.isEmpty()
+                        ? FlinkRelDistribution.SINGLETON()
+                        : FlinkRelDistribution.hash(logicalMatch.getPartitionKeys().asList(), true);
+        RelTraitSet requiredTraitSet =
+                rel.getCluster()
+                        .getPlanner()
+                        .emptyTraitSet()
+                        .replace(requiredDistribution)
+                        .replace(convention);
+
+        RelNode convertInput = RelOptRule.convert(logicalMatch.getInput(), requiredTraitSet);
+
+        try {
+            Class.forName(
+                    "org.apache.flink.cep.pattern.Pattern",
+                    false,
+                    Thread.currentThread().getContextClassLoader());
+        } catch (ClassNotFoundException e) {
+            throw new TableException(
+                    "MATCH RECOGNIZE clause requires flink-cep dependency to be present on the classpath.",
+                    e);
+        }
+        return convertToPhysicalMatch(
+                rel.getCluster(),
+                traitSet,
+                convertInput,
+                new MatchRecognize(
+                        logicalMatch.getPattern(),
+                        logicalMatch.getPatternDefinitions(),
+                        logicalMatch.getMeasures(),
+                        logicalMatch.getAfter(),
+                        logicalMatch.getSubsets(),
+                        logicalMatch.isAllRows(),
+                        logicalMatch.getPartitionKeys(),
+                        logicalMatch.getOrderKeys(),
+                        logicalMatch.getInterval()),
+                logicalMatch.getRowType());
+    }
+
+    public abstract RelNode convertToPhysicalMatch(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            RelNode convertInput,
+            MatchRecognize matchRecognize,
+            RelDataType rowType);
+
+    private void validateAggregations(Iterable<RexNode> expr) {
+        AggregationsValidator validator = new AggregationsValidator();
+        expr.forEach(e -> e.accept(validator));
+    }
+
+    private void validateAmbiguousColumns(FlinkLogicalMatch logicalMatch) {
+        if (logicalMatch.isAllRows()) {
+            throw new TableException("All rows per match mode is not supported yet.");
+        } else {
+            validateAmbiguousColumnsOnRowPerMatch(
+                    logicalMatch.getPartitionKeys(),
+                    logicalMatch.getMeasures().keySet(),
+                    logicalMatch.getInput().getRowType(),
+                    logicalMatch.getRowType());
+        }
+    }
+
+    private void validateAmbiguousColumnsOnRowPerMatch(
+            ImmutableBitSet partitionKeys,
+            Set<String> measuresNames,
+            RelDataType inputSchema,
+            RelDataType expectedSchema) {
+        int actualSize = partitionKeys.toArray().length + measuresNames.size();
+        int expectedSize = expectedSchema.getFieldCount();
+        if (actualSize != expectedSize) {
+            // try to find ambiguous column
+
+            String ambiguousColumns =
+                    "{"
+                            + Arrays.stream(partitionKeys.toArray())
+                                    .mapToObj(k -> inputSchema.getFieldList().get(k).getName())
+                                    .filter(measuresNames::contains)
+                                    .collect(Collectors.joining(", "))
+                            + "}";

Review Comment:
   Collectors.joining(", ", "{", "}")



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalMatch.scala:
##########
@@ -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.flink.table.planner.plan.nodes.physical.batch
+
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory
+import org.apache.flink.table.planner.plan.logical.MatchRecognize
+import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty}
+import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecMatch
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalMatch
+import org.apache.flink.table.planner.plan.utils.MatchUtil
+import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig
+
+import _root_.java.util
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel._
+import org.apache.calcite.rel.`type`.RelDataType
+
+/** Batch physical RelNode which matches along with MATCH_RECOGNIZE. */
+class BatchPhysicalMatch(

Review Comment:
   use Java instead of Scala



-- 
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-unsubscribe@flink.apache.org

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