-
Notifications
You must be signed in to change notification settings - Fork 4.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[Go SDK]: Create natsio.Write transform for writing to NATS (#29184)
* Create natsio.Write transform for writing to NATS * Emit representation of acknowledged message from writeFn * Use type map[string][]string for ProduceMessage headers
- Loading branch information
1 parent
ad28719
commit cf560ca
Showing
8 changed files
with
664 additions
and
3 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,58 @@ | ||
// 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 natsio contains transforms for interacting with NATS. | ||
package natsio | ||
|
||
import ( | ||
"fmt" | ||
|
||
"github.com/nats-io/nats.go" | ||
"github.com/nats-io/nats.go/jetstream" | ||
) | ||
|
||
type natsFn struct { | ||
URI string | ||
CredsFile string | ||
nc *nats.Conn | ||
js jetstream.JetStream | ||
} | ||
|
||
func (fn *natsFn) Setup() error { | ||
var opts []nats.Option | ||
if fn.CredsFile != "" { | ||
opts = append(opts, nats.UserCredentials(fn.CredsFile)) | ||
} | ||
|
||
conn, err := nats.Connect(fn.URI, opts...) | ||
if err != nil { | ||
return fmt.Errorf("error connecting to NATS: %v", err) | ||
} | ||
fn.nc = conn | ||
|
||
js, err := jetstream.New(fn.nc) | ||
if err != nil { | ||
return fmt.Errorf("error creating JetStream context: %v", err) | ||
} | ||
fn.js = js | ||
|
||
return nil | ||
} | ||
|
||
func (fn *natsFn) Teardown() { | ||
if fn.nc != nil { | ||
fn.nc.Close() | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,55 @@ | ||
// 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 natsio_test | ||
|
||
import ( | ||
"context" | ||
"log" | ||
|
||
"github.com/apache/beam/sdks/v2/go/pkg/beam" | ||
"github.com/apache/beam/sdks/v2/go/pkg/beam/io/natsio" | ||
"github.com/apache/beam/sdks/v2/go/pkg/beam/x/beamx" | ||
"github.com/nats-io/nats.go" | ||
) | ||
|
||
func ExampleWrite() { | ||
beam.Init() | ||
|
||
p, s := beam.NewPipelineWithRoot() | ||
|
||
uri := "nats://localhost:4222" | ||
msgs := []natsio.ProducerMessage{ | ||
{ | ||
Subject: "events.1", | ||
ID: "123", | ||
Data: []byte("hello"), | ||
Headers: nats.Header{"key": []string{"val1"}}, | ||
}, | ||
{ | ||
Subject: "events.2", | ||
ID: "124", | ||
Data: []byte("world"), | ||
Headers: nats.Header{"key": []string{"val2"}}, | ||
}, | ||
} | ||
|
||
input := beam.CreateList(s, msgs) | ||
natsio.Write(s, uri, input) | ||
|
||
if err := beamx.Run(context.Background(), p); err != nil { | ||
log.Fatalf("Failed to execute job: %v", err) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,130 @@ | ||
// 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 natsio | ||
|
||
import ( | ||
"context" | ||
"testing" | ||
|
||
"github.com/nats-io/nats-server/v2/server" | ||
"github.com/nats-io/nats-server/v2/test" | ||
"github.com/nats-io/nats.go" | ||
"github.com/nats-io/nats.go/jetstream" | ||
) | ||
|
||
func newServer(t *testing.T) *server.Server { | ||
t.Helper() | ||
|
||
opts := &test.DefaultTestOptions | ||
opts.Port = server.RANDOM_PORT | ||
opts.JetStream = true | ||
|
||
srv := test.RunServer(opts) | ||
t.Cleanup(srv.Shutdown) | ||
|
||
return srv | ||
} | ||
|
||
func newConn(t *testing.T, uri string) *nats.Conn { | ||
t.Helper() | ||
|
||
conn, err := nats.Connect(uri) | ||
if err != nil { | ||
t.Fatalf("Failed to connect to NATS: %v", err) | ||
} | ||
t.Cleanup(conn.Close) | ||
|
||
return conn | ||
} | ||
|
||
func newJetStream(t *testing.T, conn *nats.Conn) jetstream.JetStream { | ||
t.Helper() | ||
|
||
js, err := jetstream.New(conn) | ||
if err != nil { | ||
t.Fatalf("Failed to create JetStream instance: %v", err) | ||
} | ||
|
||
return js | ||
} | ||
|
||
func createStream( | ||
t *testing.T, | ||
ctx context.Context, | ||
js jetstream.JetStream, | ||
stream string, | ||
subjects []string, | ||
) jetstream.Stream { | ||
t.Helper() | ||
|
||
cfg := jetstream.StreamConfig{ | ||
Name: stream, | ||
Subjects: subjects, | ||
} | ||
str, err := js.CreateStream(ctx, cfg) | ||
if err != nil { | ||
t.Fatalf("Failed to create stream: %v", err) | ||
} | ||
|
||
t.Cleanup(func() { | ||
if err := js.DeleteStream(ctx, stream); err != nil { | ||
t.Fatalf("Failed to delete stream: %v", err) | ||
} | ||
}) | ||
|
||
return str | ||
} | ||
|
||
func createConsumer( | ||
t *testing.T, | ||
ctx context.Context, | ||
js jetstream.JetStream, | ||
stream string, | ||
subjects []string, | ||
) jetstream.Consumer { | ||
t.Helper() | ||
|
||
cfg := jetstream.OrderedConsumerConfig{ | ||
FilterSubjects: subjects, | ||
} | ||
cons, err := js.OrderedConsumer(ctx, stream, cfg) | ||
if err != nil { | ||
t.Fatalf("Failed to create consumer: %v", err) | ||
} | ||
|
||
return cons | ||
} | ||
|
||
func fetchMessages(t *testing.T, cons jetstream.Consumer, size int) []jetstream.Msg { | ||
t.Helper() | ||
|
||
msgs, err := cons.FetchNoWait(size) | ||
if err != nil { | ||
t.Fatalf("Failed to fetch messages: %v", err) | ||
} | ||
|
||
var result []jetstream.Msg | ||
|
||
for msg := range msgs.Messages() { | ||
if err := msg.Ack(); err != nil { | ||
t.Fatalf("Failed to ack message: %v", err) | ||
} | ||
|
||
result = append(result, msg) | ||
} | ||
|
||
return result | ||
} |
Oops, something went wrong.