forked from confluentinc/kafka-streams-examples
-
Notifications
You must be signed in to change notification settings - Fork 1
/
SpecificAvroScalaIntegrationTest.scala
124 lines (107 loc) · 5.07 KB
/
SpecificAvroScalaIntegrationTest.scala
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
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
/*
* Copyright Confluent Inc.
*
* Licensed 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 io.confluent.examples.streams
import java.util.{Collections, Properties}
import io.confluent.examples.streams.avro.WikiFeed
import io.confluent.examples.streams.kafka.EmbeddedSingleNodeKafkaCluster
import io.confluent.kafka.serializers.{AbstractKafkaAvroSerDeConfig, KafkaAvroDeserializer, KafkaAvroDeserializerConfig, KafkaAvroSerializer}
import io.confluent.kafka.streams.serdes.avro.SpecificAvroSerde
import org.apache.kafka.clients.consumer.ConsumerConfig
import org.apache.kafka.clients.producer.ProducerConfig
import org.apache.kafka.common.serialization._
import org.apache.kafka.streams.scala.StreamsBuilder
import org.apache.kafka.streams.{KafkaStreams, StreamsConfig}
import org.junit._
import org.scalatest.junit.AssertionsForJUnit
/**
* End-to-end integration test that demonstrates how to work on Specific Avro data.
*
* See [[GenericAvroScalaIntegrationTest]] for the equivalent Generic Avro integration test.
*/
class SpecificAvroScalaIntegrationTest extends AssertionsForJUnit {
import org.apache.kafka.streams.scala.Serdes._
import org.apache.kafka.streams.scala.ImplicitConversions._
private val privateCluster: EmbeddedSingleNodeKafkaCluster = new EmbeddedSingleNodeKafkaCluster
@Rule def cluster: EmbeddedSingleNodeKafkaCluster = privateCluster
private val inputTopic = "inputTopic"
private val outputTopic = "output-topic"
@Before
def startKafkaCluster() {
cluster.createTopic(inputTopic, 2, 1)
cluster.createTopic(outputTopic)
}
@Test
def shouldRoundTripGenericAvroDataThroughKafka() {
val f: WikiFeed = WikiFeed.newBuilder.setUser("alice").setIsNew(true).setContent("lorem ipsum").build
val inputValues: Seq[WikiFeed] = Seq(f)
//
// Step 1: Configure and start the processor topology.
//
val builder = new StreamsBuilder
val streamsConfiguration: Properties = {
val p = new Properties()
p.put(StreamsConfig.APPLICATION_ID_CONFIG, "specific-avro-scala-integration-test")
p.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers())
p.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, cluster.schemaRegistryUrl)
p.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
p
}
// Make an implicit serde available for WikiFeed, which is required for operations such as `to()` below.
implicit val specificAvroSerde: Serde[WikiFeed] = {
val sas = new SpecificAvroSerde[WikiFeed]
val isKeySerde: Boolean = false
sas.configure(Collections.singletonMap(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, cluster.schemaRegistryUrl), isKeySerde)
sas
}
// Write the input data as-is to the output topic.
builder.stream[String, WikiFeed](inputTopic).to(outputTopic)
val streams: KafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration)
streams.start()
//
// Step 2: Produce some input data to the input topic.
//
val producerConfig: Properties = {
val p = new Properties()
p.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers())
p.put(ProducerConfig.ACKS_CONFIG, "all")
p.put(ProducerConfig.RETRIES_CONFIG, "0")
p.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[ByteArraySerializer])
p.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[KafkaAvroSerializer])
p.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, cluster.schemaRegistryUrl)
p
}
import collection.JavaConverters._
IntegrationTestUtils.produceValuesSynchronously(inputTopic, inputValues.asJava, producerConfig)
//
// Step 3: Verify the application's output data.
//
val consumerConfig = {
val p = new Properties()
p.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers())
p.put(ConsumerConfig.GROUP_ID_CONFIG, "specific-avro-scala-integration-test-standard-consumer")
p.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
p.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer])
p.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[KafkaAvroDeserializer])
p.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, cluster.schemaRegistryUrl)
p.put(KafkaAvroDeserializerConfig.SPECIFIC_AVRO_READER_CONFIG, "true")
p
}
val actualValues: java.util.List[WikiFeed] =
IntegrationTestUtils.waitUntilMinValuesRecordsReceived(consumerConfig, outputTopic, inputValues.size)
streams.close()
assert(actualValues === inputValues.asJava)
}
}