/
kafka.go
85 lines (74 loc) · 3.21 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
78
79
80
81
82
83
84
85
// 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/v2/go/pkg/beam"
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder"
"github.com/apache/beam/sdks/v2/go/pkg/beam/io/xlang/kafkaio"
"github.com/apache/beam/sdks/v2/go/pkg/beam/testing/passert"
"github.com/google/uuid"
)
func appendUuid(prefix string) string {
return fmt.Sprintf("%v_%v", prefix, uuid.New())
}
// writeList encodes a list of ints and sends encoded ints to Kafka.
func writeInts(s beam.Scope, expansionAddr, bootstrapAddr, topic string, inputs []int) {
s = s.Scope("kafka_test.WriteListToKafka")
ins := beam.CreateList(s, inputs)
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)
}
// readList reads a set number of elements from Kafka and decodes them to ints.
func readInts(s beam.Scope, expansionAddr, bootstrapAddr, topic string, numRecords int64) beam.PCollection {
s = s.Scope("kafka_test.ReadListFromKafka")
reads := kafkaio.Read(s, expansionAddr, bootstrapAddr, []string{topic},
kafkaio.MaxNumRecords(numRecords),
kafkaio.ConsumerConfigs(map[string]string{"auto.offset.reset": "earliest"}))
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)
return decoded
}
// WritePipeline creates a pipeline that writes a given slice of ints to Kafka.
func WritePipeline(expansionAddr, bootstrapAddr, topic string, inputs []int) *beam.Pipeline {
p, s := beam.NewPipelineWithRoot()
writeInts(s, expansionAddr, bootstrapAddr, topic, inputs)
return p
}
// ReadPipeline creates a pipeline that reads ints from Kafka and asserts that
// they match a given slice of ints. This reads a number of records equal to
// the length of the given slice.
func ReadPipeline(expansionAddr, bootstrapAddr, topic string, inputs []int) *beam.Pipeline {
p, s := beam.NewPipelineWithRoot()
result := readInts(s, expansionAddr, bootstrapAddr, topic, int64(len(inputs)))
// Validate that records read from Kafka match the given slice.
ins := beam.CreateList(s, inputs)
passert.Equals(s, result, ins)
return p
}