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/06 03:55:01 UTC

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

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


##########
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:
   @godfreyhe, `translatePattern` is called by `org.apache.flink.table.planner.match.PatternTranslatorTestBase`, hence this uses `public`.



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