|
17 | 17 | */ |
18 | 18 | package org.apache.beam.runners.dataflow; |
19 | 19 |
|
| 20 | +import static org.apache.beam.runners.dataflow.util.Structs.getBoolean; |
20 | 21 | import static org.apache.beam.runners.dataflow.util.Structs.getString; |
21 | 22 | import static org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray; |
22 | 23 | import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; |
|
89 | 90 | import org.apache.beam.sdk.io.range.OffsetRange; |
90 | 91 | import org.apache.beam.sdk.options.PipelineOptions; |
91 | 92 | import org.apache.beam.sdk.options.PipelineOptionsFactory; |
| 93 | +import org.apache.beam.sdk.options.StreamingOptions; |
92 | 94 | import org.apache.beam.sdk.options.ValueProvider; |
93 | 95 | import org.apache.beam.sdk.state.StateSpec; |
94 | 96 | import org.apache.beam.sdk.state.StateSpecs; |
95 | 97 | import org.apache.beam.sdk.state.ValueState; |
96 | 98 | import org.apache.beam.sdk.testing.PAssert; |
| 99 | +import org.apache.beam.sdk.transforms.Combine; |
97 | 100 | import org.apache.beam.sdk.transforms.Count; |
98 | 101 | import org.apache.beam.sdk.transforms.Create; |
99 | 102 | import org.apache.beam.sdk.transforms.DoFn; |
|
110 | 113 | import org.apache.beam.sdk.transforms.resourcehints.ResourceHints; |
111 | 114 | import org.apache.beam.sdk.transforms.resourcehints.ResourceHintsOptions; |
112 | 115 | import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; |
| 116 | +import org.apache.beam.sdk.transforms.windowing.AfterAll; |
| 117 | +import org.apache.beam.sdk.transforms.windowing.AfterFirst; |
| 118 | +import org.apache.beam.sdk.transforms.windowing.AfterPane; |
| 119 | +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; |
| 120 | +import org.apache.beam.sdk.transforms.windowing.AfterSynchronizedProcessingTime; |
| 121 | +import org.apache.beam.sdk.transforms.windowing.AfterWatermark; |
| 122 | +import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; |
113 | 123 | import org.apache.beam.sdk.transforms.windowing.FixedWindows; |
| 124 | +import org.apache.beam.sdk.transforms.windowing.Never; |
| 125 | +import org.apache.beam.sdk.transforms.windowing.Repeatedly; |
| 126 | +import org.apache.beam.sdk.transforms.windowing.Trigger; |
114 | 127 | import org.apache.beam.sdk.transforms.windowing.Window; |
115 | 128 | import org.apache.beam.sdk.transforms.windowing.WindowFn; |
116 | 129 | import org.apache.beam.sdk.util.DoFnInfo; |
@@ -235,6 +248,131 @@ private static DataflowPipelineOptions buildPipelineOptions() throws IOException |
235 | 248 | return options; |
236 | 249 | } |
237 | 250 |
|
| 251 | + private void testTriggerCombinerLiftingDisabled(Trigger trigger) throws Exception { |
| 252 | + DataflowPipelineOptions options = buildPipelineOptions(); |
| 253 | + options.setRunner(DataflowRunner.class); |
| 254 | + options.as(StreamingOptions.class).setStreaming(true); |
| 255 | + Pipeline p = Pipeline.create(options); |
| 256 | + |
| 257 | + p.traverseTopologically(new RecordingPipelineVisitor()); |
| 258 | + SdkComponents sdkComponents = createSdkComponents(options); |
| 259 | + |
| 260 | + p.apply("create", Create.of(1, 2, 3, 4).withCoder(VarIntCoder.of())) |
| 261 | + .setIsBoundedInternal(IsBounded.UNBOUNDED) |
| 262 | + .apply("window", Window.<Integer>configure().triggering(trigger).discardingFiredPanes()) |
| 263 | + .apply("count", Combine.globally(Count.<Integer>combineFn()).withoutDefaults()); |
| 264 | + |
| 265 | + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p, sdkComponents, true); |
| 266 | + DataflowPipelineOptions translatorOptions = |
| 267 | + PipelineOptionsFactory.as(DataflowPipelineOptions.class); |
| 268 | + translatorOptions.setStreaming(true); |
| 269 | + DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(translatorOptions); |
| 270 | + |
| 271 | + JobSpecification jobSpecification = |
| 272 | + t.translate( |
| 273 | + p, |
| 274 | + pipelineProto, |
| 275 | + sdkComponents, |
| 276 | + DataflowRunner.fromOptions(options), |
| 277 | + Collections.emptyList()); |
| 278 | + |
| 279 | + boolean foundDisable = false; |
| 280 | + for (Step step : jobSpecification.getJob().getSteps()) { |
| 281 | + if (getBoolean(step.getProperties(), PropertyNames.DISALLOW_COMBINER_LIFTING, false)) { |
| 282 | + foundDisable = true; |
| 283 | + } |
| 284 | + } |
| 285 | + assertTrue(foundDisable); |
| 286 | + } |
| 287 | + |
| 288 | + @Test |
| 289 | + public void testRepeatedCountTriggerDisablesCombinerLifting() throws IOException, Exception { |
| 290 | + testTriggerCombinerLiftingDisabled(Repeatedly.forever((AfterPane.elementCountAtLeast(1)))); |
| 291 | + } |
| 292 | + |
| 293 | + @Test |
| 294 | + public void testEarlyCountTriggerDisablesCombinerLifting() throws IOException, Exception { |
| 295 | + testTriggerCombinerLiftingDisabled( |
| 296 | + AfterWatermark.pastEndOfWindow().withEarlyFirings(AfterPane.elementCountAtLeast(1))); |
| 297 | + } |
| 298 | + |
| 299 | + @Test |
| 300 | + public void testAfterFirstCountTriggerDisablesCombinerLifting() throws IOException, Exception { |
| 301 | + testTriggerCombinerLiftingDisabled( |
| 302 | + Repeatedly.forever(AfterFirst.of(Never.ever(), AfterPane.elementCountAtLeast(1)))); |
| 303 | + } |
| 304 | + |
| 305 | + @Test |
| 306 | + public void testAfterAllCountTriggerDisablesCombinerLifting() throws IOException, Exception { |
| 307 | + testTriggerCombinerLiftingDisabled( |
| 308 | + Repeatedly.forever(AfterAll.of(Never.ever(), AfterPane.elementCountAtLeast(1)))); |
| 309 | + } |
| 310 | + |
| 311 | + @Test |
| 312 | + public void testCombinerLiftingEnabled() throws IOException, Exception { |
| 313 | + DataflowPipelineOptions options = buildPipelineOptions(); |
| 314 | + options.setRunner(DataflowRunner.class); |
| 315 | + options.as(StreamingOptions.class).setStreaming(true); |
| 316 | + Pipeline p = Pipeline.create(options); |
| 317 | + |
| 318 | + p.traverseTopologically(new RecordingPipelineVisitor()); |
| 319 | + SdkComponents sdkComponents = createSdkComponents(options); |
| 320 | + |
| 321 | + PCollection<Integer> input = |
| 322 | + p.apply("create", Create.of(1, 2, 3, 4).withCoder(VarIntCoder.of())); |
| 323 | + |
| 324 | + input |
| 325 | + .setIsBoundedInternal(IsBounded.UNBOUNDED) |
| 326 | + .apply( |
| 327 | + "window1", |
| 328 | + Window.<Integer>into(FixedWindows.of(Duration.millis(1))) |
| 329 | + .triggering(DefaultTrigger.of()) |
| 330 | + .discardingFiredPanes()) |
| 331 | + .apply("count", Combine.globally(Count.<Integer>combineFn()).withoutDefaults()); |
| 332 | + |
| 333 | + input |
| 334 | + .apply( |
| 335 | + "window2", |
| 336 | + Window.<Integer>configure() |
| 337 | + .triggering(AfterWatermark.pastEndOfWindow()) |
| 338 | + .discardingFiredPanes()) |
| 339 | + .apply("count2", Combine.globally(Count.<Integer>combineFn()).withoutDefaults()); |
| 340 | + |
| 341 | + input |
| 342 | + .apply( |
| 343 | + "window3", |
| 344 | + Window.<Integer>configure() |
| 345 | + .triggering( |
| 346 | + AfterWatermark.pastEndOfWindow() |
| 347 | + .withEarlyFirings( |
| 348 | + AfterProcessingTime.pastFirstElementInPane().plusDelayOf(Duration.ZERO)) |
| 349 | + .withLateFirings(AfterSynchronizedProcessingTime.ofFirstElement())) |
| 350 | + .discardingFiredPanes()) |
| 351 | + .apply("count3", Combine.globally(Count.<Integer>combineFn()).withoutDefaults()); |
| 352 | + |
| 353 | + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p, sdkComponents, true); |
| 354 | + DataflowPipelineOptions translatorOptions = |
| 355 | + PipelineOptionsFactory.as(DataflowPipelineOptions.class); |
| 356 | + translatorOptions.setStreaming(true); |
| 357 | + DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(translatorOptions); |
| 358 | + |
| 359 | + JobSpecification jobSpecification = |
| 360 | + t.translate( |
| 361 | + p, |
| 362 | + pipelineProto, |
| 363 | + sdkComponents, |
| 364 | + DataflowRunner.fromOptions(options), |
| 365 | + Collections.emptyList()); |
| 366 | + |
| 367 | + boolean foundDisable = false; |
| 368 | + for (Step step : jobSpecification.getJob().getSteps()) { |
| 369 | + if (getBoolean(step.getProperties(), PropertyNames.DISALLOW_COMBINER_LIFTING, false)) { |
| 370 | + foundDisable = true; |
| 371 | + } |
| 372 | + } |
| 373 | + assertFalse(foundDisable); |
| 374 | + } |
| 375 | + |
238 | 376 | // Test that the transform names for Storage Write API for streaming pipelines are what we expect |
239 | 377 | // them to be. This is required since the Windmill backend expects the step to contain that name. |
240 | 378 | // For a more stable solution, we should use URN, but that is not currently used in the legacy |
|
0 commit comments