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

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

                Author: ASF GitHub Bot
            Created on: 30/Jul/18 22:11
            Start Date: 30/Jul/18 22:11
    Worklog Time Spent: 10m 
      Work Description: amaliujia commented on a change in pull request #6006: 
[BEAM-4808][SQL] add e2e test for BeamSqlLine.
URL: https://github.com/apache/beam/pull/6006#discussion_r206337441
 
 

 ##########
 File path: 
sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java
 ##########
 @@ -0,0 +1,328 @@
+/*
+ * 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.jdbc;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static 
org.apache.beam.sdk.extensions.sql.jdbc.BeamSqlLineTestingUtils.buildArgs;
+import static 
org.apache.beam.sdk.extensions.sql.jdbc.BeamSqlLineTestingUtils.toLines;
+import static 
org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.projectPathFromPath;
+import static org.hamcrest.CoreMatchers.everyItem;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import com.alibaba.fastjson.JSONObject;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Arrays;
+import java.util.List;
+import java.util.TimeZone;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
+import org.apache.beam.sdk.io.gcp.pubsub.TestPubsub;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.hamcrest.collection.IsIn;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/** BeamSqlLine integration tests. */
+public class BeamSqlLineIT implements Serializable {
+
+  @Rule public transient TestPubsub eventsTopic = TestPubsub.create();
+
+  private static String project =
+      TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject();
+  private static String createPubsubTableStatement;
+  private static String setProject;
+  private static PubsubMessageJSONStringConstructor constructor;
+  private static final String publicTopic = 
"projects/pubsub-public-data/topics/taxirides-realtime";
+
+  @BeforeClass
+  public static void setUp() {
+    setProject = String.format("SET project = '%s';", project);
+
+    createPubsubTableStatement =
+        "CREATE TABLE taxi_rides (\n"
+            + "         event_timestamp TIMESTAMP,\n"
+            + "         attributes MAP<VARCHAR, VARCHAR>,\n"
+            + "         payload ROW<\n"
+            + "           ride_id VARCHAR,\n"
+            + "           point_idx INT,\n"
+            + "           latitude DOUBLE,\n"
+            + "           longitude DOUBLE,\n"
+            + "           meter_reading DOUBLE,\n"
+            + "           meter_increment DOUBLE,\n"
+            + "           ride_status VARCHAR,\n"
+            + "           passenger_count TINYINT>)\n"
+            + "       TYPE pubsub \n"
+            + "       LOCATION '%s'\n"
+            + "       TBLPROPERTIES '{\"timestampAttributeKey\": \"ts\"}';";
+
+    constructor =
+        new PubsubMessageJSONStringConstructor(
+            "ride_id",
+            "point_idx",
+            "latitude",
+            "longitude",
+            "meter_reading",
+            "meter_increment",
+            "ride_status",
+            "passenger_count");
+  }
+
+  @Test
+  public void testSelectFromPubsub() throws Exception {
+    ExecutorService pool = Executors.newFixedThreadPool(1);
+
+    Future<List<List<String>>> expectedResult =
+        pool.submit(
+            (Callable)
+                () -> {
+                  String[] args =
+                      buildArgs(
+                          String.format(createPubsubTableStatement, 
eventsTopic.topicPath()),
+                          setProject,
+                          "SELECT event_timestamp, 
taxi_rides.payload.ride_status, taxi_rides.payload.latitude, "
+                              + "taxi_rides.payload.longitude from taxi_rides 
LIMIT 3;");
+
+                  ByteArrayOutputStream outputStream = new 
ByteArrayOutputStream();
+                  BeamSqlLine.runSqlLine(args, null, outputStream, null);
+                  return toLines(outputStream);
+                });
+
+    checkIfTopicExists(eventsTopic, 10);
+
+    List<PubsubMessage> messages =
+        ImmutableList.of(
+            message(
+                convertTimestampToMillis("2018-07-01 21:25:20"),
+                constructor.construct("id1", 1, 40.702, -74.001, 1000, 10, 
"enroute", 2)),
+            message(
+                convertTimestampToMillis("2018-07-01 21:26:06"),
+                constructor.construct("id2", 2, 40.703, -74.002, 1000, 10, 
"enroute", 4)),
+            message(
+                convertTimestampToMillis("2018-07-02 13:26:06"),
+                constructor.construct("id3", 3, 30.0, -72.32324, 2000, 20, 
"enroute", 7)));
+
+    eventsTopic.publish(messages);
+
+    assertThat(
+        Arrays.asList(
+            Arrays.asList("2018-07-01 21:25:20", "enroute", "40.702", 
"-74.001"),
+            Arrays.asList("2018-07-01 21:26:06", "enroute", "40.703", 
"-74.002"),
+            Arrays.asList("2018-07-02 13:26:06", "enroute", "30.0", 
"-72.32324")),
+        everyItem(IsIn.isOneOf(expectedResult.get(30, 
TimeUnit.SECONDS).toArray())));
+  }
+
+  @Test
+  public void testFilterForSouthManhattan() throws Exception {
+    ExecutorService pool = Executors.newFixedThreadPool(1);
+
+    Future<List<List<String>>> expectedResult =
+        pool.submit(
+            (Callable)
+                () -> {
+                  String[] args =
+                      buildArgs(
+                          String.format(createPubsubTableStatement, 
eventsTopic.topicPath()),
+                          setProject,
+                          "SELECT event_timestamp, 
taxi_rides.payload.ride_status, \n"
+                              + "taxi_rides.payload.latitude, 
taxi_rides.payload.longitude from taxi_rides\n"
+                              + "       WHERE taxi_rides.payload.longitude > 
-74.747\n"
+                              + "         AND taxi_rides.payload.longitude < 
-73.969\n"
+                              + "         AND taxi_rides.payload.latitude > 
40.699\n"
+                              + "         AND taxi_rides.payload.latitude < 
40.720 LIMIT 2;");
+
+                  ByteArrayOutputStream outputStream = new 
ByteArrayOutputStream();
+                  BeamSqlLine.runSqlLine(args, null, outputStream, null);
+                  return toLines(outputStream);
+                });
+
+    checkIfTopicExists(eventsTopic, 10);
+
+    List<PubsubMessage> messages =
+        ImmutableList.of(
+            message(
+                convertTimestampToMillis("2018-07-01 21:25:20"),
+                constructor.construct("id1", 1, 40.701, -74.001, 1000, 10, 
"enroute", 2)),
+            message(
+                convertTimestampToMillis("2018-07-01 21:26:06"),
+                constructor.construct("id2", 2, 40.702, -74.002, 1000, 10, 
"enroute", 4)),
+            message(
+                convertTimestampToMillis("2018-07-02 13:26:06"),
+                constructor.construct("id3", 3, 30, -72.32324, 2000, 20, 
"enroute", 7)),
+            message(
+                convertTimestampToMillis("2018-07-02 14:28:22"),
+                constructor.construct("id4", 4, 34, -73.32324, 2000, 20, 
"enroute", 8)));
+
+    eventsTopic.publish(messages);
+
+    assertThat(
+        Arrays.asList(
+            Arrays.asList("2018-07-01 21:25:20", "enroute", "40.701", 
"-74.001"),
+            Arrays.asList("2018-07-01 21:26:06", "enroute", "40.702", 
"-74.002")),
+        everyItem(IsIn.isOneOf(expectedResult.get(30, 
TimeUnit.SECONDS).toArray())));
+  }
+
+  @Test
+  public void testFixedWindow() throws Exception {
+    ExecutorService pool = Executors.newFixedThreadPool(1);
+
+    Future<List<List<String>>> expectedResult =
+        pool.submit(
+            (Callable)
+                () -> {
+                  String[] args =
+                      buildArgs(
+                          String.format(createPubsubTableStatement, 
publicTopic),
+                          setProject,
+                          "WITH geo_cells AS (\n"
+                              + "         SELECT 
FLOOR(taxi_rides.payload.latitude / 0.05) * 0.05 AS reduced_lat,\n"
+                              + "                
FLOOR(taxi_rides.payload.longitude / 0.05) * 0.05 AS reduced_lon,\n"
+                              + "                taxi_rides.event_timestamp\n"
+                              + "         FROM taxi_rides)\n"
+                              + "       SELECT COUNT(*) as num_events,\n"
+                              + "              geo_cells.reduced_lat,\n"
+                              + "              geo_cells.reduced_lon, \n"
+                              + "              
TUMBLE_START(geo_cells.event_timestamp, INTERVAL '1' SECOND)\n"
+                              + "       FROM geo_cells    \n"
+                              + "       GROUP BY geo_cells.reduced_lat,\n"
+                              + "                geo_cells.reduced_lon,\n"
+                              + "                
TUMBLE(geo_cells.event_timestamp, INTERVAL '1' SECOND)\n"
+                              + "       LIMIT 2;");
+                  ByteArrayOutputStream outputStream = new 
ByteArrayOutputStream();
+                  BeamSqlLine.runSqlLine(args, null, outputStream, null);
+                  return toLines(outputStream);
+                });
+
+    // LIMIT 2 is supposed to be parsed into 6 lines.
+    assertTrue(expectedResult.get().size() == 6);
+  }
+
+  @Test
+  public void testSlidingWindow() throws Exception {
+    ExecutorService pool = Executors.newFixedThreadPool(1);
+
+    Future<List<List<String>>> expectedResult =
+        pool.submit(
+            (Callable)
+                () -> {
+                  String[] args =
+                      buildArgs(
+                          String.format(createPubsubTableStatement, 
publicTopic),
+                          setProject,
+                          "SELECT COUNT(*) AS num_events,\n"
+                              + "              
SUM(taxi_rides.payload.meter_increment) as revenue,\n"
+                              + "              HOP_END(\n"
+                              + "                 taxi_rides.event_timestamp, 
\n"
+                              + "                 INTERVAL '1' SECOND, \n"
+                              + "                 INTERVAL '2' SECOND) as 
minute_end\n"
+                              + "              FROM taxi_rides\n"
+                              + "              GROUP BY HOP(\n"
+                              + "                 
taxi_rides.event_timestamp,\n"
+                              + "                 INTERVAL '1' SECOND, \n"
+                              + "                 INTERVAL '2' SECOND) LIMIT 
2");
+                  ByteArrayOutputStream outputStream = new 
ByteArrayOutputStream();
+                  BeamSqlLine.runSqlLine(args, null, outputStream, null);
+                  return toLines(outputStream);
+                });
+
+    // LIMIT 2 is supposed to be parsed into 6 lines.
+    assertTrue(expectedResult.get().size() == 6);
+  }
+
+  /**
+   * Check if topic exists {@param attempts} times. There is 1 sec wait time 
between two attempts.
+   *
+   * @param attempts number of attempts. Has to be a value > 0
+   */
+  private void checkIfTopicExists(TestPubsub eventsTopic, int attempts)
 
 Review comment:
   Gotcha. Since the Duration here does not mean exact time but an 
approximation, I can go with Duration.

----------------------------------------------------------------
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: 128982)
    Time Spent: 15h 40m  (was: 15.5h)

> Add an integration test for BeamSqlLine
> ---------------------------------------
>
>                 Key: BEAM-4808
>                 URL: https://issues.apache.org/jira/browse/BEAM-4808
>             Project: Beam
>          Issue Type: Improvement
>          Components: dsl-sql
>            Reporter: Rui Wang
>            Assignee: Rui Wang
>            Priority: Major
>          Time Spent: 15h 40m
>  Remaining Estimate: 0h
>




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

Reply via email to