[
https://issues.apache.org/jira/browse/BEAM-9543?focusedWorklogId=450614&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-450614
]
ASF GitHub Bot logged work on BEAM-9543:
----------------------------------------
Author: ASF GitHub Bot
Created on: 24/Jun/20 20:22
Start Date: 24/Jun/20 20:22
Worklog Time Spent: 10m
Work Description: amaliujia commented on a change in pull request #12073:
URL: https://github.com/apache/beam/pull/12073#discussion_r445149049
##########
File path:
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java
##########
@@ -0,0 +1,279 @@
+package org.apache.beam.sdk.extensions.sql.impl.rel;
+
+import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException;
+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.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+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.RexNode;
+import
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexVariable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Array;
+import java.util.*;
+
+import static
org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+
+/** {@link BeamRelNode} to replace a {@link Match} node. */
+public class BeamMatchRel extends Match implements BeamRelNode {
+
+ private 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);
+ }
+
+ private static class matchTransform extends
PTransform<PCollectionList<Row>, PCollection<Row>> {
+
+ private final List<RexNode> parKeys;
+ private final RelCollation orderKeys;
+
+ public matchTransform(List<RexNode> parKeys, RelCollation orderKeys) {
+ this.parKeys = parKeys;
+ this.orderKeys = orderKeys;
+ }
+
+ @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)));
+
+ // sort within each partition
+ PCollection<KV<Row, ArrayList<Row>>> orderedUpstream =
keyedUpstream
+ .apply(Combine.<Row, Row, ArrayList<Row>>perKey(new
SortParts(mySchema, orderKeys)));
+
+ return null;
+ }
+
+ private static class SortParts extends Combine.CombineFn<Row,
ArrayList<Row>, ArrayList<Row>> {
+
+ private final Schema mySchema;
+ private final List<RelFieldCollation> orderKeys;
+
+ public SortParts(Schema mySchema, RelCollation orderKeys) {
+ this.mySchema = mySchema;
+ List<RelFieldCollation> revOrderKeys =
orderKeys.getFieldCollations();
+ Collections.reverse(revOrderKeys);
+ this.orderKeys = revOrderKeys;
+ }
+
+ @Override
+ public ArrayList<Row> createAccumulator() {
+ return new ArrayList<Row>();
+ }
+
+ @Override
+ public ArrayList<Row> addInput(ArrayList<Row> Accum, Row inRow) {
+ Accum.add(inRow);
+ return Accum;
+ }
+
+ @Override
+ public ArrayList<Row> mergeAccumulators(Iterable<ArrayList<Row>>
Accums) {
+ ArrayList<Row> aggAccum = new ArrayList<Row>();
+ for (ArrayList<Row> i : Accums) {
+ aggAccum.addAll(i);
+ }
+ return aggAccum;
+ }
+
+ @Override
+ public ArrayList<Row> extractOutput(ArrayList<Row> rawRows) {
+ for (RelFieldCollation i : orderKeys) {
+ int fIndex = i.getFieldIndex();
+ RelFieldCollation.Direction dir = i.getDirection();
+ if (dir == RelFieldCollation.Direction.ASCENDING) {
+ Collections.sort(rawRows, new sortComparator(fIndex,
true));
+ }
+ }
+ return rawRows;
+ }
+
+ private class sortComparator implements Comparator<Row> {
+
+ private final int fIndex;
+ private final int inv;
+
+ public sortComparator(int fIndex, boolean inverse) {
+ this.fIndex = fIndex;
+ this.inv = inverse ? -1 : 1;
+ }
+
+ @Override
+ public int compare(Row o1, Row o2) {
+ Schema.Field fd = mySchema.getField(fIndex);
+ Schema.FieldType dtype = fd.getType();
+ switch (dtype.getTypeName()) {
Review comment:
There is a 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
----------------------------------------------------------------
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:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 450614)
Time Spent: 40m (was: 0.5h)
> BeamSQL Pattern Recognization Functionality
> -------------------------------------------
>
> Key: BEAM-9543
> URL: https://issues.apache.org/jira/browse/BEAM-9543
> Project: Beam
> Issue Type: New Feature
> Components: dsl-sql
> Reporter: Rui Wang
> Assignee: Qihang Zeng
> Priority: P2
> Labels: gsoc, gsoc2020, mentor
> Time Spent: 40m
> Remaining Estimate: 0h
>
> The goal of this Jira is to support the following syntax in BeamSQL:
> {code:sql}
> SELECT T.aid, T.bid, T.cid
> FROM MyTable
> MATCH_RECOGNIZE (
> PARTITION BY userid
> ORDER BY proctime
> MEASURES
> A.id AS aid,
> B.id AS bid,
> C.id AS cid
> PATTERN (A B C)
> DEFINE
> A AS name = 'a',
> B AS name = 'b',
> C AS name = 'c'
> ) AS T
> {code}
> match_recognize is in SQL standard 2016. Currently Calcite also supports it.
> A good reference to match_recognize is [1]
> This will requires touch core components of BeamSQL:
> 1. SQL parser to support the syntax above.
> 2. SQL core to implement physical relational operator.
> 3. Distributed algorithms to implement a list of functions in a distributed
> manner.
> other references:
> Calcite match_recognize syntax [2]
> [1]:
> https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/streaming/match_recognize.html
> [2]: https://calcite.apache.org/docs/reference.html#syntax-1
--
This message was sent by Atlassian Jira
(v8.3.4#803005)