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

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

                Author: ASF GitHub Bot
            Created on: 17/Apr/18 21:00
            Start Date: 17/Apr/18 21:00
    Worklog Time Spent: 10m 
      Work Description: robertwb closed pull request #4939: 
[BEAM-3856][BEAM-3854] Add prototype of Go streaming on Dataflow with PubSub
URL: https://github.com/apache/beam/pull/4939
 
 
   

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/go/examples/streaming_wordcap/wordcap.go 
b/sdks/go/examples/streaming_wordcap/wordcap.go
new file mode 100644
index 00000000000..045fb375577
--- /dev/null
+++ b/sdks/go/examples/streaming_wordcap/wordcap.go
@@ -0,0 +1,81 @@
+// 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.
+
+// streaming_wordcap is a toy streaming pipeline that uses PubSub. It
+// does the following:
+//    (1) create a topic and publish a few messages to it
+//    (2) start a streaming pipeline that converts the messages to
+//        upper case and logs the result.
+//
+// NOTE: it only runs on Dataflow and must be manually cancelled.
+package main
+
+import (
+       "context"
+       "flag"
+       "os"
+       "strings"
+
+       "github.com/apache/beam/sdks/go/pkg/beam"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/util/stringx"
+       "github.com/apache/beam/sdks/go/pkg/beam/io/pubsubio"
+       "github.com/apache/beam/sdks/go/pkg/beam/log"
+       "github.com/apache/beam/sdks/go/pkg/beam/options/gcpopts"
+       "github.com/apache/beam/sdks/go/pkg/beam/util/pubsubx"
+       "github.com/apache/beam/sdks/go/pkg/beam/x/beamx"
+       "github.com/apache/beam/sdks/go/pkg/beam/x/debug"
+)
+
+var (
+       input = flag.String("input", os.ExpandEnv("$USER-wordcap"), "Pubsub 
input topic.")
+)
+
+var (
+       data = []string{
+               "foo",
+               "bar",
+               "baz",
+       }
+)
+
+func main() {
+       flag.Parse()
+       beam.Init()
+
+       ctx := context.Background()
+       project := gcpopts.GetProject(ctx)
+
+       log.Infof(ctx, "Publishing %v messages to: %v", len(data), *input)
+
+       defer pubsubx.CleanupTopic(ctx, project, *input)
+       sub, err := pubsubx.Publish(ctx, project, *input, data...)
+       if err != nil {
+               log.Fatal(ctx, err)
+       }
+
+       log.Infof(ctx, "Running streaming wordcap with subscription: %v", 
sub.ID())
+
+       p := beam.NewPipeline()
+       s := p.Root()
+
+       col := pubsubio.Read(s, project, *input, 
&pubsubio.ReadOptions{Subscription: sub.ID()})
+       str := beam.ParDo(s, stringx.FromBytes, col)
+       cap := beam.ParDo(s, strings.ToUpper, str)
+       debug.Print(s, cap)
+
+       if err := beamx.Run(context.Background(), p); err != nil {
+               log.Exitf(ctx, "Failed to execute job: %v", err)
+       }
+}
diff --git a/sdks/go/pkg/beam/core/util/stringx/bytes.go 
b/sdks/go/pkg/beam/core/util/stringx/bytes.go
new file mode 100644
index 00000000000..b2110fa1072
--- /dev/null
+++ b/sdks/go/pkg/beam/core/util/stringx/bytes.go
@@ -0,0 +1,28 @@
+// 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 stringx contains utilities for working with strings. It
+// complements the standard "strings" package.
+package stringx
+
+// ToBytes converts a string to a byte slice.
+func ToBytes(s string) []byte {
+       return ([]byte)(s)
+}
+
+// FromBytes converts a byte slice to a string.
+func FromBytes(b []byte) string {
+       return (string)(b)
+}
diff --git a/sdks/go/pkg/beam/io/pubsubio/pubsubio.go 
b/sdks/go/pkg/beam/io/pubsubio/pubsubio.go
new file mode 100644
index 00000000000..fca98a906c7
--- /dev/null
+++ b/sdks/go/pkg/beam/io/pubsubio/pubsubio.go
@@ -0,0 +1,95 @@
+// 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 pubsubio provides access to PubSub on Dataflow streaming.
+// Experimental.
+package pubsubio
+
+import (
+       "reflect"
+
+       "github.com/apache/beam/sdks/go/pkg/beam"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/typex"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/util/protox"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+       "github.com/apache/beam/sdks/go/pkg/beam/io/pubsubio/v1"
+       "github.com/apache/beam/sdks/go/pkg/beam/util/pubsubx"
+       "github.com/golang/protobuf/proto"
+       pb "google.golang.org/genproto/googleapis/pubsub/v1"
+)
+
+func init() {
+       beam.RegisterType(reflect.TypeOf((*pb.PubsubMessage)(nil)).Elem())
+       beam.RegisterFunction(unmarshalMessageFn)
+}
+
+// ReadOptions represents options for reading from PubSub.
+type ReadOptions struct {
+       Subscription       string
+       IDAttribute        string
+       TimestampAttribute string
+       WithAttributes     bool
+}
+
+// Read reads an unbounded number of PubSubMessages from the given
+// pubsub topic. It produces an unbounded PCollecton<*PubSubMessage>,
+// if WithAttributes is set, or an unbounded PCollection<[]byte>.
+func Read(s beam.Scope, project, topic string, opts *ReadOptions) 
beam.PCollection {
+       s = s.Scope("pubsubio.Read")
+
+       payload := &v1.PubSubPayload{
+               Op:    v1.PubSubPayload_READ,
+               Topic: pubsubx.MakeQualifiedTopicName(project, topic),
+       }
+       if opts != nil {
+               payload.IdAttribute = opts.IDAttribute
+               payload.TimestampAttribute = opts.TimestampAttribute
+               if opts.Subscription != "" {
+                       payload.Subscription = 
pubsubx.MakeQualifiedSubscriptionName(project, opts.Subscription)
+               }
+               payload.WithAttributes = opts.WithAttributes
+       }
+
+       out := beam.External(s, v1.PubSubPayloadURN, 
protox.MustEncode(payload), nil, []beam.FullType{typex.New(reflectx.ByteSlice)})
+       if opts.WithAttributes {
+               return beam.ParDo(s, unmarshalMessageFn, out[0])
+       }
+       return out[0]
+}
+
+func unmarshalMessageFn(raw []byte) (*pb.PubsubMessage, error) {
+       var msg pb.PubsubMessage
+       if err := proto.Unmarshal(raw, &msg); err != nil {
+               return nil, err
+       }
+       return &msg, nil
+}
+
+// Write writes PubSubMessages or bytes to the given pubsub topic.
+func Write(s beam.Scope, project, topic string, col beam.PCollection) {
+       s = s.Scope("pubsubio.Write")
+
+       payload := &v1.PubSubPayload{
+               Op:    v1.PubSubPayload_WRITE,
+               Topic: pubsubx.MakeQualifiedTopicName(project, topic),
+       }
+
+       out := col
+       if col.Type().Type() != reflectx.ByteSlice {
+               out = beam.ParDo(s, proto.Marshal, col)
+               payload.WithAttributes = true
+       }
+       beam.External(s, v1.PubSubPayloadURN, protox.MustEncode(payload), 
[]beam.PCollection{out}, nil)
+}
diff --git a/sdks/go/pkg/beam/io/pubsubio/v1/gen.go 
b/sdks/go/pkg/beam/io/pubsubio/v1/gen.go
new file mode 100644
index 00000000000..55191112ce7
--- /dev/null
+++ b/sdks/go/pkg/beam/io/pubsubio/v1/gen.go
@@ -0,0 +1,21 @@
+// 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 v1
+
+//go:generate protoc -I . v1.proto --go_out=.
+
+// PubSubPayloadURN is the URN of the pubsub proto payload.
+const PubSubPayloadURN = "beam:go:payload:pubsub:v1"
diff --git a/sdks/go/pkg/beam/io/pubsubio/v1/v1.pb.go 
b/sdks/go/pkg/beam/io/pubsubio/v1/v1.pb.go
new file mode 100644
index 00000000000..e398a09d1ed
--- /dev/null
+++ b/sdks/go/pkg/beam/io/pubsubio/v1/v1.pb.go
@@ -0,0 +1,134 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: v1.proto
+
+/*
+Package v1 is a generated protocol buffer package.
+
+It is generated from these files:
+       v1.proto
+
+It has these top-level messages:
+       PubSubPayload
+*/
+package v1
+
+import proto "github.com/golang/protobuf/proto"
+import fmt "fmt"
+import math "math"
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type PubSubPayload_Op int32
+
+const (
+       PubSubPayload_INVALID PubSubPayload_Op = 0
+       PubSubPayload_READ    PubSubPayload_Op = 1
+       PubSubPayload_WRITE   PubSubPayload_Op = 2
+)
+
+var PubSubPayload_Op_name = map[int32]string{
+       0: "INVALID",
+       1: "READ",
+       2: "WRITE",
+}
+var PubSubPayload_Op_value = map[string]int32{
+       "INVALID": 0,
+       "READ":    1,
+       "WRITE":   2,
+}
+
+func (x PubSubPayload_Op) String() string {
+       return proto.EnumName(PubSubPayload_Op_name, int32(x))
+}
+func (PubSubPayload_Op) EnumDescriptor() ([]byte, []int) { return 
fileDescriptor0, []int{0, 0} }
+
+type PubSubPayload struct {
+       Op                 PubSubPayload_Op 
`protobuf:"varint,1,opt,name=op,enum=v1.PubSubPayload_Op" json:"op,omitempty"`
+       Topic              string           `protobuf:"bytes,2,opt,name=Topic" 
json:"Topic,omitempty"`
+       Subscription       string           
`protobuf:"bytes,3,opt,name=Subscription" json:"Subscription,omitempty"`
+       IdAttribute        string           
`protobuf:"bytes,4,opt,name=IdAttribute" json:"IdAttribute,omitempty"`
+       TimestampAttribute string           
`protobuf:"bytes,5,opt,name=TimestampAttribute" 
json:"TimestampAttribute,omitempty"`
+       WithAttributes     bool             
`protobuf:"varint,6,opt,name=WithAttributes" json:"WithAttributes,omitempty"`
+}
+
+func (m *PubSubPayload) Reset()                    { *m = PubSubPayload{} }
+func (m *PubSubPayload) String() string            { return 
proto.CompactTextString(m) }
+func (*PubSubPayload) ProtoMessage()               {}
+func (*PubSubPayload) Descriptor() ([]byte, []int) { return fileDescriptor0, 
[]int{0} }
+
+func (m *PubSubPayload) GetOp() PubSubPayload_Op {
+       if m != nil {
+               return m.Op
+       }
+       return PubSubPayload_INVALID
+}
+
+func (m *PubSubPayload) GetTopic() string {
+       if m != nil {
+               return m.Topic
+       }
+       return ""
+}
+
+func (m *PubSubPayload) GetSubscription() string {
+       if m != nil {
+               return m.Subscription
+       }
+       return ""
+}
+
+func (m *PubSubPayload) GetIdAttribute() string {
+       if m != nil {
+               return m.IdAttribute
+       }
+       return ""
+}
+
+func (m *PubSubPayload) GetTimestampAttribute() string {
+       if m != nil {
+               return m.TimestampAttribute
+       }
+       return ""
+}
+
+func (m *PubSubPayload) GetWithAttributes() bool {
+       if m != nil {
+               return m.WithAttributes
+       }
+       return false
+}
+
+func init() {
+       proto.RegisterType((*PubSubPayload)(nil), "v1.PubSubPayload")
+       proto.RegisterEnum("v1.PubSubPayload_Op", PubSubPayload_Op_name, 
PubSubPayload_Op_value)
+}
+
+func init() { proto.RegisterFile("v1.proto", fileDescriptor0) }
+
+var fileDescriptor0 = []byte{
+       // 226 bytes of a gzipped FileDescriptorProto
+       0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x90, 
0x41, 0x4b, 0xc3, 0x30,
+       0x18, 0x86, 0x4d, 0x5c, 0x67, 0xf7, 0x4d, 0x47, 0xf9, 0xd8, 0x21, 0xc7, 
0x52, 0x64, 0xf4, 0x14,
+       0xa8, 0xfe, 0x82, 0xc2, 0x76, 0x08, 0x88, 0x1b, 0x59, 0x71, 0xe7, 0x66, 
0x1b, 0x18, 0x70, 0xe6,
+       0xa3, 0x4d, 0x0a, 0xfe, 0x0c, 0xff, 0xb1, 0x10, 0x41, 0x9d, 0xec, 0xf8, 
0x3e, 0xcf, 0x73, 0x7a,
+       0x21, 0x1d, 0x2a, 0x49, 0x9d, 0xf3, 0x0e, 0xf9, 0x50, 0x15, 0x9f, 0x1c, 
0xee, 0x36, 0xc1, 0x6c,
+       0x83, 0xd9, 0xb4, 0x1f, 0x6f, 0xae, 0x3d, 0xe0, 0x3d, 0x70, 0x47, 0x82, 
0xe5, 0xac, 0x9c, 0x3d,
+       0xcc, 0xe5, 0x50, 0xc9, 0x33, 0x2d, 0xd7, 0xa4, 0xb9, 0x23, 0x9c, 0x43, 
0xd2, 0x38, 0xb2, 0x7b,
+       0xc1, 0x73, 0x56, 0x4e, 0xf4, 0xf7, 0xc0, 0x02, 0x6e, 0xb7, 0xc1, 0xf4, 
0xfb, 0xce, 0x92, 0xb7,
+       0xee, 0x5d, 0x5c, 0x47, 0x79, 0xc6, 0x30, 0x87, 0xa9, 0x3a, 0xd4, 0xde, 
0x77, 0xd6, 0x04, 0x7f,
+       0x14, 0xa3, 0x98, 0xfc, 0x45, 0x28, 0x01, 0x1b, 0x7b, 0x3a, 0xf6, 0xbe, 
0x3d, 0xd1, 0x6f, 0x98,
+       0xc4, 0xf0, 0x82, 0xc1, 0x05, 0xcc, 0x76, 0xd6, 0xbf, 0xfe, 0x80, 0x5e, 
0x8c, 0x73, 0x56, 0xa6,
+       0xfa, 0x1f, 0x2d, 0x16, 0xc0, 0xd7, 0x84, 0x53, 0xb8, 0x51, 0xcf, 0x2f, 
0xf5, 0x93, 0x5a, 0x66,
+       0x57, 0x98, 0xc2, 0x48, 0xaf, 0xea, 0x65, 0xc6, 0x70, 0x02, 0xc9, 0x4e, 
0xab, 0x66, 0x95, 0x71,
+       0x33, 0x8e, 0xf7, 0x3c, 0x7e, 0x05, 0x00, 0x00, 0xff, 0xff, 0x3a, 0x1b, 
0xdf, 0x4b, 0x2a, 0x01,
+       0x00, 0x00,
+}
diff --git a/sdks/go/pkg/beam/io/pubsubio/v1/v1.proto 
b/sdks/go/pkg/beam/io/pubsubio/v1/v1.proto
new file mode 100644
index 00000000000..e55930ed6ec
--- /dev/null
+++ b/sdks/go/pkg/beam/io/pubsubio/v1/v1.proto
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+/*
+ * Protocol Buffers describing pubsub payload (v1) to pass to beam.External.
+ */
+syntax = "proto3";
+
+package v1;
+
+message PubSubPayload {
+    enum Op {
+        INVALID = 0;
+        READ = 1;
+        WRITE = 2;
+    }
+    Op op = 1;
+
+    string Topic = 2;
+    string Subscription = 3;
+    string IdAttribute = 4;
+    string TimestampAttribute = 5;
+    bool WithAttributes = 6;
+}
diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflow.go 
b/sdks/go/pkg/beam/runners/dataflow/dataflow.go
index 57c91cd6459..d9816aff5a0 100644
--- a/sdks/go/pkg/beam/runners/dataflow/dataflow.go
+++ b/sdks/go/pkg/beam/runners/dataflow/dataflow.go
@@ -58,6 +58,7 @@ var (
        network         = flag.String("network", "", "GCP network (optional)")
        tempLocation    = flag.String("temp_location", "", "Temp location 
(optional)")
        machineType     = flag.String("worker_machine_type", "", "GCE machine 
type (optional)")
+       streaming       = flag.Bool("streaming", false, "Streaming job")
 
        dryRun         = flag.Bool("dry_run", false, "Dry run. Just print the 
job, but don't submit it.")
        teardownPolicy = flag.String("teardown_policy", "", "Job teardown 
policy (internal only).")
@@ -151,17 +152,24 @@ func Execute(ctx context.Context, p *beam.Pipeline) error 
{
                return err
        }
 
+       jobType := "JOB_TYPE_BATCH"
+       apiJobType := "FNAPI_BATCH"
+       if *streaming {
+               jobType = "JOB_TYPE_STREAMING"
+               apiJobType = "FNAPI_STREAMING"
+       }
+
        job := &df.Job{
                ProjectId: project,
                Name:      jobName,
-               Type:      "JOB_TYPE_BATCH",
+               Type:      jobType,
                Environment: &df.Environment{
                        UserAgent: newMsg(userAgent{
                                Name:    "Apache Beam SDK for Go",
                                Version: "0.3.0",
                        }),
                        Version: newMsg(version{
-                               JobType: "FNAPI_BATCH",
+                               JobType: apiJobType,
                                Major:   "6",
                        }),
                        SdkPipelineOptions: newMsg(pipelineOptions{
@@ -198,6 +206,10 @@ func Execute(ctx context.Context, p *beam.Pipeline) error {
        if *tempLocation != "" {
                job.Environment.TempStoragePrefix = *tempLocation
        }
+       if *streaming {
+               // Add separate data disk for streaming jobs
+               job.Environment.WorkerPools[0].DataDisks = []*df.Disk{{}}
+       }
        printJob(ctx, job)
 
        if *dryRun {
@@ -239,6 +251,10 @@ func Execute(ctx context.Context, p *beam.Pipeline) error {
                        log.Info(ctx, "Job succeeded!")
                        return nil
 
+               case "JOB_STATE_CANCELLED":
+                       log.Info(ctx, "Job cancelled")
+                       return nil
+
                case "JOB_STATE_FAILED":
                        return fmt.Errorf("job %s failed", upd.Id)
 
diff --git a/sdks/go/pkg/beam/runners/dataflow/messages.go 
b/sdks/go/pkg/beam/runners/dataflow/messages.go
index 80fcebdadf0..28e5d46fdfa 100644
--- a/sdks/go/pkg/beam/runners/dataflow/messages.go
+++ b/sdks/go/pkg/beam/runners/dataflow/messages.go
@@ -81,6 +81,19 @@ type properties struct {
        OutputInfo              []output                    
`json:"output_info,omitempty"`               // Source, ParDo, GBK, Flatten, 
Combine
        ParallelInput           *outputReference            
`json:"parallel_input,omitempty"`            // ParDo, GBK, Flatten, Combine
        SerializedFn            string                      
`json:"serialized_fn,omitempty"`             // ParDo, Combine
+
+       PubSubTopic          string `json:"pubsub_topic,omitempty"`           
// Read,Write
+       PubSubSubscription   string `json:"pubsub_subscription,omitempty"`    
// Read,Write
+       PubSubIDLabel        string `json:"pubsub_id_label,omitempty"`        
// Read,Write
+       PubSubTimestampLabel string `json:"pubsub_timestamp_label,omitempty"` 
// Read,Write
+
+       // This special property triggers whether the below struct should be 
used instead.
+       PubSubWithAttributes bool `json:"pubsub_with_attributes,omitempty"`
+}
+
+type propertiesWithPubSubMessage struct {
+       properties
+       PubSubSerializedAttributesFn string 
`json:"pubsub_serialized_attributes_fn"` // Read,Write
 }
 
 type output struct {
diff --git a/sdks/go/pkg/beam/runners/dataflow/translate.go 
b/sdks/go/pkg/beam/runners/dataflow/translate.go
index 440dcd7a9ab..0ee05f8f267 100644
--- a/sdks/go/pkg/beam/runners/dataflow/translate.go
+++ b/sdks/go/pkg/beam/runners/dataflow/translate.go
@@ -30,6 +30,7 @@ import (
        "github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx"
        "github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx/v1"
        "github.com/apache/beam/sdks/go/pkg/beam/core/util/protox"
+       pubsub_v1 "github.com/apache/beam/sdks/go/pkg/beam/io/pubsubio/v1"
        rnapi_pb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
        "github.com/golang/protobuf/proto"
        df "google.golang.org/api/dataflow/v1b3"
@@ -42,6 +43,10 @@ const (
        gbkKind     = "GroupByKey"
 
        sideInputKind = "CollectionToSingleton"
+
+       // Support for Dataflow native I/O, such as PubSub.
+       readKind  = "ParallelRead"
+       writeKind = "ParallelWrite"
 )
 
 // translate translates a Graph into a sequence of Dataflow steps. The step
@@ -149,6 +154,13 @@ func translate(edges []*graph.MultiEdge) ([]*df.Step, 
error) {
                        Kind:       kind,
                        Properties: newMsg(prop),
                }
+               if prop.PubSubWithAttributes {
+                       // Hack to add a empty-value property for PubSub IO. 
This
+                       // will make PubSub send the entire message, not just
+                       // the payload.
+                       prop.PubSubWithAttributes = false
+                       step.Properties = 
newMsg(propertiesWithPubSubMessage{properties: prop})
+               }
                steps = append(steps, step)
        }
 
@@ -351,6 +363,45 @@ func translateEdge(edge *graph.MultiEdge) (string, 
properties, error) {
                        UserName: buildName(edge.Scope(), "flatten"), // TODO: 
user-defined
                }, nil
 
+       case graph.External:
+               switch edge.Payload.URN {
+               case pubsub_v1.PubSubPayloadURN:
+                       // Translate to native handling of PubSub I/O.
+
+                       var msg pubsub_v1.PubSubPayload
+                       if err := proto.Unmarshal(edge.Payload.Data, &msg); err 
!= nil {
+                               return "", properties{}, fmt.Errorf("bad pubsub 
payload: %v", err)
+                       }
+                       prop := properties{
+                               UserName:             buildName(edge.Scope(), 
fmt.Sprintf("%v", msg.Op)),
+                               Format:               "pubsub",
+                               PubSubTopic:          msg.GetTopic(),
+                               PubSubSubscription:   msg.GetSubscription(),
+                               PubSubIDLabel:        msg.GetIdAttribute(),
+                               PubSubTimestampLabel: 
msg.GetTimestampAttribute(),
+                               PubSubWithAttributes: msg.GetWithAttributes(),
+                       }
+                       if prop.PubSubSubscription != "" {
+                               prop.PubSubTopic = ""
+                       }
+
+                       switch msg.Op {
+                       case pubsub_v1.PubSubPayload_READ:
+                               return readKind, prop, nil
+
+                       case pubsub_v1.PubSubPayload_WRITE:
+                               c, _ := encodeCoderRef(coder.NewBytes())
+                               prop.Encoding = c
+                               return writeKind, prop, nil
+
+                       default:
+                               return "", properties{}, fmt.Errorf("bad pubsub 
op: %v", msg.Op)
+                       }
+
+               default:
+                       return "", properties{}, fmt.Errorf("bad external urn: 
%v", edge.Payload.URN)
+               }
+
        default:
                return "", properties{}, fmt.Errorf("bad opcode: %v", edge)
        }
diff --git a/sdks/go/pkg/beam/util/pubsubx/pubsub.go 
b/sdks/go/pkg/beam/util/pubsubx/pubsub.go
new file mode 100644
index 00000000000..a9119036cc6
--- /dev/null
+++ b/sdks/go/pkg/beam/util/pubsubx/pubsub.go
@@ -0,0 +1,110 @@
+// 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 pubsubx contains utilities for working with Google PubSub.
+package pubsubx
+
+import (
+       "context"
+       "fmt"
+       "time"
+
+       "cloud.google.com/go/pubsub"
+       "github.com/apache/beam/sdks/go/pkg/beam/log"
+)
+
+// MakeQualifiedTopicName returns a fully-qualified topic name for
+// the given project and topic.
+func MakeQualifiedTopicName(project, topic string) string {
+       return fmt.Sprintf("projects/%s/topics/%s", project, topic)
+}
+
+// MakeQualifiedSubscriptionName returns a fully-qualified subscription name 
for
+// the given project and subscription id.
+func MakeQualifiedSubscriptionName(project, subscription string) string {
+       return fmt.Sprintf("projects/%s/subscriptions/%s", project, 
subscription)
+}
+
+// EnsureTopic creates a new topic, if it doesn't exist.
+func EnsureTopic(ctx context.Context, client *pubsub.Client, topic string) 
(*pubsub.Topic, error) {
+       ret := client.Topic(topic)
+
+       exists, err := ret.Exists(ctx)
+       if err != nil {
+               return nil, err
+       }
+       if !exists {
+               return client.CreateTopic(ctx, topic)
+       }
+       return ret, nil
+}
+
+// EnsureSubscription creates a new subscription with the given name, if it 
doesn't exist.
+func EnsureSubscription(ctx context.Context, client *pubsub.Client, topic, id 
string) (*pubsub.Subscription, error) {
+       ret := client.Subscription(id)
+       exists, err := ret.Exists(ctx)
+       if err != nil {
+               return nil, err
+       }
+       if !exists {
+               cfg := pubsub.SubscriptionConfig{
+                       Topic: client.Topic(topic),
+               }
+               return client.CreateSubscription(ctx, id, cfg)
+       }
+       return ret, nil
+}
+
+// CleanupTopic deletes a topic with all subscriptions and logs any
+// error. Useful for defer.
+func CleanupTopic(ctx context.Context, project, topic string) {
+       client, err := pubsub.NewClient(ctx, project)
+       if err != nil {
+               log.Errorf(ctx, "Failed to delete topic %v", topic, err)
+       }
+       if err := client.Topic(topic).Delete(ctx); err != nil {
+               log.Errorf(ctx, "Failed to delete topic %v", topic, err)
+       }
+}
+
+// Publish is a simple utility for publishing a set of string messages
+// serially to a pubsub topic. Small scale use only.
+func Publish(ctx context.Context, project, topic string, messages ...string) 
(*pubsub.Subscription, error) {
+       client, err := pubsub.NewClient(ctx, project)
+       if err != nil {
+               return nil, err
+       }
+       t, err := EnsureTopic(ctx, client, topic)
+       if err != nil {
+               return nil, err
+       }
+       sub, err := EnsureSubscription(ctx, client, topic, 
fmt.Sprintf("%v.sub.%v", topic, time.Now().Unix()))
+       if err != nil {
+               return nil, err
+       }
+
+       for _, msg := range messages {
+               m := &pubsub.Message{
+                       Data: ([]byte)(msg),
+                       // Attributes: ??
+               }
+               id, err := t.Publish(ctx, m).Get(ctx)
+               if err != nil {
+                       return nil, fmt.Errorf("failed to publish '%v': %v", 
msg, err)
+               }
+               log.Infof(ctx, "Published %v with id: %v", msg, id)
+       }
+       return sub, nil
+}


 

----------------------------------------------------------------
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:
[email protected]


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

    Worklog Id:     (was: 91922)
    Time Spent: 2h 40m  (was: 2.5h)

> Add prototype support for Go SDK streaming
> ------------------------------------------
>
>                 Key: BEAM-3856
>                 URL: https://issues.apache.org/jira/browse/BEAM-3856
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-go
>            Reporter: Willy Lulciuc
>            Assignee: Henning Rohde
>            Priority: Major
>          Time Spent: 2h 40m
>  Remaining Estimate: 0h
>




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

Reply via email to