[
https://issues.apache.org/jira/browse/BEAM-3182?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16328315#comment-16328315
]
ASF GitHub Bot commented on BEAM-3182:
--------------------------------------
kennknowles closed pull request #4129: [BEAM-3182][Nexmark][SQL] Implement
query 1
URL: https://github.com/apache/beam/pull/4129
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 a4b5c5aaf5c..6e24207951a 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
@@ -82,6 +82,7 @@
import org.apache.beam.sdk.nexmark.queries.Query9Model;
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.testing.PAssert;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.ParDo;
@@ -1198,7 +1199,8 @@ private NexmarkQuery getNexmarkQuery() {
private List<NexmarkQuery> createSqlQueries() {
return Arrays.<NexmarkQuery> asList(
- new NexmarkSqlQuery(configuration, new SqlQuery0()));
+ new NexmarkSqlQuery(configuration, new SqlQuery0()),
+ new NexmarkSqlQuery(configuration, new SqlQuery1()));
}
private List<NexmarkQuery> createJavaQueries() {
diff --git
a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery1.java
b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery1.java
new file mode 100644
index 00000000000..1c44558545b
--- /dev/null
+++
b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery1.java
@@ -0,0 +1,81 @@
+/*
+ * 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.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.BeamRecord;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Query 1, 'Currency Conversion'. Convert each bid value from dollars to
euros.
+ * In CQL syntax:
+ *
+ * <pre>
+ * SELECT Istream(auction, DOLTOEUR(price), bidder, datetime)
+ * FROM bid [ROWS UNBOUNDED];
+ * </pre>
+ *
+ * <p>To make things more interesting, allow the 'currency conversion' to be
arbitrarily
+ * slowed down.
+ */
+public class SqlQuery1 extends PTransform<PCollection<Event>,
PCollection<BeamRecord>> {
+
+ private static final BeamSql.SimpleQueryTransform QUERY = BeamSql
+ .query("SELECT auction, bidder, DolToEur(price) as price, dateTime,
extra FROM PCOLLECTION")
+ .withUdf("DolToEur", new DolToEur());
+
+ /**
+ * Dollar to Euro conversion.
+ */
+ public static class DolToEur implements SerializableFunction<Long, Long> {
+ @Override
+ public Long apply(Long price) {
+ return (price * 89) / 100;
+ }
+ }
+
+ public SqlQuery1() {
+ super("SqlQuery1");
+ }
+
+ @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/SqlQuery1Test.java
b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery1Test.java
new file mode 100644
index 00000000000..2d90c9bf22d
--- /dev/null
+++
b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery1Test.java
@@ -0,0 +1,84 @@
+/*
+ * 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.junit.Assert.assertEquals;
+
+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.testing.TestStream;
+import org.apache.beam.sdk.values.BeamRecord;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Unit tests for {@link SqlQuery1}.
+ */
+public class SqlQuery1Test {
+
+ private static final Bid BID1_USD =
+ new Bid(5L, 3L, 100L, 43234234L, "extra1");
+
+ private static final Bid BID2_USD =
+ new Bid(6L, 4L, 500L, 13234234L, "extra2");
+
+ private static final Bid BID1_EUR =
+ new Bid(5L, 3L, 89L, 43234234L, "extra1");
+
+ private static final Bid BID2_EUR =
+ new Bid(6L, 4L, 445L, 13234234L, "extra2");
+
+ private static final ModelFieldsAdapter<Bid> BID_ADAPTER =
+ ADAPTERS.get(Bid.class);
+
+ private static final BeamRecord BID1_EUR_RECORD =
+ new BeamRecord(BID_ADAPTER.getRecordType(),
BID_ADAPTER.getFieldsValues(BID1_EUR));
+
+ private static final BeamRecord BID2_EUR_RECORD =
+ new BeamRecord(BID_ADAPTER.getRecordType(),
BID_ADAPTER.getFieldsValues(BID2_EUR));
+
+ @Rule
+ public TestPipeline testPipeline = TestPipeline.create();
+
+ @Test
+ public void testDolToEurConversion() {
+ SqlQuery1.DolToEur dolToEur = new SqlQuery1.DolToEur();
+ assertEquals(Long.valueOf(445), dolToEur.apply(500L));
+ }
+
+ @Test
+ public void testConvertsPriceToEur() throws Exception {
+ PCollection<Event> bids = testPipeline.apply(
+ TestStream.create(Event.CODER)
+ .addElements(new Event(BID1_USD))
+ .addElements(new Event(BID2_USD))
+ .advanceWatermarkToInfinity());
+
+ PAssert
+ .that(bids.apply(new SqlQuery1()))
+ .containsInAnyOrder(BID1_EUR_RECORD, BID2_EUR_RECORD);
+
+ testPipeline.run();
+ }
+}
----------------------------------------------------------------
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)