You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/06/03 15:45:02 UTC

[GitHub] [beam] jrmccluskey opened a new pull request, #17955: Add native PubSub IO prototype to Go

jrmccluskey opened a new pull request, #17955:
URL: https://github.com/apache/beam/pull/17955

   Adds a prototype PubSub IO in native Go. Also implements a small native streaming version of the wordcap example pipeline. This feature is fully experimental and should not be used over the current cross-language implementation. 
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Add a link to the appropriate issue in your description, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   To check the build health, please visit [https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md)
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
lostluck commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894871945


##########
sdks/go/examples/native_wordcap/wordcap.go:
##########
@@ -0,0 +1,79 @@
+// 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.
+
+// native_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.
+//
+// This is a portable PubSub option and does not need to be run on Dataflow.
+package main
+
+import (
+	"context"
+	"flag"
+	"os"
+	"strings"
+
+	"github.com/apache/beam/sdks/v2/go/examples/native_wordcap/nativepubsubio"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/stringx"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/options/gcpopts"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/x/beamx"
+	"github.com/apache/beam/sdks/v2/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)
+
+	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 native wordcap with subscription: %v", sub.ID())
+
+	p := beam.NewPipeline()
+	s := p.Root()
+
+	col := nativepubsubio.NativeRead(ctx, s, project, *input, sub.ID())

Review Comment:
   We've got a stutter here. The exposed methods should just be Read and Write, rather than have Native prefixed there. Remember that since users will have to write the package name, information in the package short name shouldn't be repeated if possible.  (eg. context.Context stutters on the type, but there's no good alternative name when the package is context.)



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,247 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn5x2[context.Context, beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&pubSubRead{})
+	register.DoFn2x1[context.Context, []byte, error](&pubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// pubSubRead is a structural DoFn representing a read from a given subscription ID.
+type pubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	client            *pubsub.Client
+	processedMessages []*pubsub.Message
+}
+
+// newPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func newPubSubRead(ctx context.Context, projectID, topic, subscription string) (*pubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &pubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *pubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *pubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *pubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *pubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubRead) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(ctx, r.ProjectID)
+		if err != nil {
+			return err
+		}
+		r.client = client
+	}
+	return nil
+}
+
+var messageTimeout time.Duration = 5 * time.Second
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *pubSubRead) ProcessElement(ctx context.Context, bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.processedMessages {
+			m.Ack()
+		}
+		r.processedMessages = nil
+		return nil
+	})
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}

Review Comment:
   As written, we're sort of prevented from having the runner force a checkpoint (only allowing DoFn checkpoints, but given we don't currently split, this is probably fine for now.  
   
   The risk is a higher throughput pubsub, which will prevent this code from checkpointing, which will prevent downstream aggregations.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894821312


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)

Review Comment:
   Okay I think I have an implementation. This has the benefit of keeping the DoFn up longer when checking for messages, although dataflow does use this longer-running form to try and split it. Not as noisy as before though, so it works. This is a pattern we want users and devs to use over sleeps anyway, so it's worth including



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r889308951


##########
sdks/go/examples/native_wordcap/nativepubsubio/subscriptiontracker.go:
##########
@@ -0,0 +1,80 @@
+// 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 nativepubsubio
+
+import "math"
+
+// The SubscriptionRTracker maintains a single entry string to keep up with
+// the PubSub subscription being used in the NativeRead SDF.
+type SubscriptionRTracker struct {
+	Subscription string
+	Done         bool
+}
+
+// NewSubscriptionRTracker returns an RTracker wrapping the
+// provided subscription and a "Done" boolean.
+func NewSubscriptionRTracker(entry string) *SubscriptionRTracker {
+	return &SubscriptionRTracker{Subscription: entry, Done: false}
+}
+
+// TryClaim returns true iff the given position is a string and matches the underlying
+// subscription ID.
+func (s *SubscriptionRTracker) TryClaim(pos interface{}) bool {
+	posString, ok := pos.(string)
+	return ok && posString == s.Subscription
+}
+
+// TrySplit is a no-op for the StaticRTracker in the normal case and moves the subscription
+// to the residual in the checkpointing case, marking itself as done to keep the logical checks
+// around SDF data loss happy.
+func (s *SubscriptionRTracker) TrySplit(frac float64) (primary, residual interface{}, err error) {
+	if math.Abs(frac-0.000) < 0.0001 {

Review Comment:
   Oops



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894785067


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)
+
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.ProcessedMessages = append(r.ProcessedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+			default:
+				log.Debug(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := NewPubSubRead(context.Background(), project, topic, subscription)
+	if err != nil {
+		panic(err)
+	}
+	return beam.ParDo(s, psRead, beam.Impulse(s))
+}
+
+// PubSubWrite is a structural DoFn representing writes to a given PubSub topic.
+type PubSubWrite struct {
+	ProjectID string
+	Topic     string
+	Client    *pubsub.Client
+}
+
+// ProcessElement takes a []byte element and publishes it to the provided PubSub
+// topic.
+func (w *PubSubWrite) ProcessElement(elm []byte) error {
+	// Initialize PubSub client if one has not been created already
+	if w.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), w.ProjectID)
+		if err != nil {
+			return err
+		}
+		w.Client = client
+	}

Review Comment:
   Done



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r889280034


##########
sdks/go/examples/native_wordcap/nativepubsubio/subscriptiontracker.go:
##########
@@ -0,0 +1,80 @@
+// 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 nativepubsubio
+
+import "math"
+
+// The SubscriptionRTracker maintains a single entry string to keep up with
+// the PubSub subscription being used in the NativeRead SDF.
+type SubscriptionRTracker struct {
+	Subscription string
+	Done         bool
+}
+
+// NewSubscriptionRTracker returns an RTracker wrapping the
+// provided subscription and a "Done" boolean.
+func NewSubscriptionRTracker(entry string) *SubscriptionRTracker {
+	return &SubscriptionRTracker{Subscription: entry, Done: false}
+}
+
+// TryClaim returns true iff the given position is a string and matches the underlying
+// subscription ID.
+func (s *SubscriptionRTracker) TryClaim(pos interface{}) bool {
+	posString, ok := pos.(string)
+	return ok && posString == s.Subscription
+}
+
+// TrySplit is a no-op for the StaticRTracker in the normal case and moves the subscription
+// to the residual in the checkpointing case, marking itself as done to keep the logical checks
+// around SDF data loss happy.
+func (s *SubscriptionRTracker) TrySplit(frac float64) (primary, residual interface{}, err error) {
+	if math.Abs(frac-0.000) < 0.0001 {

Review Comment:
   It's likely overkill in this case but I usually err on the side of not fully trusting floating point numbers. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894850138


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,247 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn5x2[context.Context, beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&pubSubRead{})
+	register.DoFn2x1[context.Context, []byte, error](&pubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// pubSubRead is a structural DoFn representing a read from a given subscription ID.
+type pubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	client            *pubsub.Client
+	processedMessages []*pubsub.Message
+}
+
+// newPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func newPubSubRead(ctx context.Context, projectID, topic, subscription string) (*pubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &pubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *pubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *pubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *pubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *pubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubRead) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return err
+		}
+		r.client = client
+	}
+	return nil
+}
+
+var messageTimeout time.Duration = 5 * time.Second
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *pubSubRead) ProcessElement(ctx context.Context, bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.processedMessages {
+			m.Ack()
+		}
+		r.processedMessages = nil
+		return nil
+	})
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(ctx)
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		timeout := time.NewTimer(messageTimeout)
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.processedMessages = append(r.processedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+				if !timeout.Stop() {
+					<-timeout.C
+				}
+				timeout.Reset(messageTimeout)
+			case <-timeout.C:
+				log.Debugf(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := newPubSubRead(context.Background(), project, topic, subscription)

Review Comment:
   Gotcha. Changed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on PR #17955:
URL: https://github.com/apache/beam/pull/17955#issuecomment-1152687842

   Cracking up because I had all of those context changes then something got scrambled when I was making the setup functions and I had to step back and undo them, then forgot to put them back again. Oh well, should be fixed now


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894876605


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,247 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn5x2[context.Context, beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&pubSubRead{})
+	register.DoFn2x1[context.Context, []byte, error](&pubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// pubSubRead is a structural DoFn representing a read from a given subscription ID.
+type pubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	client            *pubsub.Client
+	processedMessages []*pubsub.Message
+}
+
+// newPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func newPubSubRead(ctx context.Context, projectID, topic, subscription string) (*pubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &pubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *pubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *pubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *pubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *pubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubRead) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(ctx, r.ProjectID)
+		if err != nil {
+			return err
+		}
+		r.client = client
+	}
+	return nil
+}
+
+var messageTimeout time.Duration = 5 * time.Second
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *pubSubRead) ProcessElement(ctx context.Context, bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.processedMessages {
+			m.Ack()
+		}
+		r.processedMessages = nil
+		return nil
+	})
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}

Review Comment:
   For a short example I'm not going to sweat it too much, we can revisit how the runner would split this later. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894752774


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {

Review Comment:
   Done.



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] damccorm commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
damccorm commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r889283226


##########
sdks/go/examples/native_wordcap/nativepubsubio/subscriptiontracker.go:
##########
@@ -0,0 +1,80 @@
+// 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 nativepubsubio
+
+import "math"
+
+// The SubscriptionRTracker maintains a single entry string to keep up with
+// the PubSub subscription being used in the NativeRead SDF.
+type SubscriptionRTracker struct {
+	Subscription string
+	Done         bool
+}
+
+// NewSubscriptionRTracker returns an RTracker wrapping the
+// provided subscription and a "Done" boolean.
+func NewSubscriptionRTracker(entry string) *SubscriptionRTracker {
+	return &SubscriptionRTracker{Subscription: entry, Done: false}
+}
+
+// TryClaim returns true iff the given position is a string and matches the underlying
+// subscription ID.
+func (s *SubscriptionRTracker) TryClaim(pos interface{}) bool {
+	posString, ok := pos.(string)
+	return ok && posString == s.Subscription
+}
+
+// TrySplit is a no-op for the StaticRTracker in the normal case and moves the subscription
+// to the residual in the checkpointing case, marking itself as done to keep the logical checks
+// around SDF data loss happy.
+func (s *SubscriptionRTracker) TrySplit(frac float64) (primary, residual interface{}, err error) {
+	if math.Abs(frac-0.000) < 0.0001 {

Review Comment:
   I'd vote we verify `== 0.0` works and then use it - people will copy the patterns they see here, and this is unnecessarily complex if 0.0 does work. Plus this theoretically could be bad for sufficiently large restrictions (if you truly got a restriction of 0.00001 or something).
   
   Regardless, the math.Abs is definitely unnecessary, any negative value should equate to a 0.0 split



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] github-actions[bot] commented on pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #17955:
URL: https://github.com/apache/beam/pull/17955#issuecomment-1146165862

   Assigning reviewers. If you would like to opt out of this review, comment `assign to next reviewer`:
   
   R: @damccorm for label go.
   
   Available commands:
   - `stop reviewer notifications` - opt out of the automated review tooling
   - `remind me after tests pass` - tag the comment author after tests pass
   - `waiting on author` - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)
   
   The PR bot will only process comments in the main thread (not review comments).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
lostluck commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894830666


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,247 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn5x2[context.Context, beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&pubSubRead{})
+	register.DoFn2x1[context.Context, []byte, error](&pubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// pubSubRead is a structural DoFn representing a read from a given subscription ID.
+type pubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	client            *pubsub.Client
+	processedMessages []*pubsub.Message
+}
+
+// newPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func newPubSubRead(ctx context.Context, projectID, topic, subscription string) (*pubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &pubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *pubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *pubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *pubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *pubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubRead) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)

Review Comment:
   ```suggestion
   		client, err := pubsub.NewClient(ctx, r.ProjectID)
   ```



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,247 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn5x2[context.Context, beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&pubSubRead{})
+	register.DoFn2x1[context.Context, []byte, error](&pubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// pubSubRead is a structural DoFn representing a read from a given subscription ID.
+type pubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	client            *pubsub.Client
+	processedMessages []*pubsub.Message
+}
+
+// newPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func newPubSubRead(ctx context.Context, projectID, topic, subscription string) (*pubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &pubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *pubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *pubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *pubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *pubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubRead) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return err
+		}
+		r.client = client
+	}
+	return nil
+}
+
+var messageTimeout time.Duration = 5 * time.Second
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *pubSubRead) ProcessElement(ctx context.Context, bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.processedMessages {
+			m.Ack()
+		}
+		r.processedMessages = nil
+		return nil
+	})
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(ctx)
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		timeout := time.NewTimer(messageTimeout)
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")

Review Comment:
   ```suggestion
   					log.Debug(ctx, "stopping bundle processing")
   ```



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,247 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn5x2[context.Context, beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&pubSubRead{})
+	register.DoFn2x1[context.Context, []byte, error](&pubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// pubSubRead is a structural DoFn representing a read from a given subscription ID.
+type pubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	client            *pubsub.Client
+	processedMessages []*pubsub.Message
+}
+
+// newPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func newPubSubRead(ctx context.Context, projectID, topic, subscription string) (*pubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &pubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *pubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *pubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *pubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *pubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubRead) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return err
+		}
+		r.client = client
+	}
+	return nil
+}
+
+var messageTimeout time.Duration = 5 * time.Second
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *pubSubRead) ProcessElement(ctx context.Context, bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.processedMessages {
+			m.Ack()
+		}
+		r.processedMessages = nil
+		return nil
+	})
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(ctx)

Review Comment:
   Might want to use a different variable for the pubsub context to avoid having it shadow the log context, in the event cancellations happens.



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,247 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn5x2[context.Context, beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&pubSubRead{})
+	register.DoFn2x1[context.Context, []byte, error](&pubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// pubSubRead is a structural DoFn representing a read from a given subscription ID.
+type pubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	client            *pubsub.Client
+	processedMessages []*pubsub.Message
+}
+
+// newPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func newPubSubRead(ctx context.Context, projectID, topic, subscription string) (*pubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &pubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *pubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *pubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *pubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *pubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubRead) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return err
+		}
+		r.client = client
+	}
+	return nil
+}
+
+var messageTimeout time.Duration = 5 * time.Second
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *pubSubRead) ProcessElement(ctx context.Context, bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.processedMessages {
+			m.Ack()
+		}
+		r.processedMessages = nil
+		return nil
+	})
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(ctx)
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		timeout := time.NewTimer(messageTimeout)
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.processedMessages = append(r.processedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+				if !timeout.Stop() {
+					<-timeout.C
+				}
+				timeout.Reset(messageTimeout)
+			case <-timeout.C:
+				log.Debugf(context.Background(), "cancelling receive context, scheduling resumption")

Review Comment:
   ```suggestion
   				log.Debugf(ctx, "cancelling receive context, scheduling resumption")
   ```



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,247 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn5x2[context.Context, beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&pubSubRead{})
+	register.DoFn2x1[context.Context, []byte, error](&pubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// pubSubRead is a structural DoFn representing a read from a given subscription ID.
+type pubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	client            *pubsub.Client
+	processedMessages []*pubsub.Message
+}
+
+// newPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func newPubSubRead(ctx context.Context, projectID, topic, subscription string) (*pubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &pubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *pubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *pubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *pubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *pubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubRead) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return err
+		}
+		r.client = client
+	}
+	return nil
+}
+
+var messageTimeout time.Duration = 5 * time.Second
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *pubSubRead) ProcessElement(ctx context.Context, bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.processedMessages {
+			m.Ack()
+		}
+		r.processedMessages = nil
+		return nil
+	})
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(ctx)
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		timeout := time.NewTimer(messageTimeout)
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.processedMessages = append(r.processedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+				if !timeout.Stop() {
+					<-timeout.C
+				}
+				timeout.Reset(messageTimeout)
+			case <-timeout.C:
+				log.Debugf(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := newPubSubRead(context.Background(), project, topic, subscription)

Review Comment:
   Idiomatic go would be to accept the context as the first parameter and then use that, which signals to the user that some RPCs may happen, and allow them to do whatever machinations they desire to the context.
   
   Idiomatic Go has context.Background() only called by end binary authors, rather than at arbitrary library levels & using context.TODO() to signal where additional plumbing is required when a caller doesn't have a context parameter itself.



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,247 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn5x2[context.Context, beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&pubSubRead{})
+	register.DoFn2x1[context.Context, []byte, error](&pubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// pubSubRead is a structural DoFn representing a read from a given subscription ID.
+type pubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	client            *pubsub.Client
+	processedMessages []*pubsub.Message
+}
+
+// newPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func newPubSubRead(ctx context.Context, projectID, topic, subscription string) (*pubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &pubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *pubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *pubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *pubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *pubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubRead) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return err
+		}
+		r.client = client
+	}
+	return nil
+}
+
+var messageTimeout time.Duration = 5 * time.Second
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *pubSubRead) ProcessElement(ctx context.Context, bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.processedMessages {
+			m.Ack()
+		}
+		r.processedMessages = nil
+		return nil
+	})
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(ctx)
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		timeout := time.NewTimer(messageTimeout)
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.processedMessages = append(r.processedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+				if !timeout.Stop() {
+					<-timeout.C
+				}
+				timeout.Reset(messageTimeout)
+			case <-timeout.C:
+				log.Debugf(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := newPubSubRead(context.Background(), project, topic, subscription)
+	if err != nil {
+		panic(err)
+	}
+	return beam.ParDo(s, psRead, beam.Impulse(s))
+}
+
+// pubSubWrite is a structural DoFn representing writes to a given PubSub topic.
+type pubSubWrite struct {
+	ProjectID string
+	Topic     string
+	client    *pubsub.Client
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubWrite) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return err
+		}
+		r.client = client
+	}
+	return nil
+}
+
+// ProcessElement takes a []byte element and publishes it to the provided PubSub
+// topic.
+func (w *pubSubWrite) ProcessElement(ctx context.Context, elm []byte) error {
+	top := w.client.Topic(w.Topic)
+
+	psMess := &pubsub.Message{Data: elm}
+	result := top.Publish(ctx, psMess)
+	if _, err := result.Get(ctx); err != nil {
+		return err
+	}
+	return nil
+}
+
+// newPubSubWrite inserts a write to a PubSub topic into the pipeline.
+func newPubSubWrite(ctx context.Context, projectID, topic string) (*pubSubWrite, error) {
+	if topic == "" {
+		return nil, errors.New("please provide a topic to write to")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	return &pubSubWrite{ProjectID: projectID, Topic: top.ID()}, nil
+}
+
+// NativeWrite publishes elements from a PCollection of byte slices to a PubSub topic.
+// If the topic does not exist at pipeline construction time, the function will panic.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Write() instead.
+func NativeWrite(s beam.Scope, col beam.PCollection, project, topic string) {
+	s = s.Scope("pubsubio.NativeWrite")
+
+	psWrite, err := newPubSubWrite(context.Background(), project, topic)

Review Comment:
   Same here, have a context passed in, and use that.



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,247 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn5x2[context.Context, beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&pubSubRead{})
+	register.DoFn2x1[context.Context, []byte, error](&pubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// pubSubRead is a structural DoFn representing a read from a given subscription ID.
+type pubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	client            *pubsub.Client
+	processedMessages []*pubsub.Message
+}
+
+// newPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func newPubSubRead(ctx context.Context, projectID, topic, subscription string) (*pubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &pubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *pubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *pubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *pubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *pubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubRead) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return err
+		}
+		r.client = client
+	}
+	return nil
+}
+
+var messageTimeout time.Duration = 5 * time.Second
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *pubSubRead) ProcessElement(ctx context.Context, bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.processedMessages {
+			m.Ack()
+		}
+		r.processedMessages = nil
+		return nil
+	})
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(ctx)
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		timeout := time.NewTimer(messageTimeout)
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.processedMessages = append(r.processedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+				if !timeout.Stop() {
+					<-timeout.C
+				}
+				timeout.Reset(messageTimeout)
+			case <-timeout.C:
+				log.Debugf(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := newPubSubRead(context.Background(), project, topic, subscription)
+	if err != nil {
+		panic(err)
+	}
+	return beam.ParDo(s, psRead, beam.Impulse(s))
+}
+
+// pubSubWrite is a structural DoFn representing writes to a given PubSub topic.
+type pubSubWrite struct {
+	ProjectID string
+	Topic     string
+	client    *pubsub.Client
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubWrite) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)

Review Comment:
   ```suggestion
   		client, err := pubsub.NewClient(ctx, r.ProjectID)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894849926


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,247 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn5x2[context.Context, beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&pubSubRead{})
+	register.DoFn2x1[context.Context, []byte, error](&pubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// pubSubRead is a structural DoFn representing a read from a given subscription ID.
+type pubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	client            *pubsub.Client
+	processedMessages []*pubsub.Message
+}
+
+// newPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func newPubSubRead(ctx context.Context, projectID, topic, subscription string) (*pubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &pubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *pubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *pubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *pubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *pubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubRead) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return err
+		}
+		r.client = client
+	}
+	return nil
+}
+
+var messageTimeout time.Duration = 5 * time.Second
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *pubSubRead) ProcessElement(ctx context.Context, bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.processedMessages {
+			m.Ack()
+		}
+		r.processedMessages = nil
+		return nil
+	})
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(ctx)

Review Comment:
   Fair note, changed



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,247 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn5x2[context.Context, beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&pubSubRead{})
+	register.DoFn2x1[context.Context, []byte, error](&pubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// pubSubRead is a structural DoFn representing a read from a given subscription ID.
+type pubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	client            *pubsub.Client
+	processedMessages []*pubsub.Message
+}
+
+// newPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func newPubSubRead(ctx context.Context, projectID, topic, subscription string) (*pubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &pubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *pubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *pubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *pubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *pubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubRead) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return err
+		}
+		r.client = client
+	}
+	return nil
+}
+
+var messageTimeout time.Duration = 5 * time.Second
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *pubSubRead) ProcessElement(ctx context.Context, bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.processedMessages {
+			m.Ack()
+		}
+		r.processedMessages = nil
+		return nil
+	})
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(ctx)
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		timeout := time.NewTimer(messageTimeout)
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.processedMessages = append(r.processedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+				if !timeout.Stop() {
+					<-timeout.C
+				}
+				timeout.Reset(messageTimeout)
+			case <-timeout.C:
+				log.Debugf(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := newPubSubRead(context.Background(), project, topic, subscription)
+	if err != nil {
+		panic(err)
+	}
+	return beam.ParDo(s, psRead, beam.Impulse(s))
+}
+
+// pubSubWrite is a structural DoFn representing writes to a given PubSub topic.
+type pubSubWrite struct {
+	ProjectID string
+	Topic     string
+	client    *pubsub.Client
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubWrite) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return err
+		}
+		r.client = client
+	}
+	return nil
+}
+
+// ProcessElement takes a []byte element and publishes it to the provided PubSub
+// topic.
+func (w *pubSubWrite) ProcessElement(ctx context.Context, elm []byte) error {
+	top := w.client.Topic(w.Topic)
+
+	psMess := &pubsub.Message{Data: elm}
+	result := top.Publish(ctx, psMess)
+	if _, err := result.Get(ctx); err != nil {
+		return err
+	}
+	return nil
+}
+
+// newPubSubWrite inserts a write to a PubSub topic into the pipeline.
+func newPubSubWrite(ctx context.Context, projectID, topic string) (*pubSubWrite, error) {
+	if topic == "" {
+		return nil, errors.New("please provide a topic to write to")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	return &pubSubWrite{ProjectID: projectID, Topic: top.ID()}, nil
+}
+
+// NativeWrite publishes elements from a PCollection of byte slices to a PubSub topic.
+// If the topic does not exist at pipeline construction time, the function will panic.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Write() instead.
+func NativeWrite(s beam.Scope, col beam.PCollection, project, topic string) {
+	s = s.Scope("pubsubio.NativeWrite")
+
+	psWrite, err := newPubSubWrite(context.Background(), project, topic)

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] asf-ci commented on pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
asf-ci commented on PR #17955:
URL: https://github.com/apache/beam/pull/17955#issuecomment-1146092088

   Can one of the admins verify this patch?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894752610


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}

Review Comment:
   Ah, a Go thing I did not know. I always figured that the default check being for a zero-length slice instead of nil would lead to managing a cleared list being an empty slice instead.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894876291


##########
sdks/go/examples/native_wordcap/wordcap.go:
##########
@@ -0,0 +1,79 @@
+// 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.
+
+// native_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.
+//
+// This is a portable PubSub option and does not need to be run on Dataflow.
+package main
+
+import (
+	"context"
+	"flag"
+	"os"
+	"strings"
+
+	"github.com/apache/beam/sdks/v2/go/examples/native_wordcap/nativepubsubio"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/stringx"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/options/gcpopts"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/x/beamx"
+	"github.com/apache/beam/sdks/v2/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)
+
+	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 native wordcap with subscription: %v", sub.ID())
+
+	p := beam.NewPipeline()
+	s := p.Root()
+
+	col := nativepubsubio.NativeRead(ctx, s, project, *input, sub.ID())

Review Comment:
   That's a holdover from when this was in the pubsub package. Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r889287769


##########
sdks/go/examples/native_wordcap/nativepubsubio/subscriptiontracker.go:
##########
@@ -0,0 +1,80 @@
+// 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 nativepubsubio
+
+import "math"
+
+// The SubscriptionRTracker maintains a single entry string to keep up with
+// the PubSub subscription being used in the NativeRead SDF.
+type SubscriptionRTracker struct {
+	Subscription string
+	Done         bool
+}
+
+// NewSubscriptionRTracker returns an RTracker wrapping the
+// provided subscription and a "Done" boolean.
+func NewSubscriptionRTracker(entry string) *SubscriptionRTracker {
+	return &SubscriptionRTracker{Subscription: entry, Done: false}
+}
+
+// TryClaim returns true iff the given position is a string and matches the underlying
+// subscription ID.
+func (s *SubscriptionRTracker) TryClaim(pos interface{}) bool {
+	posString, ok := pos.(string)
+	return ok && posString == s.Subscription
+}
+
+// TrySplit is a no-op for the StaticRTracker in the normal case and moves the subscription
+// to the residual in the checkpointing case, marking itself as done to keep the logical checks
+// around SDF data loss happy.
+func (s *SubscriptionRTracker) TrySplit(frac float64) (primary, residual interface{}, err error) {
+	if math.Abs(frac-0.000) < 0.0001 {

Review Comment:
   Validated that it did work with thee exact fractional check, should be good to go now



##########
sdks/go/examples/native_wordcap/nativepubsubio/subscriptiontracker.go:
##########
@@ -0,0 +1,80 @@
+// 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 nativepubsubio
+
+import "math"
+
+// The SubscriptionRTracker maintains a single entry string to keep up with
+// the PubSub subscription being used in the NativeRead SDF.
+type SubscriptionRTracker struct {
+	Subscription string
+	Done         bool
+}
+
+// NewSubscriptionRTracker returns an RTracker wrapping the
+// provided subscription and a "Done" boolean.
+func NewSubscriptionRTracker(entry string) *SubscriptionRTracker {
+	return &SubscriptionRTracker{Subscription: entry, Done: false}
+}
+
+// TryClaim returns true iff the given position is a string and matches the underlying
+// subscription ID.
+func (s *SubscriptionRTracker) TryClaim(pos interface{}) bool {
+	posString, ok := pos.(string)
+	return ok && posString == s.Subscription
+}
+
+// TrySplit is a no-op for the StaticRTracker in the normal case and moves the subscription
+// to the residual in the checkpointing case, marking itself as done to keep the logical checks
+// around SDF data loss happy.
+func (s *SubscriptionRTracker) TrySplit(frac float64) (primary, residual interface{}, err error) {
+	if math.Abs(frac-0.000) < 0.0001 {

Review Comment:
   Validated that it did work with the exact 0.0 check, should be good to go now



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r889310456


##########
sdks/go/examples/native_wordcap/nativepubsubio/subscriptiontracker.go:
##########
@@ -0,0 +1,80 @@
+// 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 nativepubsubio
+
+import "math"
+
+// The SubscriptionRTracker maintains a single entry string to keep up with
+// the PubSub subscription being used in the NativeRead SDF.
+type SubscriptionRTracker struct {
+	Subscription string
+	Done         bool
+}
+
+// NewSubscriptionRTracker returns an RTracker wrapping the
+// provided subscription and a "Done" boolean.
+func NewSubscriptionRTracker(entry string) *SubscriptionRTracker {
+	return &SubscriptionRTracker{Subscription: entry, Done: false}
+}
+
+// TryClaim returns true iff the given position is a string and matches the underlying
+// subscription ID.
+func (s *SubscriptionRTracker) TryClaim(pos interface{}) bool {
+	posString, ok := pos.(string)
+	return ok && posString == s.Subscription
+}
+
+// TrySplit is a no-op for the StaticRTracker in the normal case and moves the subscription
+// to the residual in the checkpointing case, marking itself as done to keep the logical checks
+// around SDF data loss happy.
+func (s *SubscriptionRTracker) TrySplit(frac float64) (primary, residual interface{}, err error) {
+	if math.Abs(frac-0.000) < 0.0001 {

Review Comment:
   Made the commit and missed the most important part



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894800913


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)

Review Comment:
   So you're suggesting something like an instantiated timer + a select `time, ok <- t.C` case that would fire in a timeout, while the message read logic would also reset that timer?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
lostluck commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894809281


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)

Review Comment:
   That's the one!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r889280288


##########
sdks/go/examples/native_wordcap/nativepubsubio/subscriptiontracker.go:
##########
@@ -0,0 +1,80 @@
+// 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 nativepubsubio
+
+import "math"
+
+// The SubscriptionRTracker maintains a single entry string to keep up with
+// the PubSub subscription being used in the NativeRead SDF.
+type SubscriptionRTracker struct {
+	Subscription string
+	Done         bool
+}
+
+// NewSubscriptionRTracker returns an RTracker wrapping the
+// provided subscription and a "Done" boolean.
+func NewSubscriptionRTracker(entry string) *SubscriptionRTracker {
+	return &SubscriptionRTracker{Subscription: entry, Done: false}
+}
+
+// TryClaim returns true iff the given position is a string and matches the underlying
+// subscription ID.
+func (s *SubscriptionRTracker) TryClaim(pos interface{}) bool {
+	posString, ok := pos.(string)
+	return ok && posString == s.Subscription
+}
+
+// TrySplit is a no-op for the StaticRTracker in the normal case and moves the subscription
+// to the residual in the checkpointing case, marking itself as done to keep the logical checks
+// around SDF data loss happy.
+func (s *SubscriptionRTracker) TrySplit(frac float64) (primary, residual interface{}, err error) {
+	if math.Abs(frac-0.000) < 0.0001 {

Review Comment:
   Definitely more of me covering my bases out of habit than necessity for how checkpointing is implemented



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894753435


##########
sdks/go/examples/native_wordcap/wordcap.go:
##########
@@ -0,0 +1,79 @@
+// 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.
+
+// native_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.

Review Comment:
   Updated the comment



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894784676


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)
+
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.ProcessedMessages = append(r.ProcessedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+			default:
+				log.Debug(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := NewPubSubRead(context.Background(), project, topic, subscription)
+	if err != nil {
+		panic(err)
+	}
+	return beam.ParDo(s, psRead, beam.Impulse(s))
+}
+
+// PubSubWrite is a structural DoFn representing writes to a given PubSub topic.
+type PubSubWrite struct {

Review Comment:
   Fixed



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck commented on pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
lostluck commented on PR #17955:
URL: https://github.com/apache/beam/pull/17955#issuecomment-1152730729

   Thanks!
   
   As an example it's *most likely* to be viewed live, so it's easy enough to show/see improvements later.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] damccorm commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
damccorm commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r889278013


##########
sdks/go/examples/native_wordcap/nativepubsubio/subscriptiontracker.go:
##########
@@ -0,0 +1,80 @@
+// 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 nativepubsubio
+
+import "math"
+
+// The SubscriptionRTracker maintains a single entry string to keep up with
+// the PubSub subscription being used in the NativeRead SDF.
+type SubscriptionRTracker struct {
+	Subscription string
+	Done         bool
+}
+
+// NewSubscriptionRTracker returns an RTracker wrapping the
+// provided subscription and a "Done" boolean.
+func NewSubscriptionRTracker(entry string) *SubscriptionRTracker {
+	return &SubscriptionRTracker{Subscription: entry, Done: false}
+}
+
+// TryClaim returns true iff the given position is a string and matches the underlying
+// subscription ID.
+func (s *SubscriptionRTracker) TryClaim(pos interface{}) bool {
+	posString, ok := pos.(string)
+	return ok && posString == s.Subscription
+}
+
+// TrySplit is a no-op for the StaticRTracker in the normal case and moves the subscription
+// to the residual in the checkpointing case, marking itself as done to keep the logical checks
+// around SDF data loss happy.
+func (s *SubscriptionRTracker) TrySplit(frac float64) (primary, residual interface{}, err error) {
+	if math.Abs(frac-0.000) < 0.0001 {

Review Comment:
   Does `frac == 0.0` not work? If so, that's unfortunate - I imagine this won't be a totally uncommon pattern.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894752988


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {

Review Comment:
   Done.



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)
+
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.ProcessedMessages = append(r.ProcessedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+			default:
+				log.Debug(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := NewPubSubRead(context.Background(), project, topic, subscription)
+	if err != nil {
+		panic(err)
+	}
+	return beam.ParDo(s, psRead, beam.Impulse(s))
+}
+
+// PubSubWrite is a structural DoFn representing writes to a given PubSub topic.
+type PubSubWrite struct {
+	ProjectID string
+	Topic     string
+	Client    *pubsub.Client
+}
+
+// ProcessElement takes a []byte element and publishes it to the provided PubSub
+// topic.
+func (w *PubSubWrite) ProcessElement(elm []byte) error {
+	// Initialize PubSub client if one has not been created already
+	if w.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), w.ProjectID)
+		if err != nil {
+			return err
+		}
+		w.Client = client
+	}
+	top := w.Client.Topic(w.Topic)
+
+	psMess := &pubsub.Message{Data: elm}
+	result := top.Publish(context.Background(), psMess)
+	if _, err := result.Get(context.Background()); err != nil {
+		return err
+	}
+	return nil
+}
+
+// NewPubSubWrite inserts a write to a PubSub topic into the pipeline.
+func NewPubSubWrite(ctx context.Context, projectID, topic string) (*PubSubWrite, error) {

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
lostluck commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894809823


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)

Review Comment:
   Ah I see I forgot to remove my first Boolean suggestion, which I thought I cleaned up after reading timer.Reset's doc.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
lostluck commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894068750


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {

Review Comment:
   Don't export the types if a user is never supposed to view it.



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {

Review Comment:
   It's good to move this to a function and to have documentation, but unless you expect packages outside of this one to call it, don't export it.



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)

Review Comment:
   Prefer creating a timer instead of sleeing: https://pkg.go.dev/time#NewTimer
   
   Then you can "react" to the timer firing with the t.C channel.
   
   And then you can avoid the default statement, and sleeping while we wait. Technically, you could also avoid the buffered channel too, which will let everything block. A boolean can be added WRT to see if there were any messages since the last sleep, which can be tripped in the messChan case.
   
   The timer case can then check if there was a message, and if so, simply go back to sleep, and if not, reset the bundle. Or something with the timer's Reset method.
   
   The only awkward bit is we'd probably want to self checkpoint a little sooner than the Ack deadline on any of the messages, so the runner has time to trigger bundleFinalizing for the messages.  But since the client automatically [extends the AckDeadline to up to 60 minutes by default](https://pkg.go.dev/cloud.google.com/go/pubsub#hdr-Deadlines) from it's start of 10 seconds, we're probably fine.
   



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription

Review Comment:
   Definitely a "maybe try this later" improvement, but to allow for more parallelism, if the backlog on a subscription becomes large for example, it could be worth having a "primary" and "secondary" restrictions, where a primary can split into secondary restrictions if the backlog is getting too large, and "secondary" restrictions can terminate when it gets too small. This would require information to be passed between the watermark estimation and the tracker to manage the restrictions, but that's not impossible.



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)
+
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.ProcessedMessages = append(r.ProcessedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+			default:
+				log.Debug(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := NewPubSubRead(context.Background(), project, topic, subscription)
+	if err != nil {
+		panic(err)
+	}
+	return beam.ParDo(s, psRead, beam.Impulse(s))
+}
+
+// PubSubWrite is a structural DoFn representing writes to a given PubSub topic.
+type PubSubWrite struct {
+	ProjectID string
+	Topic     string
+	Client    *pubsub.Client
+}
+
+// ProcessElement takes a []byte element and publishes it to the provided PubSub
+// topic.
+func (w *PubSubWrite) ProcessElement(elm []byte) error {
+	// Initialize PubSub client if one has not been created already
+	if w.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), w.ProjectID)
+		if err != nil {
+			return err
+		}
+		w.Client = client
+	}

Review Comment:
   Same thing, put it in a Setup method.



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}

Review Comment:
   This should happen in a Setup(context.Context) error method (which can return error).



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)
+
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.ProcessedMessages = append(r.ProcessedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+			default:
+				log.Debug(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := NewPubSubRead(context.Background(), project, topic, subscription)
+	if err != nil {
+		panic(err)
+	}
+	return beam.ParDo(s, psRead, beam.Impulse(s))
+}
+
+// PubSubWrite is a structural DoFn representing writes to a given PubSub topic.
+type PubSubWrite struct {

Review Comment:
   Same thing, don't export it if it's not necessary for package users to use it directly.



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())

Review Comment:
   This ProcessElement method should accept a context.Context as the first parameter, and then use that as the basis of this cancelable context.



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message

Review Comment:
   I'm surprised these passed through, but if these are only needed at pipeline execution time, unexport the fields so they aren't attempted to be serialized. I doubt the client is meaningfully serializable.



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)
+
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.ProcessedMessages = append(r.ProcessedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+			default:
+				log.Debug(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := NewPubSubRead(context.Background(), project, topic, subscription)
+	if err != nil {
+		panic(err)
+	}
+	return beam.ParDo(s, psRead, beam.Impulse(s))
+}
+
+// PubSubWrite is a structural DoFn representing writes to a given PubSub topic.
+type PubSubWrite struct {
+	ProjectID string
+	Topic     string
+	Client    *pubsub.Client
+}
+
+// ProcessElement takes a []byte element and publishes it to the provided PubSub
+// topic.
+func (w *PubSubWrite) ProcessElement(elm []byte) error {
+	// Initialize PubSub client if one has not been created already
+	if w.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), w.ProjectID)
+		if err != nil {
+			return err
+		}
+		w.Client = client
+	}
+	top := w.Client.Topic(w.Topic)
+
+	psMess := &pubsub.Message{Data: elm}
+	result := top.Publish(context.Background(), psMess)
+	if _, err := result.Get(context.Background()); err != nil {
+		return err
+	}
+	return nil
+}
+
+// NewPubSubWrite inserts a write to a PubSub topic into the pipeline.
+func NewPubSubWrite(ctx context.Context, projectID, topic string) (*PubSubWrite, error) {

Review Comment:
   Unexport this please.



##########
sdks/go/examples/native_wordcap/wordcap.go:
##########
@@ -0,0 +1,79 @@
+// 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.
+
+// native_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.

Review Comment:
   Technically, this one *doesn't* have to be run on dataflow! It's portable. Granted it'd be up to the user to have the environment to have whatever credentials present for lookup for the GCP permissions. 



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}

Review Comment:
   Prefer nil for empty slices. Code shouldn't be making a distinction for nil and non-nil and it doesn't matter to append.



##########
sdks/go/examples/native_wordcap/wordcap.go:
##########
@@ -0,0 +1,79 @@
+// 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.
+
+// native_wordcap is a toy streaming pipeline that uses PubSub. It

Review Comment:
   Clarify that it ises the portable/native pubsub, to distinguish itself from the other wordcap.



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)
+
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.ProcessedMessages = append(r.ProcessedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+			default:
+				log.Debug(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := NewPubSubRead(context.Background(), project, topic, subscription)
+	if err != nil {
+		panic(err)
+	}
+	return beam.ParDo(s, psRead, beam.Impulse(s))
+}
+
+// PubSubWrite is a structural DoFn representing writes to a given PubSub topic.
+type PubSubWrite struct {
+	ProjectID string
+	Topic     string
+	Client    *pubsub.Client
+}
+
+// ProcessElement takes a []byte element and publishes it to the provided PubSub
+// topic.
+func (w *PubSubWrite) ProcessElement(elm []byte) error {
+	// Initialize PubSub client if one has not been created already
+	if w.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), w.ProjectID)
+		if err != nil {
+			return err
+		}
+		w.Client = client
+	}
+	top := w.Client.Topic(w.Topic)
+
+	psMess := &pubsub.Message{Data: elm}
+	result := top.Publish(context.Background(), psMess)
+	if _, err := result.Get(context.Background()); err != nil {

Review Comment:
   Have a context.Context parameter and use that instead of context.Background()



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)
+
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.ProcessedMessages = append(r.ProcessedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+			default:
+				log.Debug(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := NewPubSubRead(context.Background(), project, topic, subscription)
+	if err != nil {
+		panic(err)
+	}
+	return beam.ParDo(s, psRead, beam.Impulse(s))
+}
+
+// PubSubWrite is a structural DoFn representing writes to a given PubSub topic.
+type PubSubWrite struct {
+	ProjectID string
+	Topic     string
+	Client    *pubsub.Client

Review Comment:
   Unexport the fields that don't need to be serialized please.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] jrmccluskey commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894751769


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)
+
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.ProcessedMessages = append(r.ProcessedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+			default:
+				log.Debug(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := NewPubSubRead(context.Background(), project, topic, subscription)
+	if err != nil {
+		panic(err)
+	}
+	return beam.ParDo(s, psRead, beam.Impulse(s))
+}
+
+// PubSubWrite is a structural DoFn representing writes to a given PubSub topic.
+type PubSubWrite struct {
+	ProjectID string
+	Topic     string
+	Client    *pubsub.Client
+}
+
+// ProcessElement takes a []byte element and publishes it to the provided PubSub
+// topic.
+func (w *PubSubWrite) ProcessElement(elm []byte) error {
+	// Initialize PubSub client if one has not been created already
+	if w.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), w.ProjectID)
+		if err != nil {
+			return err
+		}
+		w.Client = client
+	}
+	top := w.Client.Topic(w.Topic)
+
+	psMess := &pubsub.Message{Data: elm}
+	result := top.Publish(context.Background(), psMess)
+	if _, err := result.Get(context.Background()); err != nil {

Review Comment:
   Fixed



##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,236 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn4x2[beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&PubSubRead{})
+	register.DoFn1x1[[]byte, error](&PubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// PubSubRead is a structural DoFn representing a read from a given subscription ID.
+type PubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	Client            *pubsub.Client
+	ProcessedMessages []*pubsub.Message
+}
+
+// NewPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func NewPubSubRead(ctx context.Context, projectID, topic, subscription string) (*PubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &PubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *PubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *PubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *PubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *PubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *PubSubRead) ProcessElement(bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.ProcessedMessages {
+			m.Ack()
+		}
+		r.ProcessedMessages = []*pubsub.Message{}
+		return nil
+	})
+
+	// Initialize PubSub client if one has not been created already
+	if r.Client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return sdf.StopProcessing(), err
+		}
+		r.Client = client
+	}
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.Client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(context.Background())
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)
+		go func(sendch chan<- *pubsub.Message) {
+			err := sub.Receive(ctx, func(ctx context.Context, m *pubsub.Message) {
+				messChan <- m
+			})
+			if (err != nil) && (err != context.Canceled) {
+				log.Errorf(ctx, "error reading from PubSub: %v, stopping processing", err)
+				cFn()
+				close(messChan)
+			}
+		}(messChan)
+
+		// Give the goroutines time to start polling.
+		time.Sleep(5 * time.Second)
+
+		for {
+			select {
+			case m, ok := <-messChan:
+				if !ok {
+					log.Debug(context.Background(), "stopping bundle processing")
+					return sdf.StopProcessing(), nil
+				}
+				r.ProcessedMessages = append(r.ProcessedMessages, m)
+				emit(beam.EventTime(m.PublishTime.UnixMilli()), m.Data)
+			default:
+				log.Debug(context.Background(), "cancelling receive context, scheduling resumption")
+				cFn()
+				return sdf.ResumeProcessingIn(10 * time.Second), nil
+			}
+		}
+	}
+}
+
+// NativeRead reads messages from a PubSub topic in a streaming context, outputting
+// received messages as a PCollection of byte slices. If the provided subscription
+// name exists for the given topic, the DoFn will read from that subscription; otherwise,
+// a new subscription with the given subscription name will be created and read from.
+//
+// This feature is experimental and subject to change, including its behavior and function signature.
+// Please use the cross-language implementation Read() instead.
+func NativeRead(s beam.Scope, project, topic, subscription string) beam.PCollection {
+	s = s.Scope("pubsubio.NativeRead")
+
+	psRead, err := NewPubSubRead(context.Background(), project, topic, subscription)
+	if err != nil {
+		panic(err)
+	}
+	return beam.ParDo(s, psRead, beam.Impulse(s))
+}
+
+// PubSubWrite is a structural DoFn representing writes to a given PubSub topic.
+type PubSubWrite struct {
+	ProjectID string
+	Topic     string
+	Client    *pubsub.Client

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck merged pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
lostluck merged PR #17955:
URL: https://github.com/apache/beam/pull/17955


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
lostluck commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r894828559


##########
sdks/go/examples/native_wordcap/nativepubsubio/native.go:
##########
@@ -0,0 +1,247 @@
+// 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 nativepubsubio contains a Golang implementation of streaming reads
+// and writes to PubSub. This is not as fully featured as the cross-language
+// pubsubio package present in the Beam Go repository and should not be used
+// in place of it.
+package nativepubsubio
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"time"
+
+	"cloud.google.com/go/pubsub"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx"
+)
+
+func init() {
+	register.DoFn5x2[context.Context, beam.BundleFinalization, *sdf.LockRTracker, []byte, func(beam.EventTime, []byte), sdf.ProcessContinuation, error](&pubSubRead{})
+	register.DoFn2x1[context.Context, []byte, error](&pubSubWrite{})
+	register.Emitter2[beam.EventTime, []byte]()
+}
+
+// pubSubRead is a structural DoFn representing a read from a given subscription ID.
+type pubSubRead struct {
+	ProjectID         string
+	Subscription      string
+	client            *pubsub.Client
+	processedMessages []*pubsub.Message
+}
+
+// newPubSubRead inserts an unbounded read from a PubSub topic into the pipeline. If an existing subscription
+// is provided, the DoFn will read using that subscription; otherwise, a new subscription to the topic
+// will be created using the provided subscription name.
+func newPubSubRead(ctx context.Context, projectID, topic, subscription string) (*pubSubRead, error) {
+	if topic == "" {
+		return nil, errors.New("please provide either a topic to read from")
+	}
+	client, err := pubsub.NewClient(ctx, projectID)
+	if err != nil {
+		return nil, err
+	}
+
+	top := client.Topic(topic)
+	if ok, err := top.Exists(ctx); !ok || err != nil {
+		return nil, fmt.Errorf("failed to get topic; exists: %v, error: %v", ok, err)
+	}
+	sub, err := pubsubx.EnsureSubscription(ctx, client, topic, subscription)
+	if err != nil {
+		return nil, err
+	}
+	return &pubSubRead{ProjectID: projectID, Subscription: sub.ID()}, nil
+}
+
+// CreateInitialRestriction() establishes the PubSub subscription ID as the
+// initial restriction
+func (r *pubSubRead) CreateInitialRestriction(_ []byte) string {
+	return r.Subscription
+}
+
+// CreateTracker wraps the PubSub subscription ID in a StaticRTracker
+// and applies a mutex via LockRTracker.
+func (r *pubSubRead) CreateTracker(rest string) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(NewSubscriptionRTracker(rest))
+}
+
+// RestrictionSize always returns 1.0, as the restriction is always 1 subscription.
+func (r *pubSubRead) RestrictionSize(_ []byte, rest string) float64 {
+	return 1.0
+}
+
+// SplitRestriction is a no-op as the restriction cannot be split.
+func (r *pubSubRead) SplitRestriction(_ []byte, rest string) []string {
+	return []string{rest}
+}
+
+// Setup initializes a PubSub client if one has not been created already
+func (r *pubSubRead) Setup(ctx context.Context) error {
+	if r.client == nil {
+		client, err := pubsub.NewClient(context.Background(), r.ProjectID)
+		if err != nil {
+			return err
+		}
+		r.client = client
+	}
+	return nil
+}
+
+var messageTimeout time.Duration = 5 * time.Second
+
+// ProcessElement initializes a PubSub client if one has not been created already, reads from the PubSub subscription,
+// and emits elements as it reads them. If no messages are available, the DoFn will schedule itself to resume processing
+// later. If polling the subscription returns an error, the error will be logged and the DoFn will not reschedule itself.
+func (r *pubSubRead) ProcessElement(ctx context.Context, bf beam.BundleFinalization, rt *sdf.LockRTracker, _ []byte, emit func(beam.EventTime, []byte)) (sdf.ProcessContinuation, error) {
+	// Register finalization callback
+	bf.RegisterCallback(5*time.Minute, func() error {
+		for _, m := range r.processedMessages {
+			m.Ack()
+		}
+		r.processedMessages = nil
+		return nil
+	})
+
+	for {
+		ok := rt.TryClaim(r.Subscription)
+		if !ok {
+			return sdf.ResumeProcessingIn(5 * time.Second), nil
+		}
+		sub := r.client.Subscription(r.Subscription)
+		ctx, cFn := context.WithCancel(ctx)
+
+		// Because emitters are not thread safe and synchronous Receive() behavior
+		// is deprecated, we have to collect messages in a goroutine and pipe them
+		// out through a channel.
+		messChan := make(chan *pubsub.Message, 1)

Review Comment:
   No action required, I just had to confirm to myself that the "comma ok" channel receive was still blocking. It is, so this is fantastic.
   
   https://go.dev/play/p/p93DFrK4DCA



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] codecov[bot] commented on pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
codecov[bot] commented on PR #17955:
URL: https://github.com/apache/beam/pull/17955#issuecomment-1146094845

   # [Codecov](https://codecov.io/gh/apache/beam/pull/17955?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#17955](https://codecov.io/gh/apache/beam/pull/17955?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d56c8db) into [master](https://codecov.io/gh/apache/beam/commit/86d8e30409565559bc784a6117c8ec2a7ae481d1?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (86d8e30) will **not change** coverage.
   > The diff coverage is `n/a`.
   
   ```diff
   @@           Coverage Diff           @@
   ##           master   #17955   +/-   ##
   =======================================
     Coverage   74.07%   74.07%           
   =======================================
     Files         697      697           
     Lines       91986    91986           
   =======================================
     Hits        68141    68141           
     Misses      22596    22596           
     Partials     1249     1249           
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | go | `50.83% <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17955?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17955?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [86d8e30...d56c8db](https://codecov.io/gh/apache/beam/pull/17955?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] asf-ci commented on pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
asf-ci commented on PR #17955:
URL: https://github.com/apache/beam/pull/17955#issuecomment-1146092097

   Can one of the admins verify this patch?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] damccorm commented on a diff in pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
damccorm commented on code in PR #17955:
URL: https://github.com/apache/beam/pull/17955#discussion_r889303813


##########
sdks/go/examples/native_wordcap/nativepubsubio/subscriptiontracker.go:
##########
@@ -0,0 +1,80 @@
+// 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 nativepubsubio
+
+import "math"
+
+// The SubscriptionRTracker maintains a single entry string to keep up with
+// the PubSub subscription being used in the NativeRead SDF.
+type SubscriptionRTracker struct {
+	Subscription string
+	Done         bool
+}
+
+// NewSubscriptionRTracker returns an RTracker wrapping the
+// provided subscription and a "Done" boolean.
+func NewSubscriptionRTracker(entry string) *SubscriptionRTracker {
+	return &SubscriptionRTracker{Subscription: entry, Done: false}
+}
+
+// TryClaim returns true iff the given position is a string and matches the underlying
+// subscription ID.
+func (s *SubscriptionRTracker) TryClaim(pos interface{}) bool {
+	posString, ok := pos.(string)
+	return ok && posString == s.Subscription
+}
+
+// TrySplit is a no-op for the StaticRTracker in the normal case and moves the subscription
+// to the residual in the checkpointing case, marking itself as done to keep the logical checks
+// around SDF data loss happy.
+func (s *SubscriptionRTracker) TrySplit(frac float64) (primary, residual interface{}, err error) {
+	if math.Abs(frac-0.000) < 0.0001 {

Review Comment:
   Did you forget to push the change?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] github-actions[bot] commented on pull request #17955: Add native PubSub IO prototype to Go

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #17955:
URL: https://github.com/apache/beam/pull/17955#issuecomment-1146296710

   R: @lostluck for final approval


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org