Browse Source

MINOR: Suppress ProducerConfig warning in MirrorMaker

Though MirrorMaker uses the `producer.type` value of the
producer properties, ProducerConfig show the warning:
`The configuration 'producer.type' was supplied but
isn't a known config.`

Author: Shun Takebayashi <shun@takebayashi.asia>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #2676 from takebayashi/suppress-mirrormaker-warning
pull/2732/merge
Shun Takebayashi 8 years ago committed by Ismael Juma
parent
commit
ca2979f847
  1. 8
      core/src/main/scala/kafka/tools/MirrorMaker.scala
  2. 3
      core/src/test/scala/integration/kafka/tools/MirrorMakerIntegrationTest.scala

8
core/src/main/scala/kafka/tools/MirrorMaker.scala

@ -225,6 +225,8 @@ object MirrorMaker extends Logging with KafkaMetricsGroup {
// create producer // create producer
val producerProps = Utils.loadProps(options.valueOf(producerConfigOpt)) val producerProps = Utils.loadProps(options.valueOf(producerConfigOpt))
val sync = producerProps.getProperty("producer.type", "async").equals("sync")
producerProps.remove("producer.type")
// Defaults to no data loss settings. // Defaults to no data loss settings.
maybeSetDefaultProperty(producerProps, ProducerConfig.RETRIES_CONFIG, Int.MaxValue.toString) maybeSetDefaultProperty(producerProps, ProducerConfig.RETRIES_CONFIG, Int.MaxValue.toString)
maybeSetDefaultProperty(producerProps, ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.MaxValue.toString) maybeSetDefaultProperty(producerProps, ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.MaxValue.toString)
@ -233,7 +235,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup {
// Always set producer key and value serializer to ByteArraySerializer. // Always set producer key and value serializer to ByteArraySerializer.
producerProps.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") producerProps.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
producerProps.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") producerProps.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
producer = new MirrorMakerProducer(producerProps) producer = new MirrorMakerProducer(sync, producerProps)
// Create consumers // Create consumers
val mirrorMakerConsumers = if (useOldConsumer) { val mirrorMakerConsumers = if (useOldConsumer) {
@ -696,9 +698,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup {
} }
} }
private[tools] class MirrorMakerProducer(val producerProps: Properties) { private[tools] class MirrorMakerProducer(val sync: Boolean, val producerProps: Properties) {
val sync = producerProps.getProperty("producer.type", "async").equals("sync")
val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerProps) val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerProps)

3
core/src/test/scala/integration/kafka/tools/MirrorMakerIntegrationTest.scala

@ -41,10 +41,9 @@ class MirrorMakerIntegrationTest extends KafkaServerTestHarness {
val producerProps = new Properties val producerProps = new Properties
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
producerProps.put("producer.type", "sync")
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[ByteArraySerializer]) producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[ByteArraySerializer])
producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[ByteArraySerializer]) producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[ByteArraySerializer])
val producer = new MirrorMakerProducer(producerProps) val producer = new MirrorMakerProducer(true, producerProps)
MirrorMaker.producer = producer MirrorMaker.producer = producer
MirrorMaker.producer.send(new ProducerRecord(topic, msg.getBytes())) MirrorMaker.producer.send(new ProducerRecord(topic, msg.getBytes()))
MirrorMaker.producer.close() MirrorMaker.producer.close()

Loading…
Cancel
Save