MINOR: Catch NoRecordsException in testCommaSeparatedRegex() test (#5944)

This test sometimes fails with

```
kafka.tools.MirrorMaker$NoRecordsException
	at kafka.tools.MirrorMaker$ConsumerWrapper.receive(MirrorMaker.scala:483)
	at kafka.tools.MirrorMakerIntegrationTest$$anonfun$testCommaSeparatedRegex$1.apply$mcZ$sp(MirrorMakerIntegrationTest.scala:92)
	at kafka.utils.TestUtils$.waitUntilTrue(TestUtils.scala:738)
```

The test should catch `NoRecordsException` instead of `TimeoutException`.

Reviewers: Ismael Juma <ismael@juma.me.uk>
This commit is contained in:
Stanislav Kozlovski 2018-12-08 17:43:43 +00:00 committed by Ismael Juma
parent ac35ef6242
commit edfa681736
2 changed files with 5 additions and 4 deletions

View File

@ -424,7 +424,8 @@ object MirrorMaker extends Logging with KafkaMetricsGroup {
}
}
private class NoRecordsException extends RuntimeException
// package-private for tests
private[tools] class NoRecordsException extends RuntimeException
class MirrorMakerOptions(args: Array[String]) extends CommandDefaultOptions(args) {

View File

@ -20,7 +20,7 @@ import java.util.Properties
import kafka.integration.KafkaServerTestHarness
import kafka.server.KafkaConfig
import kafka.tools.MirrorMaker.{ConsumerWrapper, MirrorMakerProducer}
import kafka.tools.MirrorMaker.{ConsumerWrapper, MirrorMakerProducer, NoRecordsException}
import kafka.utils.TestUtils
import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer}
import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
@ -92,8 +92,8 @@ class MirrorMakerIntegrationTest extends KafkaServerTestHarness {
val data = mirrorMakerConsumer.receive()
data.topic == topic && new String(data.value) == msg
} catch {
// this exception is thrown if no record is returned within a short timeout, so safe to ignore
case _: TimeoutException => false
// these exceptions are thrown if no records are returned within the timeout, so safe to ignore
case _: NoRecordsException => false
}
}, "MirrorMaker consumer should read the expected message from the expected topic within the timeout")
} finally consumer.close()