This repository has been archived by the owner on Nov 3, 2022. It is now read-only.
forked from apache/beam
/
kafka.go
77 lines (67 loc) · 2.48 KB
/
kafka.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
// 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 kafka contains integration tests for cross-language Kafka IO
// transforms.
package kafka
import (
"bytes"
"fmt"
"github.com/apache/beam/sdks/go/pkg/beam"
"github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
"github.com/apache/beam/sdks/go/pkg/beam/io/xlang/kafkaio"
"github.com/apache/beam/sdks/go/pkg/beam/testing/passert"
"github.com/google/uuid"
)
func appendUuid(prefix string) string {
return fmt.Sprintf("%v_%v", prefix, uuid.New())
}
// Constants for the BasicPipeline.
const (
numRecords = 1000
basicTopic = "xlang_kafkaio_basic_test"
)
// BasicPipeline creates a pipeline that writes and then reads a range of ints
// to and from a Kafka topic and asserts that all elements are present. This
// function requires an expansion service address and a Kafka bootstrap server
// address.
func BasicPipeline(expansionAddr, bootstrapAddr string) *beam.Pipeline {
topic := appendUuid(basicTopic)
inputs := make([]int, numRecords)
for i := 0; i < numRecords; i++ {
inputs[i] = i
}
p, s := beam.NewPipelineWithRoot()
ins := beam.CreateList(s, inputs)
// Write to Kafka
encoded := beam.ParDo(s, func(i int) ([]byte, error) {
var buf bytes.Buffer
err := coder.EncodeVarInt(int64(i), &buf)
return buf.Bytes(), err
}, ins)
keyed := beam.ParDo(s, func(b []byte) ([]byte, []byte) {
return []byte(""), b
}, encoded)
kafkaio.Write(s, expansionAddr, bootstrapAddr, topic, keyed)
// Read from Kafka
reads := kafkaio.Read(s, expansionAddr, bootstrapAddr, []string{topic})
vals := beam.DropKey(s, reads)
decoded := beam.ParDo(s, func(b []byte) (int, error) {
buf := bytes.NewBuffer(b)
i, err := coder.DecodeVarInt(buf)
return int(i), err
}, vals)
passert.Equals(s, decoded, ins)
return p
}