|
97 | 97 | import org.apache.kafka.clients.admin.NewPartitions; |
98 | 98 | import org.apache.kafka.clients.admin.NewTopic; |
99 | 99 | import org.apache.kafka.clients.producer.ProducerConfig; |
| 100 | +import org.apache.kafka.common.KafkaException; |
100 | 101 | import org.apache.kafka.common.TopicPartition; |
101 | 102 | import org.apache.kafka.common.serialization.ByteArrayDeserializer; |
102 | 103 | import org.apache.kafka.common.serialization.ByteArraySerializer; |
@@ -274,6 +275,31 @@ public void testKafkaIOFailsFastWithInvalidTopics() throws IOException { |
274 | 275 | cancelIfTimeouted(readResult, readState); |
275 | 276 | } |
276 | 277 |
|
| 278 | + @Test |
| 279 | + public void testKafkaIODoesNotErrorAtValidationWithBadBootstrapServer() throws IOException { |
| 280 | + // expect an error during execution that the bootstrap server is bad, not during validation steps in |
| 281 | + // KafakUnboundedSource. |
| 282 | + thrown.expect(KafkaException.class); |
| 283 | + // Use streaming pipeline to read Kafka records. |
| 284 | + readPipeline.getOptions().as(Options.class).setStreaming(true); |
| 285 | + TopicPartition invalidPartition = new TopicPartition(options.getKafkaTopic(), 1000); |
| 286 | + readPipeline.apply( |
| 287 | + "Read from unbounded Kafka", |
| 288 | + KafkaIO.readBytes() |
| 289 | + .withBootstrapServers( |
| 290 | + "bootstrap.invalid-name.fake-region.bad-project:invalid-port") |
| 291 | + .withConsumerConfigUpdates(ImmutableMap.of("auto.offset.reset", "earliest")) |
| 292 | + .withTopicPartitions(ImmutableList.of(invalidPartition))); |
| 293 | + |
| 294 | + PipelineResult readResult = readPipeline.run(); |
| 295 | + PipelineResult.State readState = |
| 296 | + readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); |
| 297 | + |
| 298 | + // call asynchronous deleteTopics first since cancelIfTimeouted is blocking. |
| 299 | + tearDownTopic(options.getKafkaTopic()); |
| 300 | + cancelIfTimeouted(readResult, readState); |
| 301 | + } |
| 302 | + |
277 | 303 | @Test |
278 | 304 | public void testKafkaIOFailsFastWithInvalidTopicsAndDynamicRead() throws IOException { |
279 | 305 | // This test will fail on versions before 2.3.0 due to the non-existence of the |
|
0 commit comments