[ 
https://issues.apache.org/jira/browse/BEAM-3182?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16331839#comment-16331839
 ] 

ASF GitHub Bot commented on BEAM-3182:
--------------------------------------

kennknowles closed pull request #4351: [BEAM-3182][Nexmark][SQL] Implement sql 
query 2
URL: https://github.com/apache/beam/pull/4351
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java
 
b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java
index 6e24207951a..2f58a1ab971 100644
--- 
a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java
+++ 
b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java
@@ -25,7 +25,6 @@
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -34,9 +33,7 @@
 import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.concurrent.ThreadLocalRandom;
-
 import javax.annotation.Nullable;
-
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.coders.CoderException;
@@ -83,6 +80,7 @@
 import org.apache.beam.sdk.nexmark.queries.sql.NexmarkSqlQuery;
 import org.apache.beam.sdk.nexmark.queries.sql.SqlQuery0;
 import org.apache.beam.sdk.nexmark.queries.sql.SqlQuery1;
+import org.apache.beam.sdk.nexmark.queries.sql.SqlQuery2;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
@@ -92,7 +90,6 @@
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
-
 import org.joda.time.Duration;
 import org.slf4j.LoggerFactory;
 
@@ -1200,7 +1197,8 @@ private NexmarkQuery getNexmarkQuery() {
   private List<NexmarkQuery> createSqlQueries() {
     return Arrays.<NexmarkQuery> asList(
         new NexmarkSqlQuery(configuration, new SqlQuery0()),
-        new NexmarkSqlQuery(configuration, new SqlQuery1()));
+        new NexmarkSqlQuery(configuration, new SqlQuery1()),
+        new NexmarkSqlQuery(configuration, new 
SqlQuery2(configuration.auctionSkip)));
   }
 
   private List<NexmarkQuery> createJavaQueries() {
diff --git 
a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery2.java
 
b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery2.java
new file mode 100644
index 00000000000..959c71216ac
--- /dev/null
+++ 
b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery2.java
@@ -0,0 +1,77 @@
+/*
+ * 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.nexmark.queries.sql;
+
+import static 
org.apache.beam.sdk.nexmark.model.sql.adapter.ModelAdaptersMapping.ADAPTERS;
+import static org.apache.beam.sdk.nexmark.queries.NexmarkQuery.IS_BID;
+
+import org.apache.beam.sdk.coders.BeamRecordCoder;
+import org.apache.beam.sdk.extensions.sql.BeamSql;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.sql.ToBeamRecord;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.BeamRecord;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Query 2, 'Filtering. Find bids with specific auction ids and show their bid 
price.
+ * In CQL syntax:
+ *
+ * <pre>
+ * SELECT Rstream(auction, price)
+ * FROM Bid [NOW]
+ * WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 
OR auction = 2087;
+ * </pre>
+ *
+ * <p>As written that query will only yield a few hundred results over event 
streams of
+ * arbitrary size. To make it more interesting we instead choose bids for every
+ * {@code skipFactor}'th auction.
+ */
+public class SqlQuery2 extends PTransform<PCollection<Event>, 
PCollection<BeamRecord>> {
+
+  private static final String QUERY_TEMPLATE =
+      "SELECT auction, bidder, price, dateTime, extra  FROM PCOLLECTION "
+          + " WHERE MOD(auction, %d) = 0";
+
+  private final BeamSql.SimpleQueryTransform query;
+
+  public SqlQuery2(long skipFactor) {
+    super("SqlQuery2");
+
+    String queryString = String.format(QUERY_TEMPLATE, skipFactor);
+    query = BeamSql.query(queryString);
+  }
+
+  @Override
+  public PCollection<BeamRecord> expand(PCollection<Event> allEvents) {
+    BeamRecordCoder bidRecordCoder = getBidRecordCoder();
+
+    PCollection<BeamRecord> bidEventsRecords = allEvents
+        .apply(Filter.by(IS_BID))
+        .apply(ToBeamRecord.parDo())
+        .setCoder(bidRecordCoder);
+
+    return bidEventsRecords.apply(query);
+  }
+
+  private BeamRecordCoder getBidRecordCoder() {
+    return ADAPTERS.get(Bid.class).getRecordType().getRecordCoder();
+  }
+}
diff --git 
a/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery2Test.java
 
b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery2Test.java
new file mode 100644
index 00000000000..2dc57209395
--- /dev/null
+++ 
b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery2Test.java
@@ -0,0 +1,113 @@
+/*
+ * 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.nexmark.queries.sql;
+
+import static 
org.apache.beam.sdk.nexmark.model.sql.adapter.ModelAdaptersMapping.ADAPTERS;
+
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.sql.adapter.ModelFieldsAdapter;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.BeamRecord;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Unit tests for {@link SqlQuery2}.
+ */
+public class SqlQuery2Test {
+
+  private static final ModelFieldsAdapter<Bid> BID_ADAPTER = 
ADAPTERS.get(Bid.class);
+
+  private static final List<Bid> BIDS = ImmutableList.of(
+      newBid(1L),
+      newBid(2L),
+      newBid(3L),
+      newBid(4L),
+      newBid(5L),
+      newBid(6L),
+      newBid(7L),
+      newBid(8L));
+
+  private static final List<Event> BIDS_EVENTS = ImmutableList.of(
+      new Event(BIDS.get(0)),
+      new Event(BIDS.get(1)),
+      new Event(BIDS.get(2)),
+      new Event(BIDS.get(3)),
+      new Event(BIDS.get(4)),
+      new Event(BIDS.get(5)),
+      new Event(BIDS.get(6)),
+      new Event(BIDS.get(7)));
+
+  private static final List<BeamRecord> BIDS_EVEN_RECORDS = ImmutableList.of(
+      newBidRecord(BIDS.get(1)),
+      newBidRecord(BIDS.get(3)),
+      newBidRecord(BIDS.get(5)),
+      newBidRecord(BIDS.get(7)));
+
+  private static final List<BeamRecord> BIDS_EVERY_THIRD_RECORD = 
ImmutableList.of(
+      newBidRecord(BIDS.get(2)),
+      newBidRecord(BIDS.get(5)));
+
+
+  @Rule public TestPipeline testPipeline = TestPipeline.create();
+
+  @Test
+  public void testSkipsEverySecondElement() throws Exception {
+    PCollection<Event> bids =
+        PBegin
+            .in(testPipeline)
+            .apply(Create.of(BIDS_EVENTS).withCoder(Event.CODER));
+
+    PAssert
+        .that(bids.apply(new SqlQuery2(2)))
+        .containsInAnyOrder(BIDS_EVEN_RECORDS);
+
+    testPipeline.run();
+  }
+
+  @Test
+  public void testSkipsEveryThirdElement() throws Exception {
+    PCollection<Event> bids =
+        PBegin
+            .in(testPipeline)
+            .apply(Create.of(BIDS_EVENTS).withCoder(Event.CODER));
+
+    PAssert
+        .that(bids.apply(new SqlQuery2(3)))
+        .containsInAnyOrder(BIDS_EVERY_THIRD_RECORD);
+
+    testPipeline.run();
+  }
+
+  private static Bid newBid(long id) {
+    return new Bid(id, 3L, 100L, 432342L + id, "extra_" + id);
+  }
+
+  private static BeamRecord newBidRecord(Bid bid) {
+    return new BeamRecord(BID_ADAPTER.getRecordType(), 
BID_ADAPTER.getFieldsValues(bid));
+  }
+
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> [Nexmark][SQL] Implement supported queries
> ------------------------------------------
>
>                 Key: BEAM-3182
>                 URL: https://issues.apache.org/jira/browse/BEAM-3182
>             Project: Beam
>          Issue Type: Sub-task
>          Components: dsl-sql
>            Reporter: Anton Kedin
>            Assignee: Anton Kedin
>            Priority: Major
>
> Implement all queries which can be run with current SQL features and Nexmark 
> infrastructure.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to