|
37 | 37 | import java.util.function.BiFunction; |
38 | 38 | import java.util.stream.Collectors; |
39 | 39 | import java.util.stream.LongStream; |
40 | | -import org.apache.beam.sdk.Pipeline; |
41 | 40 | import org.apache.beam.sdk.PipelineResult; |
42 | 41 | import org.apache.beam.sdk.coders.ByteArrayCoder; |
43 | 42 | import org.apache.beam.sdk.coders.NullableCoder; |
|
89 | 88 | import org.apache.beam.sdk.values.Row; |
90 | 89 | import org.apache.beam.sdk.values.TypeDescriptors; |
91 | 90 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; |
92 | | -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; |
93 | 91 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; |
94 | 92 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; |
95 | 93 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; |
|
112 | 110 | import org.junit.Ignore; |
113 | 111 | import org.junit.Rule; |
114 | 112 | import org.junit.Test; |
115 | | -import org.junit.rules.ExpectedException; |
116 | 113 | import org.junit.runner.RunWith; |
117 | 114 | import org.junit.runners.JUnit4; |
118 | 115 | import org.slf4j.Logger; |
@@ -170,8 +167,6 @@ public class KafkaIOIT { |
170 | 167 |
|
171 | 168 | @Rule public TestPipeline readPipeline = TestPipeline.create(); |
172 | 169 |
|
173 | | - @Rule public ExpectedException thrown = ExpectedException.none(); |
174 | | - |
175 | 170 | private static ExperimentalOptions sdfPipelineOptions; |
176 | 171 |
|
177 | 172 | static { |
@@ -216,99 +211,6 @@ public static void afterClass() { |
216 | 211 | } |
217 | 212 | } |
218 | 213 |
|
219 | | - @Test |
220 | | - public void testKafkaIOFailsFastWithInvalidPartitions() throws IOException { |
221 | | - thrown.expect(Pipeline.PipelineExecutionException.class); |
222 | | - thrown.expectMessage( |
223 | | - "Partition 1000 does not exist for topic " |
224 | | - + options.getKafkaTopic() |
225 | | - + ". Please check Kafka configuration."); |
226 | | - |
227 | | - // Use streaming pipeline to read Kafka records. |
228 | | - readPipeline.getOptions().as(Options.class).setStreaming(true); |
229 | | - TopicPartition invalidPartition = new TopicPartition(options.getKafkaTopic(), 1000); |
230 | | - readPipeline.apply( |
231 | | - "Read from unbounded Kafka", |
232 | | - readFromKafka().withTopicPartitions(ImmutableList.of(invalidPartition))); |
233 | | - |
234 | | - PipelineResult readResult = readPipeline.run(); |
235 | | - PipelineResult.State readState = |
236 | | - readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); |
237 | | - |
238 | | - // call asynchronous deleteTopics first since cancelIfTimeouted is blocking. |
239 | | - tearDownTopic(options.getKafkaTopic()); |
240 | | - cancelIfTimeouted(readResult, readState); |
241 | | - } |
242 | | - |
243 | | - @Test |
244 | | - public void testKafkaIOFailsFastWithInvalidTopics() throws IOException { |
245 | | - // This test will fail on versions before 2.3.0 due to the non-existence of the |
246 | | - // allow.auto.create.topics |
247 | | - // flag. This can be removed when/if support for this older version is dropped. |
248 | | - String actualVer = AppInfoParser.getVersion(); |
249 | | - assumeFalse(actualVer.compareTo("2.0.0") >= 0 && actualVer.compareTo("2.3.0") < 0); |
250 | | - |
251 | | - thrown.expect(Pipeline.PipelineExecutionException.class); |
252 | | - thrown.expectMessage( |
253 | | - "Could not find any partitions info for topic invalid_topic. Please check Kafka configuration" |
254 | | - + " and make sure that provided topics exist."); |
255 | | - |
256 | | - // Use streaming pipeline to read Kafka records. |
257 | | - sdfReadPipeline.getOptions().as(Options.class).setStreaming(true); |
258 | | - String invalidTopic = "invalid_topic"; |
259 | | - sdfReadPipeline.apply( |
260 | | - "Read from unbounded Kafka", |
261 | | - KafkaIO.<byte[], byte[]>read() |
262 | | - .withConsumerConfigUpdates(ImmutableMap.of("allow.auto.create.topics", "false")) |
263 | | - .withBootstrapServers(options.getKafkaBootstrapServerAddresses()) |
264 | | - .withTopics(ImmutableList.of(invalidTopic)) |
265 | | - .withKeyDeserializer(ByteArrayDeserializer.class) |
266 | | - .withValueDeserializer(ByteArrayDeserializer.class)); |
267 | | - |
268 | | - PipelineResult readResult = sdfReadPipeline.run(); |
269 | | - PipelineResult.State readState = |
270 | | - readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); |
271 | | - |
272 | | - // call asynchronous deleteTopics first since cancelIfTimeouted is blocking. |
273 | | - tearDownTopic(options.getKafkaTopic()); |
274 | | - cancelIfTimeouted(readResult, readState); |
275 | | - } |
276 | | - |
277 | | - @Test |
278 | | - public void testKafkaIOFailsFastWithInvalidTopicsAndDynamicRead() throws IOException { |
279 | | - // This test will fail on versions before 2.3.0 due to the non-existence of the |
280 | | - // allow.auto.create.topics |
281 | | - // flag. This can be removed when/if support for this older version is dropped. |
282 | | - String actualVer = AppInfoParser.getVersion(); |
283 | | - assumeFalse(actualVer.compareTo("2.0.0") >= 0 && actualVer.compareTo("2.3.0") < 0); |
284 | | - |
285 | | - thrown.expect(Pipeline.PipelineExecutionException.class); |
286 | | - thrown.expectMessage( |
287 | | - "Could not find any partitions info for topic invalid_topic. Please check Kafka configuration" |
288 | | - + " and make sure that provided topics exist."); |
289 | | - |
290 | | - // Use streaming pipeline to read Kafka records. |
291 | | - sdfReadPipeline.getOptions().as(Options.class).setStreaming(true); |
292 | | - String invalidTopic = "invalid_topic"; |
293 | | - sdfReadPipeline.apply( |
294 | | - "Read from unbounded Kafka", |
295 | | - KafkaIO.<byte[], byte[]>read() |
296 | | - .withConsumerConfigUpdates(ImmutableMap.of("allow.auto.create.topics", "false")) |
297 | | - .withBootstrapServers(options.getKafkaBootstrapServerAddresses()) |
298 | | - .withTopics(ImmutableList.of(invalidTopic)) |
299 | | - .withDynamicRead(Duration.standardSeconds(5)) |
300 | | - .withKeyDeserializer(ByteArrayDeserializer.class) |
301 | | - .withValueDeserializer(ByteArrayDeserializer.class)); |
302 | | - |
303 | | - PipelineResult readResult = sdfReadPipeline.run(); |
304 | | - PipelineResult.State readState = |
305 | | - readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); |
306 | | - |
307 | | - // call asynchronous deleteTopics first since cancelIfTimeouted is blocking. |
308 | | - tearDownTopic(options.getKafkaTopic()); |
309 | | - cancelIfTimeouted(readResult, readState); |
310 | | - } |
311 | | - |
312 | 214 | @Test |
313 | 215 | public void testKafkaIOReadsAndWritesCorrectlyInStreaming() throws IOException { |
314 | 216 | // Use batch pipeline to write records. |
|
0 commit comments