github.com/apache/beam/sdks/v2@v2.48.2/go/test/integration/io/xlang/kafka/kafka_test.go (about) 1 // Licensed to the Apache Software Foundation (ASF) under one or more 2 // contributor license agreements. See the NOTICE file distributed with 3 // this work for additional information regarding copyright ownership. 4 // The ASF licenses this file to You under the Apache License, Version 2.0 5 // (the "License"); you may not use this file except in compliance with 6 // the License. You may obtain a copy of the License at 7 // 8 // http://www.apache.org/licenses/LICENSE-2.0 9 // 10 // Unless required by applicable law or agreed to in writing, software 11 // distributed under the License is distributed on an "AS IS" BASIS, 12 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 // See the License for the specific language governing permissions and 14 // limitations under the License. 15 16 package kafka 17 18 import ( 19 "flag" 20 "fmt" 21 "log" 22 "testing" 23 24 "github.com/apache/beam/sdks/v2/go/pkg/beam" 25 _ "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/dataflow" 26 _ "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/flink" 27 _ "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/samza" 28 _ "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/spark" 29 "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/ptest" 30 "github.com/apache/beam/sdks/v2/go/test/integration" 31 ) 32 33 // bootstrapAddr should be set by TestMain once a Kafka cluster has been 34 // started, and is used by each test. 35 var bootstrapAddr string 36 var expansionAddr string // Populate with expansion address labelled "io". 37 38 const ( 39 basicTopic = "xlang_kafkaio_basic_test" 40 numRecords = 1000 41 ) 42 43 func checkFlags(t *testing.T) { 44 if bootstrapAddr == "" { 45 t.Skip("No bootstrap server address provided.") 46 } 47 } 48 49 // TestBasicPipeline basic writes and reads from Kafka with as few optional 50 // parameters or extra features as possible. 51 func TestKafkaIO_BasicReadWrite(t *testing.T) { 52 integration.CheckFilters(t) 53 checkFlags(t) 54 55 inputs := make([]int, numRecords) 56 for i := 0; i < numRecords; i++ { 57 inputs[i] = i 58 } 59 topic := appendUuid(basicTopic) 60 61 write := WritePipeline(expansionAddr, bootstrapAddr, topic, inputs) 62 ptest.RunAndValidate(t, write) 63 read := ReadPipeline(expansionAddr, bootstrapAddr, topic, inputs) 64 ptest.RunAndValidate(t, read) 65 } 66 67 // TestMain starts up a Kafka cluster from integration.KafkaJar before running 68 // tests through ptest.Main. 69 func TestMain(m *testing.M) { 70 flag.Parse() 71 beam.Init() 72 73 // Start local Kafka cluster and defer its shutdown. 74 if *integration.BootstrapServers != "" { 75 bootstrapAddr = *integration.BootstrapServers 76 } else if *integration.KafkaJar != "" { 77 cluster, err := runLocalKafka(*integration.KafkaJar, *integration.KafkaJarTimeout) 78 if err != nil { 79 panic(fmt.Errorf("kafka cluster failed to start: %v", err)) 80 } 81 defer func() { cluster.Shutdown() }() 82 bootstrapAddr = cluster.bootstrapAddr 83 } 84 85 services := integration.NewExpansionServices() 86 defer func() { services.Shutdown() }() 87 addr, err := services.GetAddr("io") 88 if err != nil { 89 log.Printf("skipping missing expansion service: %v", err) 90 } else { 91 expansionAddr = addr 92 } 93 94 ptest.MainRet(m) 95 }