You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/05/03 23:43:00 UTC

[jira] [Work logged] (BEAM-4162) Wire up PubsubIO+JSON to Beam SQL

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

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

                Author: ASF GitHub Bot
            Created on: 03/May/18 23:42
            Start Date: 03/May/18 23:42
    Worklog Time Spent: 10m 
      Work Description: rangadi commented on a change in pull request #5253: [BEAM-4162][SQL] Wire up PubsubIO to SQL
URL: https://github.com/apache/beam/pull/5253#discussion_r185964760
 
 

 ##########
 File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubIOJsonTable.java
 ##########
 @@ -0,0 +1,222 @@
+/*
+ * 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.meta.provider.pubsub;
+
+import static com.google.api.client.util.DateTime.parseRfc3339;
+import static org.apache.beam.sdk.schemas.Schema.TypeName.DATETIME;
+import static org.apache.beam.sdk.util.JsonToRowUtils.newObjectMapperWith;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.auto.value.AutoValue;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.extensions.sql.BeamSqlTable;
+import org.apache.beam.sdk.extensions.sql.impl.schema.BeamIOType;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.RowJsonDeserializer;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.joda.time.DateTime;
+
+/**
+ * <i>Experimental</i>
+ *
+ * <p>Wraps the {@link PubsubIO} with JSON messages into {@link BeamSqlTable}.
+ *
+ * <p>This enables {@link PubsubIO} registration in Beam SQL environment as a table, including DDL
+ * support.
+ *
+ * <p>For example:
+ * <pre>
+ *
+ *  CREATE TABLE topic (name VARCHAR, age INTEGER)
+ *     TYPE 'pubsub'
+ *     LOCATION projects/&lt;GCP project id&gt;/topics/&lt;topic name&gt;
+ *     TBLPROPERTIES '{ \"timestampAttributeKey\" : &lt;timestamp attribute&gt; }';
+ *
+ *   SELECT name, age FROM topic;
+ *
+ * </pre>
+ */
+@AutoValue
+@Internal
+@Experimental
+abstract class PubsubIOJsonTable implements BeamSqlTable, Serializable {
+
+  /**
+   * Schema of the pubsubs message payload.
+   *
+   * <p>Only UTF-8 flat JSON objects are supported at the moment.
+   */
+  abstract Schema getPayloadSchema();
+
+  /**
+   * Attribute key of the Pubsub message from which to extract the event timestamp.
+   *
+   * <p>This attribute has to conform to the same requirements as in {@link
+   * PubsubIO.Read.Builder#withTimestampAttribute}.
+   *
+   * <p>Short version: it has to be either millis since epoch or string in RFC 3339 format.
+   */
+  abstract String getTimestampAttribute();
+
+  /**
+   * Pubsub topic name.
+   *
+   * <p>Topic is the only way to specify the Pubsub source. Explicitly specifying the subscription
+   * is not supported at the moment. Subscriptions are automatically created an managed.
+   */
+  abstract String getTopic();
+
+  static Builder builder() {
+    return new AutoValue_PubsubIOJsonTable.Builder();
+  }
+
+  /**
+   * Table schema.
+   *
+   * <p>Inherited from {@link BeamSqlTable}. Different from {@link #getPayloadSchema()},
+   * includes timestamp attribute.
+   */
+   public abstract Schema getSchema();
+
+  @Override
+  public BeamIOType getSourceType() {
+    return BeamIOType.UNBOUNDED;
+  }
+
+  @Override
+  public PCollection<Row> buildIOReader(Pipeline pipeline) {
+    return
+        PBegin
+            .in(pipeline)
+            .apply("readFromPubsub", readMessagesWithAttributes())
+            .apply("parseTimestampAttribute", parseTimestampAttribute())
+            .apply("parseJsonPayload", parseJsonPayload())
+            .setCoder(getSchema().getRowCoder());
+  }
+
+  private PubsubIO.Read<PubsubMessage> readMessagesWithAttributes() {
+    return
+        PubsubIO
+            .readMessagesWithAttributes()
+            .fromTopic(getTopic())
+            .withTimestampAttribute(getTimestampAttribute());
+  }
+
+  private ParDo.SingleOutput<PubsubMessage, KV<DateTime, PubsubMessage>> parseTimestampAttribute() {
+    return
+        ParDo.of(new DoFn<PubsubMessage, KV<DateTime, PubsubMessage>>() {
+          @ProcessElement
+          public void processElement(ProcessContext context) {
+            PubsubMessage pubsubMessage = context.element();
+            long msSinceEpoch = asMsSinceEpoch(pubsubMessage.getAttribute(getTimestampAttribute()));
 
 Review comment:
   What is the difference between this and 
   `msSinceEpoch = context.timestamp().getMillis()` ? 

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


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

    Worklog Id:     (was: 98079)
    Time Spent: 1h  (was: 50m)

> Wire up PubsubIO+JSON to Beam SQL
> ---------------------------------
>
>                 Key: BEAM-4162
>                 URL: https://issues.apache.org/jira/browse/BEAM-4162
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql
>            Reporter: Anton Kedin
>            Assignee: Anton Kedin
>            Priority: Major
>          Time Spent: 1h
>  Remaining Estimate: 0h
>
> Read JSON messages from Pubsub, convert them to Rows (BEAM-4160), wire up to Beam SQL.
>  
> Use publication time as event timestamp



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