[ https://issues.apache.org/jira/browse/BEAM-4808?focusedWorklogId=127958&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-127958 ]
ASF GitHub Bot logged work on BEAM-4808: ---------------------------------------- Author: ASF GitHub Bot Created on: 26/Jul/18 23:00 Start Date: 26/Jul/18 23:00 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_r205626293 ########## File path: sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java ########## @@ -0,0 +1,302 @@ +/* + * 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.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.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.io.gcp.pubsub.PubsubMessage; +import org.apache.beam.sdk.io.gcp.pubsub.TestPubsub; +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 final Integer numberOfThreads = 4; + private static String createPubsubTableStatement; + private static String readFromPubsub; + private static String filterForSouthManhattan; + private static String slidingWindowQuery; + private static String fixedWindowQuery; + private static PubsubMessageJSONStringConstructor constructor; + private static ExecutorService pool; + private static final String publicTopic = "projects/pubsub-public-data/topics/taxirides-realtime"; + + @BeforeClass + public static void setUp() { + pool = Executors.newFixedThreadPool(numberOfThreads); + + 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\"}';"; + + readFromPubsub = + "SELECT event_timestamp, taxi_rides.payload.ride_status, taxi_rides.payload.latitude, " + + "taxi_rides.payload.longitude from taxi_rides LIMIT 3;"; + + filterForSouthManhattan = + "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;"; + + fixedWindowQuery = + "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;"; + + slidingWindowQuery = + "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"; + + constructor = + new PubsubMessageJSONStringConstructor( + "ride_id", + "point_idx", + "latitude", + "longitude", + "meter_reading", + "meter_increment", + "ride_status", + "passenger_count"); + } + + @Test + public void testSelectFromPubsub() throws Exception { + Future<List<List<String>>> expectedResult = + pool.submit( + (Callable) + () -> { + String[] args = + buildArgs( + String.format(createPubsubTableStatement, eventsTopic.topicPath()), + readFromPubsub); + + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + BeamSqlLine.runSqlLine(args, null, outputStream, null); + return toLines(outputStream); + }); + + // Wait 10 sec to allow creating a subscription. + Thread.sleep(10 * 1000); Review comment: I thought about the waiting for solution but eventually choose this one because e2e tests talk to real systems. Waiting for 10 sec could be flaky but monitoring the creation of subscription could also have infinite waiting issue. In order to avoid long or infinite wait for subscription, the separate thread would require a timeout setting, which is an variant of this 10 sec sleep, but just more complicated. I would say how flaky it is will be decided by Pubsub. If 99.9% subscription can be created within 10 second, then current idea should be ok. ---------------------------------------------------------------- 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: 127958) Time Spent: 8h (was: 7h 50m) > 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: 8h > Remaining Estimate: 0h > -- This message was sent by Atlassian JIRA (v7.6.3#76005)