[ 
https://issues.apache.org/jira/browse/BEAM-5817?focusedWorklogId=169098&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-169098
 ]

ASF GitHub Bot logged work on BEAM-5817:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 24/Nov/18 03:44
            Start Date: 24/Nov/18 03:44
    Worklog Time Spent: 10m 
      Work Description: kennknowles closed pull request #7114: [BEAM-5817] Add 
Nexmark SqlBoundedSideInputJoin
URL: https://github.com/apache/beam/pull/7114
 
 
   

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/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
 
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
index 9f840bdb243..1b02506979a 100644
--- 
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
+++ 
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
@@ -157,9 +157,6 @@ private boolean isSideInputJoin() {
       PCollection<Row> leftRows = pinput.get(0);
       PCollection<Row> rightRows = pinput.get(1);
 
-      verifySupportedTrigger(leftRows);
-      verifySupportedTrigger(rightRows);
-
       WindowFn leftWinFn = leftRows.getWindowingStrategy().getWindowFn();
       WindowFn rightWinFn = rightRows.getWindowingStrategy().getWindowFn();
 
@@ -198,6 +195,9 @@ private boolean isSideInputJoin() {
       if ((leftRows.isBounded() == PCollection.IsBounded.BOUNDED
               && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)
           || (leftRows.isBounded() == UNBOUNDED && rightRows.isBounded() == 
UNBOUNDED)) {
+        verifySupportedTrigger(leftRows);
+        verifySupportedTrigger(rightRows);
+
         try {
           leftWinFn.verifyCompatibility(rightWinFn);
         } catch (IncompatibleWindowException e) {
diff --git 
a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/BoundedSideInputJoin.java
 
b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/BoundedSideInputJoin.java
index c5f0f58472d..700fae72f1c 100644
--- 
a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/BoundedSideInputJoin.java
+++ 
b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/BoundedSideInputJoin.java
@@ -42,7 +42,7 @@
   private final NexmarkConfiguration configuration;
 
   public BoundedSideInputJoin(NexmarkConfiguration configuration) {
-    super("JoinToFiles");
+    super("BoundedSideInputJoin");
     this.configuration = configuration;
   }
 
diff --git 
a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/BoundedSideInputJoinModel.java
 
b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/BoundedSideInputJoinModel.java
index f36fee97a9a..884df89394d 100644
--- 
a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/BoundedSideInputJoinModel.java
+++ 
b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/BoundedSideInputJoinModel.java
@@ -76,6 +76,6 @@ public BoundedSideInputJoinModel(NexmarkConfiguration 
configuration) {
 
   @Override
   protected Collection<String> toCollection(Iterator<TimestampedValue<Bid>> 
itr) {
-    return toValueTimestamp(itr);
+    return toValue(itr);
   }
 }
diff --git 
a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlBoundedSideInputJoin.java
 
b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlBoundedSideInputJoin.java
new file mode 100644
index 00000000000..a91c6ee8e76
--- /dev/null
+++ 
b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlBoundedSideInputJoin.java
@@ -0,0 +1,100 @@
+/*
+ * 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 com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.sdk.extensions.sql.SqlTransform;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.sql.SelectEvent;
+import org.apache.beam.sdk.nexmark.queries.NexmarkQueryTransform;
+import org.apache.beam.sdk.nexmark.queries.NexmarkQueryUtil;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.transforms.Convert;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+
+/** Basic stream enrichment: join a stream to a bounded side input. */
+public class SqlBoundedSideInputJoin extends NexmarkQueryTransform<Bid> {
+  private final String query;
+
+  public SqlBoundedSideInputJoin(NexmarkConfiguration configuration) {
+    super("SqlBoundedSideInputJoin");
+
+    // Notes on the sensitivities of our parsing and planning:
+    //  - cannot directly join MOD(bidder, x) = side.id because only equijoins 
on col refs allowed,
+    //    so we need a WITH clause or subquery
+    //  - must have the CAST inside the WITH clause for the same reason, 
otherwise the cast
+    //    occurs in the join condition CAST(side_id AS BIGINT) = side.id
+    query =
+        String.format(
+            "WITH bid_with_side (auction, bidder, price, dateTime, extra, 
side_id) AS (%n"
+                + "  SELECT *, CAST(MOD(bidder, %d) AS BIGINT) side_id FROM 
bid%n"
+                + ")%n"
+                + " SELECT bid_with_side.auction%n"
+                + ", bid_with_side.bidder%n"
+                + ", bid_with_side.price%n"
+                + ", bid_with_side.dateTime%n"
+                + ", side.extra%n"
+                + " FROM bid_with_side, side%n"
+                + " WHERE bid_with_side.side_id = side.id",
+            configuration.sideInputRowCount);
+  }
+
+  @Override
+  public boolean needsSideInput() {
+    return true;
+  }
+
+  @Override
+  public PCollection<Bid> expand(PCollection<Event> events) {
+    PCollection<Row> bids =
+        events
+            .apply(Filter.by(NexmarkQueryUtil.IS_BID))
+            .apply(getName() + ".SelectEvent", new 
SelectEvent(Event.Type.BID));
+
+    checkState(getSideInput() != null, "Configuration error: side input is 
null");
+
+    TupleTag<Row> sideTag = new TupleTag<Row>("side") {};
+    TupleTag<Row> bidTag = new TupleTag<Row>("bid") {};
+
+    Schema schema =
+        Schema.of(
+            Schema.Field.of("id", Schema.FieldType.INT64),
+            Schema.Field.of("extra", Schema.FieldType.STRING));
+
+    PCollection<Row> sideRows =
+        getSideInput()
+            .setSchema(
+                schema,
+                kv -> Row.withSchema(schema).addValues(kv.getKey(), 
kv.getValue()).build(),
+                row -> KV.of(row.getInt64("id"), row.getString("extra")))
+            .apply("SideToRows", Convert.toRows());
+
+    return PCollectionTuple.of(bidTag, bids)
+        .and(sideTag, sideRows)
+        .apply(SqlTransform.query(query))
+        .apply("ResultToBid", Convert.fromRows(Bid.class));
+  }
+}
diff --git 
a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlBoundedSideInputJoinTest.java
 
b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlBoundedSideInputJoinTest.java
new file mode 100644
index 00000000000..ac8d2bfab58
--- /dev/null
+++ 
b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlBoundedSideInputJoinTest.java
@@ -0,0 +1,213 @@
+/*
+ * 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.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.collect.Iterables;
+import java.util.Random;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.io.fs.ResourceId;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.nexmark.queries.BoundedSideInputJoinModel;
+import org.apache.beam.sdk.nexmark.queries.NexmarkQuery;
+import org.apache.beam.sdk.nexmark.queries.NexmarkQueryModel;
+import org.apache.beam.sdk.nexmark.queries.NexmarkQueryTransform;
+import org.apache.beam.sdk.nexmark.queries.NexmarkQueryUtil;
+import org.apache.beam.sdk.testing.NeedsRunner;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Flatten;
+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.TimestampedValue;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Test the various NEXMark queries yield results coherent with their models. 
*/
+@RunWith(JUnit4.class)
+public class SqlBoundedSideInputJoinTest {
+
+  @Rule public TestPipeline p = TestPipeline.create();
+
+  @Before
+  public void setupPipeline() {
+    NexmarkUtils.setupPipeline(NexmarkUtils.CoderStrategy.HAND, p);
+  }
+
+  /** Test {@code query} matches {@code model}. */
+  private <T extends KnownSize> void queryMatchesModel(
+      String name,
+      NexmarkConfiguration config,
+      NexmarkQueryTransform<T> query,
+      NexmarkQueryModel<T> model,
+      boolean streamingMode)
+      throws Exception {
+
+    ResourceId sideInputResourceId =
+        FileSystems.matchNewResource(
+            String.format(
+                "%s/JoinToFiles-%s", p.getOptions().getTempLocation(), new 
Random().nextInt()),
+            false);
+    config.sideInputUrl = sideInputResourceId.toString();
+
+    try {
+      PCollection<KV<Long, String>> sideInput = 
NexmarkUtils.prepareSideInput(p, config);
+      query.setSideInput(sideInput);
+
+      PCollection<Event> events =
+          p.apply(
+              name + ".Read",
+              streamingMode
+                  ? NexmarkUtils.streamEventsSource(config)
+                  : NexmarkUtils.batchEventsSource(config));
+
+      PCollection<TimestampedValue<T>> results =
+          (PCollection<TimestampedValue<T>>) events.apply(new 
NexmarkQuery<>(config, query));
+      PAssert.that(results).satisfies(model.assertionFor());
+      PipelineResult result = p.run();
+      result.waitUntilFinish();
+    } finally {
+      NexmarkUtils.cleanUpSideInput(config);
+    }
+  }
+
+  /**
+   * A smoke test that the count of input bids and outputs are the same, to 
help diagnose flakiness
+   * in more complex tests.
+   */
+  @Test
+  @Category(NeedsRunner.class)
+  public void inputOutputSameEvents() throws Exception {
+    NexmarkConfiguration config = NexmarkConfiguration.DEFAULT.copy();
+    config.sideInputType = NexmarkUtils.SideInputType.DIRECT;
+    config.numEventGenerators = 1;
+    config.numEvents = 5000;
+    config.sideInputRowCount = 10;
+    config.sideInputNumShards = 3;
+    PCollection<KV<Long, String>> sideInput = NexmarkUtils.prepareSideInput(p, 
config);
+
+    try {
+      PCollection<Event> input = 
p.apply(NexmarkUtils.batchEventsSource(config));
+      PCollection<Bid> justBids = input.apply(NexmarkQueryUtil.JUST_BIDS);
+      PCollection<Long> bidCount = justBids.apply("Count Bids", 
Count.globally());
+
+      NexmarkQueryTransform<Bid> query = new SqlBoundedSideInputJoin(config);
+      query.setSideInput(sideInput);
+
+      PCollection<TimestampedValue<Bid>> output =
+          (PCollection<TimestampedValue<Bid>>) input.apply(new 
NexmarkQuery(config, query));
+      PCollection<Long> outputCount = output.apply("Count outputs", 
Count.globally());
+
+      
PAssert.that(PCollectionList.of(bidCount).and(outputCount).apply(Flatten.pCollections()))
+          .satisfies(
+              counts -> {
+                assertThat(Iterables.size(counts), equalTo(2));
+                assertThat(Iterables.get(counts, 0), greaterThan(0L));
+                assertThat(Iterables.get(counts, 0), 
equalTo(Iterables.get(counts, 1)));
+                return null;
+              });
+      p.run();
+    } finally {
+      NexmarkUtils.cleanUpSideInput(config);
+    }
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void queryMatchesModelBatchDirect() throws Exception {
+    NexmarkConfiguration config = NexmarkConfiguration.DEFAULT.copy();
+    config.sideInputType = NexmarkUtils.SideInputType.DIRECT;
+    config.numEventGenerators = 1;
+    config.numEvents = 5000;
+    config.sideInputRowCount = 10;
+    config.sideInputNumShards = 3;
+
+    queryMatchesModel(
+        "SqlBoundedSideInputJoinTestBatch",
+        config,
+        new SqlBoundedSideInputJoin(config),
+        new BoundedSideInputJoinModel(config),
+        false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void queryMatchesModelStreamingDirect() throws Exception {
+    NexmarkConfiguration config = NexmarkConfiguration.DEFAULT.copy();
+    config.sideInputType = NexmarkUtils.SideInputType.DIRECT;
+    config.numEventGenerators = 1;
+    config.numEvents = 5000;
+    config.sideInputRowCount = 10;
+    config.sideInputNumShards = 3;
+    queryMatchesModel(
+        "SqlBoundedSideInputJoinTestStreaming",
+        config,
+        new SqlBoundedSideInputJoin(config),
+        new BoundedSideInputJoinModel(config),
+        true);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void queryMatchesModelBatchCsv() throws Exception {
+    NexmarkConfiguration config = NexmarkConfiguration.DEFAULT.copy();
+    config.sideInputType = NexmarkUtils.SideInputType.CSV;
+    config.numEventGenerators = 1;
+    config.numEvents = 5000;
+    config.sideInputRowCount = 10;
+    config.sideInputNumShards = 3;
+
+    queryMatchesModel(
+        "SqlBoundedSideInputJoinTestBatch",
+        config,
+        new SqlBoundedSideInputJoin(config),
+        new BoundedSideInputJoinModel(config),
+        false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void queryMatchesModelStreamingCsv() throws Exception {
+    NexmarkConfiguration config = NexmarkConfiguration.DEFAULT.copy();
+    config.sideInputType = NexmarkUtils.SideInputType.CSV;
+    config.numEventGenerators = 1;
+    config.numEvents = 5000;
+    config.sideInputRowCount = 10;
+    config.sideInputNumShards = 3;
+    queryMatchesModel(
+        "SqlBoundedSideInputJoinTestStreaming",
+        config,
+        new SqlBoundedSideInputJoin(config),
+        new BoundedSideInputJoinModel(config),
+        true);
+  }
+}


 

----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 169098)
    Time Spent: 7h 40m  (was: 7.5h)

> Nexmark test of joining stream to files
> ---------------------------------------
>
>                 Key: BEAM-5817
>                 URL: https://issues.apache.org/jira/browse/BEAM-5817
>             Project: Beam
>          Issue Type: New Feature
>          Components: examples-nexmark
>            Reporter: Kenneth Knowles
>            Assignee: Kenneth Knowles
>            Priority: Major
>          Time Spent: 7h 40m
>  Remaining Estimate: 0h
>
> Nexmark is a convenient framework for testing the use case of large scale 
> stream enrichment. One way is joining a stream to files, and it can be tested 
> via any source that Nexmark supports.



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

Reply via email to