|
38 | 38 | import java.util.function.BiFunction; |
39 | 39 | import java.util.stream.Collectors; |
40 | 40 | import java.util.stream.LongStream; |
| 41 | +import org.apache.beam.sdk.Pipeline; |
41 | 42 | import org.apache.beam.sdk.PipelineResult; |
42 | 43 | import org.apache.beam.sdk.coders.ByteArrayCoder; |
43 | 44 | import org.apache.beam.sdk.coders.NullableCoder; |
|
90 | 91 | import org.apache.beam.sdk.values.Row; |
91 | 92 | import org.apache.beam.sdk.values.TypeDescriptors; |
92 | 93 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; |
| 94 | +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; |
93 | 95 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; |
94 | 96 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; |
95 | 97 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; |
96 | 98 | import org.apache.kafka.clients.admin.AdminClient; |
97 | 99 | import org.apache.kafka.clients.admin.NewPartitions; |
98 | 100 | import org.apache.kafka.clients.admin.NewTopic; |
99 | 101 | import org.apache.kafka.clients.producer.ProducerConfig; |
| 102 | +import org.apache.kafka.common.KafkaException; |
100 | 103 | import org.apache.kafka.common.TopicPartition; |
101 | 104 | import org.apache.kafka.common.serialization.ByteArrayDeserializer; |
102 | 105 | import org.apache.kafka.common.serialization.ByteArraySerializer; |
|
114 | 117 | import org.junit.Ignore; |
115 | 118 | import org.junit.Rule; |
116 | 119 | import org.junit.Test; |
| 120 | +import org.junit.rules.ExpectedException; |
117 | 121 | import org.junit.runner.RunWith; |
118 | 122 | import org.junit.runners.JUnit4; |
119 | 123 | import org.slf4j.Logger; |
@@ -171,6 +175,8 @@ public class KafkaIOIT { |
171 | 175 |
|
172 | 176 | @Rule public TestPipeline readPipeline = TestPipeline.create(); |
173 | 177 |
|
| 178 | + @Rule public ExpectedException thrown = ExpectedException.none(); |
| 179 | + |
174 | 180 | private static ExperimentalOptions sdfPipelineOptions; |
175 | 181 |
|
176 | 182 | static { |
@@ -215,6 +221,175 @@ public static void afterClass() { |
215 | 221 | } |
216 | 222 | } |
217 | 223 |
|
| 224 | + @Test |
| 225 | + public void testKafkaIOFailsFastWithInvalidPartitions() throws IOException { |
| 226 | + thrown.expect(Pipeline.PipelineExecutionException.class); |
| 227 | + thrown.expectMessage( |
| 228 | + "Partition 1000 does not exist for topic " |
| 229 | + + options.getKafkaTopic() |
| 230 | + + ". Please check Kafka configuration."); |
| 231 | + |
| 232 | + // Use streaming pipeline to read Kafka records. |
| 233 | + readPipeline.getOptions().as(Options.class).setStreaming(true); |
| 234 | + TopicPartition invalidPartition = new TopicPartition(options.getKafkaTopic(), 1000); |
| 235 | + readPipeline.apply( |
| 236 | + "Read from unbounded Kafka", |
| 237 | + readFromKafka().withTopicPartitions(ImmutableList.of(invalidPartition))); |
| 238 | + |
| 239 | + PipelineResult readResult = readPipeline.run(); |
| 240 | + PipelineResult.State readState = |
| 241 | + readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); |
| 242 | + |
| 243 | + // call asynchronous deleteTopics first since cancelIfTimeouted is blocking. |
| 244 | + tearDownTopic(options.getKafkaTopic()); |
| 245 | + cancelIfTimeouted(readResult, readState); |
| 246 | + } |
| 247 | + |
| 248 | + @Test |
| 249 | + public void testKafkaIOFailsFastWithInvalidPartitionsAndFlagExplicitlySet() throws IOException { |
| 250 | + thrown.expect(Pipeline.PipelineExecutionException.class); |
| 251 | + thrown.expectMessage( |
| 252 | + "Partition 1000 does not exist for topic " |
| 253 | + + options.getKafkaTopic() |
| 254 | + + ". Please check Kafka configuration."); |
| 255 | + |
| 256 | + // Use streaming pipeline to read Kafka records. |
| 257 | + readPipeline.getOptions().as(Options.class).setStreaming(true); |
| 258 | + TopicPartition invalidPartition = new TopicPartition(options.getKafkaTopic(), 1000); |
| 259 | + readPipeline.apply( |
| 260 | + "Read from unbounded Kafka", |
| 261 | + readFromKafka().withTopicPartitions(ImmutableList.of(invalidPartition)).withTopicVerificationLogging(false)); |
| 262 | + |
| 263 | + PipelineResult readResult = readPipeline.run(); |
| 264 | + PipelineResult.State readState = |
| 265 | + readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); |
| 266 | + |
| 267 | + // call asynchronous deleteTopics first since cancelIfTimeouted is blocking. |
| 268 | + tearDownTopic(options.getKafkaTopic()); |
| 269 | + cancelIfTimeouted(readResult, readState); |
| 270 | + } |
| 271 | + |
| 272 | + @Test |
| 273 | + public void testKafkaIODoesNotFailFastWithInvalidPartitionsAndFlagExplicitlyNotSet() |
| 274 | + throws IOException { |
| 275 | + |
| 276 | + // Expect a different error which is thrown at runtime. This is because we disable the failfast |
| 277 | + // error |
| 278 | + // by setting the logging flag to True. |
| 279 | + thrown.expect(java.lang.RuntimeException.class); |
| 280 | + |
| 281 | + // Use streaming pipeline to read Kafka records. |
| 282 | + readPipeline.getOptions().as(Options.class).setStreaming(true); |
| 283 | + TopicPartition invalidPartition = new TopicPartition(options.getKafkaTopic(), 1000); |
| 284 | + readPipeline.apply( |
| 285 | + "Read from unbounded Kafka", |
| 286 | + readFromKafka() |
| 287 | + .withTopicPartitions(ImmutableList.of(invalidPartition)) |
| 288 | + .withTopicVerificationLogging(true)); |
| 289 | + |
| 290 | + PipelineResult readResult = readPipeline.run(); |
| 291 | + PipelineResult.State readState = |
| 292 | + readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); |
| 293 | + |
| 294 | + // call asynchronous deleteTopics first since cancelIfTimeouted is blocking. |
| 295 | + tearDownTopic(options.getKafkaTopic()); |
| 296 | + cancelIfTimeouted(readResult, readState); |
| 297 | + } |
| 298 | + |
| 299 | + @Test |
| 300 | + public void testKafkaIOFailsFastWithInvalidTopics() throws IOException { |
| 301 | + // This test will fail on versions before 2.3.0 due to the non-existence of the |
| 302 | + // allow.auto.create.topics |
| 303 | + // flag. This can be removed when/if support for this older version is dropped. |
| 304 | + String actualVer = AppInfoParser.getVersion(); |
| 305 | + assumeFalse(actualVer.compareTo("2.0.0") >= 0 && actualVer.compareTo("2.3.0") < 0); |
| 306 | + |
| 307 | + thrown.expect(Pipeline.PipelineExecutionException.class); |
| 308 | + thrown.expectMessage( |
| 309 | + "Could not find any partitions info for topic invalid_topic. Please check Kafka configuration" |
| 310 | + + " and make sure that provided topics exist."); |
| 311 | + |
| 312 | + // Use streaming pipeline to read Kafka records. |
| 313 | + sdfReadPipeline.getOptions().as(Options.class).setStreaming(true); |
| 314 | + String invalidTopic = "invalid_topic"; |
| 315 | + sdfReadPipeline.apply( |
| 316 | + "Read from unbounded Kafka", |
| 317 | + KafkaIO.<byte[], byte[]>read() |
| 318 | + .withConsumerConfigUpdates(ImmutableMap.of("allow.auto.create.topics", "false")) |
| 319 | + .withBootstrapServers(options.getKafkaBootstrapServerAddresses()) |
| 320 | + .withTopics(ImmutableList.of(invalidTopic)) |
| 321 | + .withKeyDeserializer(ByteArrayDeserializer.class) |
| 322 | + .withValueDeserializer(ByteArrayDeserializer.class)); |
| 323 | + |
| 324 | + PipelineResult readResult = sdfReadPipeline.run(); |
| 325 | + PipelineResult.State readState = |
| 326 | + readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); |
| 327 | + |
| 328 | + // call asynchronous deleteTopics first since cancelIfTimeouted is blocking. |
| 329 | + tearDownTopic(options.getKafkaTopic()); |
| 330 | + cancelIfTimeouted(readResult, readState); |
| 331 | + } |
| 332 | + |
| 333 | + @Test |
| 334 | + public void testKafkaIOFailsFastWithInvalidTopicsAndDynamicRead() throws IOException { |
| 335 | + // This test will fail on versions before 2.3.0 due to the non-existence of the |
| 336 | + // allow.auto.create.topics |
| 337 | + // flag. This can be removed when/if support for this older version is dropped. |
| 338 | + String actualVer = AppInfoParser.getVersion(); |
| 339 | + assumeFalse(actualVer.compareTo("2.0.0") >= 0 && actualVer.compareTo("2.3.0") < 0); |
| 340 | + |
| 341 | + thrown.expect(Pipeline.PipelineExecutionException.class); |
| 342 | + thrown.expectMessage( |
| 343 | + "Could not find any partitions info for topic invalid_topic. Please check Kafka configuration" |
| 344 | + + " and make sure that provided topics exist."); |
| 345 | + |
| 346 | + // Use streaming pipeline to read Kafka records. |
| 347 | + sdfReadPipeline.getOptions().as(Options.class).setStreaming(true); |
| 348 | + String invalidTopic = "invalid_topic"; |
| 349 | + sdfReadPipeline.apply( |
| 350 | + "Read from unbounded Kafka", |
| 351 | + KafkaIO.<byte[], byte[]>read() |
| 352 | + .withConsumerConfigUpdates(ImmutableMap.of("allow.auto.create.topics", "false")) |
| 353 | + .withBootstrapServers(options.getKafkaBootstrapServerAddresses()) |
| 354 | + .withTopics(ImmutableList.of(invalidTopic)) |
| 355 | + .withDynamicRead(Duration.standardSeconds(5)) |
| 356 | + .withKeyDeserializer(ByteArrayDeserializer.class) |
| 357 | + .withValueDeserializer(ByteArrayDeserializer.class)); |
| 358 | + |
| 359 | + PipelineResult readResult = sdfReadPipeline.run(); |
| 360 | + PipelineResult.State readState = |
| 361 | + readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); |
| 362 | + |
| 363 | + // call asynchronous deleteTopics first since cancelIfTimeouted is blocking. |
| 364 | + tearDownTopic(options.getKafkaTopic()); |
| 365 | + cancelIfTimeouted(readResult, readState); |
| 366 | + } |
| 367 | + |
| 368 | + @Test |
| 369 | + public void testKafkaIODoesNotErrorAtValidationWithBadBootstrapServer() throws IOException { |
| 370 | + // expect an error during execution that the bootstrap server is bad, not during validation |
| 371 | + // steps in |
| 372 | + // KafakUnboundedSource. |
| 373 | + thrown.expect(KafkaException.class); |
| 374 | + // Use streaming pipeline to read Kafka records. |
| 375 | + readPipeline.getOptions().as(Options.class).setStreaming(true); |
| 376 | + TopicPartition invalidPartition = new TopicPartition(options.getKafkaTopic(), 1000); |
| 377 | + readPipeline.apply( |
| 378 | + "Read from unbounded Kafka", |
| 379 | + KafkaIO.readBytes() |
| 380 | + .withBootstrapServers("bootstrap.invalid-name.fake-region.bad-project:invalid-port") |
| 381 | + .withConsumerConfigUpdates(ImmutableMap.of("auto.offset.reset", "earliest")) |
| 382 | + .withTopicPartitions(ImmutableList.of(invalidPartition))); |
| 383 | + |
| 384 | + PipelineResult readResult = readPipeline.run(); |
| 385 | + PipelineResult.State readState = |
| 386 | + readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); |
| 387 | + |
| 388 | + // call asynchronous deleteTopics first since cancelIfTimeouted is blocking. |
| 389 | + tearDownTopic(options.getKafkaTopic()); |
| 390 | + cancelIfTimeouted(readResult, readState); |
| 391 | + } |
| 392 | + |
218 | 393 | @Test |
219 | 394 | public void testKafkaIOReadsAndWritesCorrectlyInStreaming() throws IOException { |
220 | 395 | // Use batch pipeline to write records. |
|
0 commit comments