amaliujia commented on a change in pull request #12232:
URL: https://github.com/apache/beam/pull/12232#discussion_r453995446



##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPTypeName.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.beam.sdk.extensions.sql.impl.cep;
+
+import java.io.Serializable;
+
+public enum CEPTypeName implements Serializable {

Review comment:
       Is it possible to reuse 
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L413?

##########
File path: 
sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRelTest.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.beam.sdk.extensions.sql.impl.rel;
+
+import static 
org.apache.beam.sdk.extensions.sql.impl.rel.BaseRelTest.compilePipeline;
+import static 
org.apache.beam.sdk.extensions.sql.impl.rel.BaseRelTest.registerTable;
+
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestBoundedTable;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.junit.Rule;
+import org.junit.Test;
+
+public class BeamMatchRelTest {
+
+  @Rule public final TestPipeline pipeline = TestPipeline.create();
+
+  @Test
+  public void matchLogicalPlanTest() {
+    Schema schemaType =
+        Schema.builder()
+            .addInt32Field("id")
+            .addStringField("name")
+            .addInt32Field("proctime")
+            .build();
+
+    registerTable(
+        "TestTable", TestBoundedTable.of(schemaType).addRows(1, "a", 1, 1, 
"b", 2, 1, "c", 3));
+
+    String sql =
+        "SELECT * "
+            + "FROM TestTable "
+            + "MATCH_RECOGNIZE ("
+            + "PARTITION BY id "
+            + "ORDER BY proctime "
+            + "PATTERN (A B C) "

Review comment:
       To make sure I understand this example. 
   
   Does PATTERN(A B C) means it should produce rows, in which each three rows 
are a set, and in each set, names should be `a`, `b`, `c` and also in this 
order?

##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPTypeName.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.beam.sdk.extensions.sql.impl.cep;
+
+import java.io.Serializable;
+
+public enum CEPTypeName implements Serializable {

Review comment:
       It is ok though if you still want to use a separate enum for CEP types, 
since it is a standalone library.
   
   Just curious, is there a type that is not covered by 
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L413?

##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java
##########
@@ -0,0 +1,380 @@
+/*
+ * 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.beam.sdk.extensions.sql.impl.rel;
+
+import static 
org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException;
+import org.apache.beam.sdk.extensions.sql.impl.cep.CEPPattern;
+import org.apache.beam.sdk.extensions.sql.impl.cep.CEPUtil;
+import org.apache.beam.sdk.extensions.sql.impl.cep.OrderKey;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
+import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollation;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Match;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexVariable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** {@link BeamRelNode} to replace a {@link Match} node. */
+public class BeamMatchRel extends Match implements BeamRelNode {
+
+  public static final Logger LOG = LoggerFactory.getLogger(BeamMatchRel.class);
+
+  public BeamMatchRel(
+      RelOptCluster cluster,
+      RelTraitSet traitSet,
+      RelNode input,
+      RelDataType rowType,
+      RexNode pattern,
+      boolean strictStart,
+      boolean strictEnd,
+      Map<String, RexNode> patternDefinitions,
+      Map<String, RexNode> measures,
+      RexNode after,
+      Map<String, ? extends SortedSet<String>> subsets,
+      boolean allRows,
+      List<RexNode> partitionKeys,
+      RelCollation orderKeys,
+      RexNode interval) {
+
+    super(
+        cluster,
+        traitSet,
+        input,
+        rowType,
+        pattern,
+        strictStart,
+        strictEnd,
+        patternDefinitions,
+        measures,
+        after,
+        subsets,
+        allRows,
+        partitionKeys,
+        orderKeys,
+        interval);
+  }
+
+  @Override
+  public BeamCostModel beamComputeSelfCost(RelOptPlanner planner, 
RelMetadataQuery mq) {
+    return BeamCostModel.FACTORY.makeTinyCost(); // return constant costModel 
for now
+  }
+
+  @Override
+  public NodeStats estimateNodeStats(RelMetadataQuery mq) {
+    // a simple way of getting some estimate data
+    // to be examined further
+    NodeStats inputEstimate = BeamSqlRelUtils.getNodeStats(input, mq);
+    double numRows = inputEstimate.getRowCount();
+    double winSize = inputEstimate.getWindow();
+    double rate = inputEstimate.getRate();
+
+    return NodeStats.create(numRows, rate, winSize).multiply(0.5);
+  }
+
+  @Override
+  public PTransform<PCollectionList<Row>, PCollection<Row>> buildPTransform() {
+
+    return new MatchTransform(partitionKeys, orderKeys, pattern, 
patternDefinitions);
+  }
+
+  private static class MatchTransform extends PTransform<PCollectionList<Row>, 
PCollection<Row>> {
+
+    private final List<RexNode> parKeys;
+    private final RelCollation orderKeys;
+    private final RexNode pattern;
+    private final Map<String, RexNode> patternDefs;
+
+    public MatchTransform(
+        List<RexNode> parKeys,
+        RelCollation orderKeys,
+        RexNode pattern,
+        Map<String, RexNode> patternDefs) {
+      this.parKeys = parKeys;
+      this.orderKeys = orderKeys;
+      this.pattern = pattern;
+      this.patternDefs = patternDefs;
+    }
+
+    @Override
+    public PCollection<Row> expand(PCollectionList<Row> pinput) {
+      checkArgument(
+          pinput.size() == 1,
+          "Wrong number of inputs for %s: %s",
+          BeamMatchRel.class.getSimpleName(),
+          pinput);
+      PCollection<Row> upstream = pinput.get(0);
+
+      Schema collectionSchema = upstream.getSchema();
+
+      Schema.Builder schemaBuilder = new Schema.Builder();
+      for (RexNode i : parKeys) {
+        RexVariable varNode = (RexVariable) i;
+        int index = Integer.parseInt(varNode.getName().substring(1)); // get 
rid of `$`
+        schemaBuilder.addField(collectionSchema.getField(index));
+      }
+      Schema mySchema = schemaBuilder.build();

Review comment:
       Nit: name it `PartitionKeySchema` might be more readable.

##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java
##########
@@ -0,0 +1,380 @@
+/*
+ * 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.beam.sdk.extensions.sql.impl.rel;
+
+import static 
org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException;
+import org.apache.beam.sdk.extensions.sql.impl.cep.CEPPattern;
+import org.apache.beam.sdk.extensions.sql.impl.cep.CEPUtil;
+import org.apache.beam.sdk.extensions.sql.impl.cep.OrderKey;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
+import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollation;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Match;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexVariable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** {@link BeamRelNode} to replace a {@link Match} node. */
+public class BeamMatchRel extends Match implements BeamRelNode {
+
+  public static final Logger LOG = LoggerFactory.getLogger(BeamMatchRel.class);
+
+  public BeamMatchRel(
+      RelOptCluster cluster,
+      RelTraitSet traitSet,
+      RelNode input,
+      RelDataType rowType,
+      RexNode pattern,
+      boolean strictStart,
+      boolean strictEnd,
+      Map<String, RexNode> patternDefinitions,
+      Map<String, RexNode> measures,
+      RexNode after,
+      Map<String, ? extends SortedSet<String>> subsets,
+      boolean allRows,
+      List<RexNode> partitionKeys,
+      RelCollation orderKeys,
+      RexNode interval) {
+
+    super(
+        cluster,
+        traitSet,
+        input,
+        rowType,
+        pattern,
+        strictStart,
+        strictEnd,
+        patternDefinitions,
+        measures,
+        after,
+        subsets,
+        allRows,
+        partitionKeys,
+        orderKeys,
+        interval);
+  }
+
+  @Override
+  public BeamCostModel beamComputeSelfCost(RelOptPlanner planner, 
RelMetadataQuery mq) {
+    return BeamCostModel.FACTORY.makeTinyCost(); // return constant costModel 
for now
+  }
+
+  @Override
+  public NodeStats estimateNodeStats(RelMetadataQuery mq) {
+    // a simple way of getting some estimate data
+    // to be examined further
+    NodeStats inputEstimate = BeamSqlRelUtils.getNodeStats(input, mq);
+    double numRows = inputEstimate.getRowCount();
+    double winSize = inputEstimate.getWindow();
+    double rate = inputEstimate.getRate();
+
+    return NodeStats.create(numRows, rate, winSize).multiply(0.5);
+  }
+
+  @Override
+  public PTransform<PCollectionList<Row>, PCollection<Row>> buildPTransform() {
+
+    return new MatchTransform(partitionKeys, orderKeys, pattern, 
patternDefinitions);
+  }
+
+  private static class MatchTransform extends PTransform<PCollectionList<Row>, 
PCollection<Row>> {
+
+    private final List<RexNode> parKeys;
+    private final RelCollation orderKeys;
+    private final RexNode pattern;
+    private final Map<String, RexNode> patternDefs;
+
+    public MatchTransform(
+        List<RexNode> parKeys,
+        RelCollation orderKeys,
+        RexNode pattern,
+        Map<String, RexNode> patternDefs) {
+      this.parKeys = parKeys;
+      this.orderKeys = orderKeys;
+      this.pattern = pattern;
+      this.patternDefs = patternDefs;
+    }
+
+    @Override
+    public PCollection<Row> expand(PCollectionList<Row> pinput) {
+      checkArgument(
+          pinput.size() == 1,
+          "Wrong number of inputs for %s: %s",
+          BeamMatchRel.class.getSimpleName(),
+          pinput);
+      PCollection<Row> upstream = pinput.get(0);
+
+      Schema collectionSchema = upstream.getSchema();

Review comment:
       Nit: `upstreamSchema` might be a better variable name.

##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java
##########
@@ -0,0 +1,380 @@
+/*
+ * 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.beam.sdk.extensions.sql.impl.rel;
+
+import static 
org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException;
+import org.apache.beam.sdk.extensions.sql.impl.cep.CEPPattern;
+import org.apache.beam.sdk.extensions.sql.impl.cep.CEPUtil;
+import org.apache.beam.sdk.extensions.sql.impl.cep.OrderKey;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
+import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollation;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Match;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexVariable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** {@link BeamRelNode} to replace a {@link Match} node. */
+public class BeamMatchRel extends Match implements BeamRelNode {
+
+  public static final Logger LOG = LoggerFactory.getLogger(BeamMatchRel.class);
+
+  public BeamMatchRel(
+      RelOptCluster cluster,
+      RelTraitSet traitSet,
+      RelNode input,
+      RelDataType rowType,
+      RexNode pattern,
+      boolean strictStart,
+      boolean strictEnd,
+      Map<String, RexNode> patternDefinitions,
+      Map<String, RexNode> measures,
+      RexNode after,
+      Map<String, ? extends SortedSet<String>> subsets,
+      boolean allRows,
+      List<RexNode> partitionKeys,
+      RelCollation orderKeys,
+      RexNode interval) {
+
+    super(
+        cluster,
+        traitSet,
+        input,
+        rowType,
+        pattern,
+        strictStart,
+        strictEnd,
+        patternDefinitions,
+        measures,
+        after,
+        subsets,
+        allRows,
+        partitionKeys,
+        orderKeys,
+        interval);
+  }
+
+  @Override
+  public BeamCostModel beamComputeSelfCost(RelOptPlanner planner, 
RelMetadataQuery mq) {
+    return BeamCostModel.FACTORY.makeTinyCost(); // return constant costModel 
for now
+  }
+
+  @Override
+  public NodeStats estimateNodeStats(RelMetadataQuery mq) {
+    // a simple way of getting some estimate data
+    // to be examined further
+    NodeStats inputEstimate = BeamSqlRelUtils.getNodeStats(input, mq);
+    double numRows = inputEstimate.getRowCount();
+    double winSize = inputEstimate.getWindow();
+    double rate = inputEstimate.getRate();
+
+    return NodeStats.create(numRows, rate, winSize).multiply(0.5);
+  }
+
+  @Override
+  public PTransform<PCollectionList<Row>, PCollection<Row>> buildPTransform() {
+
+    return new MatchTransform(partitionKeys, orderKeys, pattern, 
patternDefinitions);
+  }
+
+  private static class MatchTransform extends PTransform<PCollectionList<Row>, 
PCollection<Row>> {
+
+    private final List<RexNode> parKeys;
+    private final RelCollation orderKeys;
+    private final RexNode pattern;
+    private final Map<String, RexNode> patternDefs;
+
+    public MatchTransform(
+        List<RexNode> parKeys,
+        RelCollation orderKeys,
+        RexNode pattern,
+        Map<String, RexNode> patternDefs) {
+      this.parKeys = parKeys;
+      this.orderKeys = orderKeys;
+      this.pattern = pattern;
+      this.patternDefs = patternDefs;
+    }
+
+    @Override
+    public PCollection<Row> expand(PCollectionList<Row> pinput) {
+      checkArgument(
+          pinput.size() == 1,
+          "Wrong number of inputs for %s: %s",
+          BeamMatchRel.class.getSimpleName(),
+          pinput);
+      PCollection<Row> upstream = pinput.get(0);
+
+      Schema collectionSchema = upstream.getSchema();
+
+      Schema.Builder schemaBuilder = new Schema.Builder();
+      for (RexNode i : parKeys) {
+        RexVariable varNode = (RexVariable) i;
+        int index = Integer.parseInt(varNode.getName().substring(1)); // get 
rid of `$`
+        schemaBuilder.addField(collectionSchema.getField(index));
+      }
+      Schema mySchema = schemaBuilder.build();
+
+      // partition according to the partition keys
+      PCollection<KV<Row, Row>> keyedUpstream = upstream.apply(ParDo.of(new 
MapKeys(mySchema)));
+
+      // group by keys
+      PCollection<KV<Row, Iterable<Row>>> groupedUpstream =
+          keyedUpstream
+              .setCoder(KvCoder.of(RowCoder.of(mySchema), 
RowCoder.of(collectionSchema)))
+              .apply(GroupByKey.create());
+
+      // sort within each keyed partition
+      PCollection<KV<Row, Iterable<Row>>> orderedUpstream =
+          groupedUpstream.apply(ParDo.of(new SortPerKey(collectionSchema, 
orderKeys)));
+
+      // apply the pattern match in each partition
+      ArrayList<CEPPattern> cepPattern =
+          CEPUtil.getCEPPatternFromPattern(collectionSchema, (RexCall) 
pattern, patternDefs);
+      String regexPattern = CEPUtil.getRegexFromPattern((RexCall) pattern);
+      PCollection<KV<Row, Iterable<Row>>> matchedUpstream =
+          orderedUpstream.apply(ParDo.of(new MatchPattern(cepPattern, 
regexPattern)));
+
+      // apply the ParDo for the measures clause
+      // for now, output the all rows of each pattern matched (for testing 
purpose)
+      PCollection<Row> outStream =
+          matchedUpstream.apply(ParDo.of(new 
Measure())).setRowSchema(collectionSchema);
+
+      return outStream;
+    }
+
+    private static class Measure extends DoFn<KV<Row, Iterable<Row>>, Row> {
+
+      @ProcessElement
+      public void processElement(@Element KV<Row, Iterable<Row>> keyRows, 
OutputReceiver<Row> out) {
+        for (Row i : keyRows.getValue()) {
+          out.output(i);
+        }
+      }
+    }
+
+    // TODO: support both ALL ROWS PER MATCH and ONE ROW PER MATCH.
+    // support only one row per match for now.
+    private static class MatchPattern extends DoFn<KV<Row, Iterable<Row>>, 
KV<Row, Iterable<Row>>> {
+
+      private final ArrayList<CEPPattern> pattern;
+      private final String regexPattern;
+
+      MatchPattern(ArrayList<CEPPattern> pattern, String regexPattern) {
+        this.pattern = pattern;
+        this.regexPattern = regexPattern;
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element KV<Row, Iterable<Row>> keyRows, OutputReceiver<KV<Row, 
Iterable<Row>>> out) {
+        ArrayList<Row> rows = new ArrayList<>();
+        StringBuilder patternString = new StringBuilder();
+        for (Row i : keyRows.getValue()) {
+          rows.add(i);
+          // check pattern of row i
+          String patternOfRow = " "; // a row with no matched pattern is 
marked by a space
+          for (int j = 0; j < pattern.size(); ++j) {
+            CEPPattern tryPattern = pattern.get(j);
+            if (tryPattern.evalRow(i)) {
+              patternOfRow = tryPattern.toString();
+            }
+          }
+          patternString.append(patternOfRow);
+        }
+
+        Pattern p = Pattern.compile(regexPattern);
+        Matcher m = p.matcher(patternString.toString());
+        // if the pattern is (A B+ C),
+        // it should return a List three rows matching A B C respectively
+        if (m.matches()) {
+          out.output(KV.of(keyRows.getKey(), rows.subList(m.start(), 
m.end())));
+        }
+      }
+    }
+
+    private static class SortPerKey extends DoFn<KV<Row, Iterable<Row>>, 
KV<Row, Iterable<Row>>> {
+
+      private final Schema cSchema;
+      private final ArrayList<OrderKey> orderKeys;
+
+      public SortPerKey(Schema cSchema, RelCollation orderKeys) {
+        this.cSchema = cSchema;
+
+        List<RelFieldCollation> revOrderKeys = orderKeys.getFieldCollations();
+        Collections.reverse(revOrderKeys);
+        ArrayList<OrderKey> revOrderKeysList = new ArrayList<>();
+        for (RelFieldCollation i : revOrderKeys) {
+          int fIndex = i.getFieldIndex();
+          RelFieldCollation.Direction dir = i.getDirection();
+          if (dir == RelFieldCollation.Direction.ASCENDING) {

Review comment:
       In fact, there is also a NullDirection to consider (Null first/Null 
last): 
https://github.com/apache/calcite/blob/master/core/src/main/java/org/apache/calcite/rel/RelFieldCollation.java#L185
   
   
   It is ok to not handle it for now, but please leave a TODO comment (i.e. `// 
TODO: handle NullDirection`)

##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperand.java
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.beam.sdk.extensions.sql.impl.cep;
+
+public class CEPOperand {}

Review comment:
       This seems a unused class?

##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMatchRule.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.beam.sdk.extensions.sql.impl.rule;
+
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamMatchRel;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Match;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalMatch;
+
+public class BeamMatchRule extends ConverterRule {

Review comment:
       For all new classes, please add javadoc to explain these classes (i.e. 
`/** */`). Adding comments are usual good idea to improve your code's 
readability. 

##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java
##########
@@ -0,0 +1,380 @@
+/*
+ * 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.beam.sdk.extensions.sql.impl.rel;
+
+import static 
org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException;
+import org.apache.beam.sdk.extensions.sql.impl.cep.CEPPattern;
+import org.apache.beam.sdk.extensions.sql.impl.cep.CEPUtil;
+import org.apache.beam.sdk.extensions.sql.impl.cep.OrderKey;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
+import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollation;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Match;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexVariable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** {@link BeamRelNode} to replace a {@link Match} node. */
+public class BeamMatchRel extends Match implements BeamRelNode {
+
+  public static final Logger LOG = LoggerFactory.getLogger(BeamMatchRel.class);
+
+  public BeamMatchRel(
+      RelOptCluster cluster,
+      RelTraitSet traitSet,
+      RelNode input,
+      RelDataType rowType,
+      RexNode pattern,
+      boolean strictStart,
+      boolean strictEnd,
+      Map<String, RexNode> patternDefinitions,
+      Map<String, RexNode> measures,
+      RexNode after,
+      Map<String, ? extends SortedSet<String>> subsets,
+      boolean allRows,
+      List<RexNode> partitionKeys,
+      RelCollation orderKeys,
+      RexNode interval) {
+
+    super(
+        cluster,
+        traitSet,
+        input,
+        rowType,
+        pattern,
+        strictStart,
+        strictEnd,
+        patternDefinitions,
+        measures,
+        after,
+        subsets,
+        allRows,
+        partitionKeys,
+        orderKeys,
+        interval);
+  }
+
+  @Override
+  public BeamCostModel beamComputeSelfCost(RelOptPlanner planner, 
RelMetadataQuery mq) {
+    return BeamCostModel.FACTORY.makeTinyCost(); // return constant costModel 
for now
+  }
+
+  @Override
+  public NodeStats estimateNodeStats(RelMetadataQuery mq) {
+    // a simple way of getting some estimate data
+    // to be examined further
+    NodeStats inputEstimate = BeamSqlRelUtils.getNodeStats(input, mq);
+    double numRows = inputEstimate.getRowCount();
+    double winSize = inputEstimate.getWindow();
+    double rate = inputEstimate.getRate();
+
+    return NodeStats.create(numRows, rate, winSize).multiply(0.5);
+  }
+
+  @Override
+  public PTransform<PCollectionList<Row>, PCollection<Row>> buildPTransform() {
+
+    return new MatchTransform(partitionKeys, orderKeys, pattern, 
patternDefinitions);
+  }
+
+  private static class MatchTransform extends PTransform<PCollectionList<Row>, 
PCollection<Row>> {
+
+    private final List<RexNode> parKeys;
+    private final RelCollation orderKeys;
+    private final RexNode pattern;
+    private final Map<String, RexNode> patternDefs;
+
+    public MatchTransform(
+        List<RexNode> parKeys,
+        RelCollation orderKeys,
+        RexNode pattern,
+        Map<String, RexNode> patternDefs) {
+      this.parKeys = parKeys;
+      this.orderKeys = orderKeys;
+      this.pattern = pattern;
+      this.patternDefs = patternDefs;
+    }
+
+    @Override
+    public PCollection<Row> expand(PCollectionList<Row> pinput) {
+      checkArgument(
+          pinput.size() == 1,
+          "Wrong number of inputs for %s: %s",
+          BeamMatchRel.class.getSimpleName(),
+          pinput);
+      PCollection<Row> upstream = pinput.get(0);
+
+      Schema collectionSchema = upstream.getSchema();
+
+      Schema.Builder schemaBuilder = new Schema.Builder();
+      for (RexNode i : parKeys) {
+        RexVariable varNode = (RexVariable) i;
+        int index = Integer.parseInt(varNode.getName().substring(1)); // get 
rid of `$`
+        schemaBuilder.addField(collectionSchema.getField(index));

Review comment:
       Ah so is `collectionSchema`'s field name the same as varNode's name 
(including that `$`)?  
   
   See Schema.getName API: 
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L1270

##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java
##########
@@ -0,0 +1,380 @@
+/*
+ * 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.beam.sdk.extensions.sql.impl.rel;
+
+import static 
org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException;
+import org.apache.beam.sdk.extensions.sql.impl.cep.CEPPattern;
+import org.apache.beam.sdk.extensions.sql.impl.cep.CEPUtil;
+import org.apache.beam.sdk.extensions.sql.impl.cep.OrderKey;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
+import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollation;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Match;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexVariable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** {@link BeamRelNode} to replace a {@link Match} node. */
+public class BeamMatchRel extends Match implements BeamRelNode {
+
+  public static final Logger LOG = LoggerFactory.getLogger(BeamMatchRel.class);
+
+  public BeamMatchRel(
+      RelOptCluster cluster,
+      RelTraitSet traitSet,
+      RelNode input,
+      RelDataType rowType,
+      RexNode pattern,
+      boolean strictStart,
+      boolean strictEnd,
+      Map<String, RexNode> patternDefinitions,
+      Map<String, RexNode> measures,
+      RexNode after,
+      Map<String, ? extends SortedSet<String>> subsets,
+      boolean allRows,
+      List<RexNode> partitionKeys,
+      RelCollation orderKeys,
+      RexNode interval) {
+
+    super(
+        cluster,
+        traitSet,
+        input,
+        rowType,
+        pattern,
+        strictStart,
+        strictEnd,
+        patternDefinitions,
+        measures,
+        after,
+        subsets,
+        allRows,
+        partitionKeys,
+        orderKeys,
+        interval);
+  }
+
+  @Override
+  public BeamCostModel beamComputeSelfCost(RelOptPlanner planner, 
RelMetadataQuery mq) {
+    return BeamCostModel.FACTORY.makeTinyCost(); // return constant costModel 
for now
+  }
+
+  @Override
+  public NodeStats estimateNodeStats(RelMetadataQuery mq) {
+    // a simple way of getting some estimate data
+    // to be examined further
+    NodeStats inputEstimate = BeamSqlRelUtils.getNodeStats(input, mq);
+    double numRows = inputEstimate.getRowCount();
+    double winSize = inputEstimate.getWindow();
+    double rate = inputEstimate.getRate();
+
+    return NodeStats.create(numRows, rate, winSize).multiply(0.5);
+  }
+
+  @Override
+  public PTransform<PCollectionList<Row>, PCollection<Row>> buildPTransform() {
+
+    return new MatchTransform(partitionKeys, orderKeys, pattern, 
patternDefinitions);
+  }
+
+  private static class MatchTransform extends PTransform<PCollectionList<Row>, 
PCollection<Row>> {
+
+    private final List<RexNode> parKeys;
+    private final RelCollation orderKeys;
+    private final RexNode pattern;
+    private final Map<String, RexNode> patternDefs;
+
+    public MatchTransform(
+        List<RexNode> parKeys,
+        RelCollation orderKeys,
+        RexNode pattern,
+        Map<String, RexNode> patternDefs) {
+      this.parKeys = parKeys;
+      this.orderKeys = orderKeys;
+      this.pattern = pattern;
+      this.patternDefs = patternDefs;
+    }
+
+    @Override
+    public PCollection<Row> expand(PCollectionList<Row> pinput) {
+      checkArgument(
+          pinput.size() == 1,
+          "Wrong number of inputs for %s: %s",
+          BeamMatchRel.class.getSimpleName(),
+          pinput);
+      PCollection<Row> upstream = pinput.get(0);
+
+      Schema collectionSchema = upstream.getSchema();
+
+      Schema.Builder schemaBuilder = new Schema.Builder();
+      for (RexNode i : parKeys) {
+        RexVariable varNode = (RexVariable) i;
+        int index = Integer.parseInt(varNode.getName().substring(1)); // get 
rid of `$`
+        schemaBuilder.addField(collectionSchema.getField(index));
+      }
+      Schema mySchema = schemaBuilder.build();
+
+      // partition according to the partition keys
+      PCollection<KV<Row, Row>> keyedUpstream = upstream.apply(ParDo.of(new 
MapKeys(mySchema)));
+
+      // group by keys
+      PCollection<KV<Row, Iterable<Row>>> groupedUpstream =
+          keyedUpstream
+              .setCoder(KvCoder.of(RowCoder.of(mySchema), 
RowCoder.of(collectionSchema)))
+              .apply(GroupByKey.create());
+
+      // sort within each keyed partition
+      PCollection<KV<Row, Iterable<Row>>> orderedUpstream =
+          groupedUpstream.apply(ParDo.of(new SortPerKey(collectionSchema, 
orderKeys)));
+
+      // apply the pattern match in each partition
+      ArrayList<CEPPattern> cepPattern =
+          CEPUtil.getCEPPatternFromPattern(collectionSchema, (RexCall) 
pattern, patternDefs);
+      String regexPattern = CEPUtil.getRegexFromPattern((RexCall) pattern);
+      PCollection<KV<Row, Iterable<Row>>> matchedUpstream =
+          orderedUpstream.apply(ParDo.of(new MatchPattern(cepPattern, 
regexPattern)));
+
+      // apply the ParDo for the measures clause
+      // for now, output the all rows of each pattern matched (for testing 
purpose)
+      PCollection<Row> outStream =
+          matchedUpstream.apply(ParDo.of(new 
Measure())).setRowSchema(collectionSchema);
+
+      return outStream;
+    }
+
+    private static class Measure extends DoFn<KV<Row, Iterable<Row>>, Row> {
+
+      @ProcessElement
+      public void processElement(@Element KV<Row, Iterable<Row>> keyRows, 
OutputReceiver<Row> out) {
+        for (Row i : keyRows.getValue()) {
+          out.output(i);
+        }
+      }
+    }
+
+    // TODO: support both ALL ROWS PER MATCH and ONE ROW PER MATCH.
+    // support only one row per match for now.
+    private static class MatchPattern extends DoFn<KV<Row, Iterable<Row>>, 
KV<Row, Iterable<Row>>> {
+
+      private final ArrayList<CEPPattern> pattern;
+      private final String regexPattern;
+
+      MatchPattern(ArrayList<CEPPattern> pattern, String regexPattern) {
+        this.pattern = pattern;
+        this.regexPattern = regexPattern;
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element KV<Row, Iterable<Row>> keyRows, OutputReceiver<KV<Row, 
Iterable<Row>>> out) {
+        ArrayList<Row> rows = new ArrayList<>();
+        StringBuilder patternString = new StringBuilder();
+        for (Row i : keyRows.getValue()) {
+          rows.add(i);
+          // check pattern of row i
+          String patternOfRow = " "; // a row with no matched pattern is 
marked by a space
+          for (int j = 0; j < pattern.size(); ++j) {
+            CEPPattern tryPattern = pattern.get(j);
+            if (tryPattern.evalRow(i)) {
+              patternOfRow = tryPattern.toString();
+            }
+          }
+          patternString.append(patternOfRow);
+        }
+
+        Pattern p = Pattern.compile(regexPattern);

Review comment:
       I think you got to make `Pattern p` as a variable to compile once? 

##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java
##########
@@ -0,0 +1,380 @@
+/*
+ * 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.beam.sdk.extensions.sql.impl.rel;
+
+import static 
org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException;
+import org.apache.beam.sdk.extensions.sql.impl.cep.CEPPattern;
+import org.apache.beam.sdk.extensions.sql.impl.cep.CEPUtil;
+import org.apache.beam.sdk.extensions.sql.impl.cep.OrderKey;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
+import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollation;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Match;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexVariable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** {@link BeamRelNode} to replace a {@link Match} node. */
+public class BeamMatchRel extends Match implements BeamRelNode {
+
+  public static final Logger LOG = LoggerFactory.getLogger(BeamMatchRel.class);
+
+  public BeamMatchRel(
+      RelOptCluster cluster,
+      RelTraitSet traitSet,
+      RelNode input,
+      RelDataType rowType,
+      RexNode pattern,
+      boolean strictStart,
+      boolean strictEnd,
+      Map<String, RexNode> patternDefinitions,
+      Map<String, RexNode> measures,
+      RexNode after,
+      Map<String, ? extends SortedSet<String>> subsets,
+      boolean allRows,
+      List<RexNode> partitionKeys,
+      RelCollation orderKeys,
+      RexNode interval) {
+
+    super(
+        cluster,
+        traitSet,
+        input,
+        rowType,
+        pattern,
+        strictStart,
+        strictEnd,
+        patternDefinitions,
+        measures,
+        after,
+        subsets,
+        allRows,
+        partitionKeys,
+        orderKeys,
+        interval);
+  }
+
+  @Override
+  public BeamCostModel beamComputeSelfCost(RelOptPlanner planner, 
RelMetadataQuery mq) {
+    return BeamCostModel.FACTORY.makeTinyCost(); // return constant costModel 
for now
+  }
+
+  @Override
+  public NodeStats estimateNodeStats(RelMetadataQuery mq) {
+    // a simple way of getting some estimate data
+    // to be examined further
+    NodeStats inputEstimate = BeamSqlRelUtils.getNodeStats(input, mq);
+    double numRows = inputEstimate.getRowCount();
+    double winSize = inputEstimate.getWindow();
+    double rate = inputEstimate.getRate();
+
+    return NodeStats.create(numRows, rate, winSize).multiply(0.5);
+  }
+
+  @Override
+  public PTransform<PCollectionList<Row>, PCollection<Row>> buildPTransform() {
+
+    return new MatchTransform(partitionKeys, orderKeys, pattern, 
patternDefinitions);
+  }
+
+  private static class MatchTransform extends PTransform<PCollectionList<Row>, 
PCollection<Row>> {
+
+    private final List<RexNode> parKeys;
+    private final RelCollation orderKeys;
+    private final RexNode pattern;
+    private final Map<String, RexNode> patternDefs;
+
+    public MatchTransform(
+        List<RexNode> parKeys,
+        RelCollation orderKeys,
+        RexNode pattern,
+        Map<String, RexNode> patternDefs) {
+      this.parKeys = parKeys;
+      this.orderKeys = orderKeys;
+      this.pattern = pattern;
+      this.patternDefs = patternDefs;
+    }
+
+    @Override
+    public PCollection<Row> expand(PCollectionList<Row> pinput) {
+      checkArgument(
+          pinput.size() == 1,
+          "Wrong number of inputs for %s: %s",
+          BeamMatchRel.class.getSimpleName(),
+          pinput);
+      PCollection<Row> upstream = pinput.get(0);
+
+      Schema collectionSchema = upstream.getSchema();
+
+      Schema.Builder schemaBuilder = new Schema.Builder();
+      for (RexNode i : parKeys) {
+        RexVariable varNode = (RexVariable) i;
+        int index = Integer.parseInt(varNode.getName().substring(1)); // get 
rid of `$`
+        schemaBuilder.addField(collectionSchema.getField(index));
+      }
+      Schema mySchema = schemaBuilder.build();
+
+      // partition according to the partition keys
+      PCollection<KV<Row, Row>> keyedUpstream = upstream.apply(ParDo.of(new 
MapKeys(mySchema)));
+
+      // group by keys
+      PCollection<KV<Row, Iterable<Row>>> groupedUpstream =
+          keyedUpstream
+              .setCoder(KvCoder.of(RowCoder.of(mySchema), 
RowCoder.of(collectionSchema)))
+              .apply(GroupByKey.create());
+
+      // sort within each keyed partition
+      PCollection<KV<Row, Iterable<Row>>> orderedUpstream =
+          groupedUpstream.apply(ParDo.of(new SortPerKey(collectionSchema, 
orderKeys)));
+
+      // apply the pattern match in each partition
+      ArrayList<CEPPattern> cepPattern =
+          CEPUtil.getCEPPatternFromPattern(collectionSchema, (RexCall) 
pattern, patternDefs);
+      String regexPattern = CEPUtil.getRegexFromPattern((RexCall) pattern);
+      PCollection<KV<Row, Iterable<Row>>> matchedUpstream =
+          orderedUpstream.apply(ParDo.of(new MatchPattern(cepPattern, 
regexPattern)));
+
+      // apply the ParDo for the measures clause
+      // for now, output the all rows of each pattern matched (for testing 
purpose)
+      PCollection<Row> outStream =
+          matchedUpstream.apply(ParDo.of(new 
Measure())).setRowSchema(collectionSchema);
+
+      return outStream;
+    }
+
+    private static class Measure extends DoFn<KV<Row, Iterable<Row>>, Row> {
+
+      @ProcessElement
+      public void processElement(@Element KV<Row, Iterable<Row>> keyRows, 
OutputReceiver<Row> out) {
+        for (Row i : keyRows.getValue()) {
+          out.output(i);
+        }
+      }
+    }
+
+    // TODO: support both ALL ROWS PER MATCH and ONE ROW PER MATCH.
+    // support only one row per match for now.
+    private static class MatchPattern extends DoFn<KV<Row, Iterable<Row>>, 
KV<Row, Iterable<Row>>> {
+
+      private final ArrayList<CEPPattern> pattern;
+      private final String regexPattern;
+
+      MatchPattern(ArrayList<CEPPattern> pattern, String regexPattern) {
+        this.pattern = pattern;
+        this.regexPattern = regexPattern;
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element KV<Row, Iterable<Row>> keyRows, OutputReceiver<KV<Row, 
Iterable<Row>>> out) {
+        ArrayList<Row> rows = new ArrayList<>();
+        StringBuilder patternString = new StringBuilder();
+        for (Row i : keyRows.getValue()) {
+          rows.add(i);
+          // check pattern of row i
+          String patternOfRow = " "; // a row with no matched pattern is 
marked by a space
+          for (int j = 0; j < pattern.size(); ++j) {
+            CEPPattern tryPattern = pattern.get(j);
+            if (tryPattern.evalRow(i)) {
+              patternOfRow = tryPattern.toString();
+            }
+          }
+          patternString.append(patternOfRow);
+        }
+
+        Pattern p = Pattern.compile(regexPattern);
+        Matcher m = p.matcher(patternString.toString());
+        // if the pattern is (A B+ C),
+        // it should return a List three rows matching A B C respectively
+        if (m.matches()) {
+          out.output(KV.of(keyRows.getKey(), rows.subList(m.start(), 
m.end())));
+        }
+      }
+    }
+
+    private static class SortPerKey extends DoFn<KV<Row, Iterable<Row>>, 
KV<Row, Iterable<Row>>> {
+
+      private final Schema cSchema;
+      private final ArrayList<OrderKey> orderKeys;
+
+      public SortPerKey(Schema cSchema, RelCollation orderKeys) {
+        this.cSchema = cSchema;
+
+        List<RelFieldCollation> revOrderKeys = orderKeys.getFieldCollations();
+        Collections.reverse(revOrderKeys);
+        ArrayList<OrderKey> revOrderKeysList = new ArrayList<>();
+        for (RelFieldCollation i : revOrderKeys) {
+          int fIndex = i.getFieldIndex();
+          RelFieldCollation.Direction dir = i.getDirection();
+          if (dir == RelFieldCollation.Direction.ASCENDING) {
+            revOrderKeysList.add(new OrderKey(fIndex, false));
+          } else {
+            revOrderKeysList.add(new OrderKey(fIndex, true));
+          }
+        }
+
+        this.orderKeys = revOrderKeysList;
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element KV<Row, Iterable<Row>> keyRows, OutputReceiver<KV<Row, 
Iterable<Row>>> out) {
+        ArrayList<Row> rows = new ArrayList<Row>();
+        for (Row i : keyRows.getValue()) {
+          rows.add(i);
+        }
+        for (OrderKey i : orderKeys) {
+          int fIndex = i.getIndex();
+          boolean dir = i.getDir();
+          rows.sort(new SortComparator(fIndex, dir));
+        }
+        // TODO: Change the comparator to the row comparator:
+        // 
https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java#L373
+
+        out.output(KV.of(keyRows.getKey(), rows));
+      }
+
+      private class SortComparator implements Comparator<Row> {
+
+        private final int fIndex;
+        private final int inv;
+

Review comment:
       In fact what you are doing is ok. This is minor.




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