wenlong88 commented on a change in pull request #14625:
URL: https://github.com/apache/flink/pull/14625#discussion_r556995228



##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.stream;
+
+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.streaming.api.operators.ProcessOperator;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.table.api.TableConfig;
+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.utils.MatchSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.SortSpec;
+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.operators.match.RowtimeProcessFunction;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimestampType;
+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;
+
+/** Stream {@link ExecNode} which matches along with LogicalMatch. */
+public class StreamExecMatch extends ExecNodeBase<RowData> implements 
StreamExecNode<RowData> {
+
+    private final MatchSpec matchSpec;
+
+    public StreamExecMatch(
+            MatchSpec matchSpec, ExecEdge inputEdge, RowType outputType, 
String description) {
+        super(Collections.singletonList(inputEdge), outputType, description);
+        this.matchSpec = matchSpec;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected Transformation<RowData> translateToPlanInternal(PlannerBase 
planner) {
+        final TableConfig config = planner.getTableConfig();
+        final ExecNode<RowData> inputNode = (ExecNode<RowData>) 
getInputNodes().get(0);
+        final Transformation<RowData> inputTransform = 
inputNode.translateToPlan(planner);
+        final RowType inputRowType = (RowType) inputNode.getOutputType();
+
+        checkOrderKeys(inputRowType);
+        final EventComparator<RowData> eventComparator =
+                createEventComparator(config, inputRowType);
+        final Transformation<RowData> timestampedInputTransform =
+                translateOrder(inputTransform, inputRowType);
+
+        final Tuple2<Pattern<RowData, RowData>, List<String>> 
cepPatternAndNames =
+                translatePattern(matchSpec, config, planner.getRelBuilder(), 
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 SortSpec.SortFieldSpec timeOrderField = 
matchSpec.getOrderKeys().getFieldSpec(0);
+        final LogicalType timeOrderFieldType =
+                inputRowType.getTypeAt(timeOrderField.getFieldIndex());
+
+        final boolean isProctime = 
TypeCheckUtils.isProcTime(timeOrderFieldType);
+        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.getRelBuilder(),
+                        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,
+                        nfaFactory,
+                        eventComparator,
+                        cepPattern.getAfterMatchSkipStrategy(),
+                        patternProcessFunction,
+                        null);
+        final OneInputTransformation<RowData, RowData> transform =
+                new OneInputTransformation<>(
+                        timestampedInputTransform,
+                        getDesc(),
+                        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;
+    }
+
+    private void checkOrderKeys(RowType inputRowType) {
+        SortSpec orderKeys = matchSpec.getOrderKeys();
+        if (orderKeys.getFieldIndices().length == 0) {

Review comment:
       use orderKeys.getFieldSize() == 0 ?  




----------------------------------------------------------------
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.

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


Reply via email to