[ https://issues.apache.org/jira/browse/BEAM-4808?focusedWorklogId=128899&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-128899 ]
ASF GitHub Bot logged work on BEAM-4808: ---------------------------------------- Author: ASF GitHub Bot Created on: 30/Jul/18 19:37 Start Date: 30/Jul/18 19:37 Worklog Time Spent: 10m Work Description: akedin 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_r206256348 ########## 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); Review comment: nit: I would just keep only these 3 lines in the `submit()` call and move everything else outside, and also would create a method to hide it, something like `Future<List<List<String>>> resultLines = startWaitingForMessagesInBackground(args)` ---------------------------------------------------------------- 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: 128899) > 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: 12h 50m > Remaining Estimate: 0h > -- This message was sent by Atlassian JIRA (v7.6.3#76005)