Skip to content

Commit f93ad4f

Browse files
HyukjinKwonsrowen
authored andcommitted
[SPARK-16776][STREAMING] Replace deprecated API in KafkaTestUtils for 0.10.0.
## What changes were proposed in this pull request? This PR replaces the old Kafka API to 0.10.0 ones in `KafkaTestUtils`. The change include: - `Producer` to `KafkaProducer` - Change configurations to equalvant ones. (I referred [here](http://kafka.apache.org/documentation.html#producerconfigs) for 0.10.0 and [here](http://kafka.apache.org/082/documentation.html#producerconfigs ) for old, 0.8.2). This PR will remove the build warning as below: ```scala [WARNING] .../spark/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala:71: class Producer in package producer is deprecated: This class has been deprecated and will be removed in a future release. Please use org.apache.kafka.clients.producer.KafkaProducer instead. [WARNING] private var producer: Producer[String, String] = _ [WARNING] ^ [WARNING] .../spark/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala:181: class Producer in package producer is deprecated: This class has been deprecated and will be removed in a future release. Please use org.apache.kafka.clients.producer.KafkaProducer instead. [WARNING] producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) [WARNING] ^ [WARNING] .../spark/streaming/kafka010/KafkaTestUtils.scala:181: class ProducerConfig in package producer is deprecated: This class has been deprecated and will be removed in a future release. Please use org.apache.kafka.clients.producer.ProducerConfig instead. [WARNING] producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) [WARNING] ^ [WARNING] .../spark/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala:182: class KeyedMessage in package producer is deprecated: This class has been deprecated and will be removed in a future release. Please use org.apache.kafka.clients.producer.ProducerRecord instead. [WARNING] producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) [WARNING] ^ [WARNING] four warnings found [WARNING] warning: [options] bootstrap class path not set in conjunction with -source 1.7 [WARNING] 1 warning ``` ## How was this patch tested? Existing tests that use `KafkaTestUtils` should cover this. Author: hyukjinkwon <[email protected]> Closes #14416 from HyukjinKwon/SPARK-16776.
1 parent 1e9b59b commit f93ad4f

File tree

1 file changed

+12
-8
lines changed

1 file changed

+12
-8
lines changed

external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala

Lines changed: 12 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -30,10 +30,10 @@ import scala.util.control.NonFatal
3030

3131
import kafka.admin.AdminUtils
3232
import kafka.api.Request
33-
import kafka.producer.{KeyedMessage, Producer, ProducerConfig}
34-
import kafka.serializer.StringEncoder
3533
import kafka.server.{KafkaConfig, KafkaServer}
3634
import kafka.utils.ZkUtils
35+
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
36+
import org.apache.kafka.common.serialization.StringSerializer
3737
import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer}
3838

3939
import org.apache.spark.SparkConf
@@ -68,7 +68,7 @@ private[kafka010] class KafkaTestUtils extends Logging {
6868
private var server: KafkaServer = _
6969

7070
// Kafka producer
71-
private var producer: Producer[String, String] = _
71+
private var producer: KafkaProducer[String, String] = _
7272

7373
// Flag to test whether the system is correctly started
7474
private var zkReady = false
@@ -178,8 +178,10 @@ private[kafka010] class KafkaTestUtils extends Logging {
178178

179179
/** Send the array of messages to the Kafka broker */
180180
def sendMessages(topic: String, messages: Array[String]): Unit = {
181-
producer = new Producer[String, String](new ProducerConfig(producerConfiguration))
182-
producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*)
181+
producer = new KafkaProducer[String, String](producerConfiguration)
182+
messages.foreach { message =>
183+
producer.send(new ProducerRecord[String, String](topic, message))
184+
}
183185
producer.close()
184186
producer = null
185187
}
@@ -198,10 +200,12 @@ private[kafka010] class KafkaTestUtils extends Logging {
198200

199201
private def producerConfiguration: Properties = {
200202
val props = new Properties()
201-
props.put("metadata.broker.list", brokerAddress)
202-
props.put("serializer.class", classOf[StringEncoder].getName)
203+
props.put("bootstrap.servers", brokerAddress)
204+
props.put("value.serializer", classOf[StringSerializer].getName)
205+
// Key serializer is required.
206+
props.put("key.serializer", classOf[StringSerializer].getName)
203207
// wait for all in-sync replicas to ack sends
204-
props.put("request.required.acks", "-1")
208+
props.put("acks", "all")
205209
props
206210
}
207211

0 commit comments

Comments
 (0)