Skip to content

Commit 5ca14c1

Browse files
authored
fix side-input existence deadline (#34046)
1 parent cf3c39c commit 5ca14c1

File tree

9 files changed

+32
-24
lines changed

9 files changed

+32
-24
lines changed

runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -193,7 +193,8 @@ public void process(ProcessContext c) {}
193193

194194
// And digging to check whether the window is ready
195195
when(mockEvaluationContext.createSideInputReader(anyList())).thenReturn(mockSideInputReader);
196-
when(mockSideInputReader.isReady(ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(false);
196+
when(mockSideInputReader.isReady(ArgumentMatchers.any(), ArgumentMatchers.any()))
197+
.thenReturn(false);
197198

198199
IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(9));
199200

runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -23,8 +23,8 @@
2323
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
2424
import static org.hamcrest.Matchers.lessThanOrEqualTo;
2525
import static org.junit.Assert.assertEquals;
26-
import static org.mockito.ArgumentMatchers.anyString;
2726
import static org.mockito.ArgumentMatchers.anyLong;
27+
import static org.mockito.ArgumentMatchers.anyString;
2828
import static org.mockito.ArgumentMatchers.eq;
2929
import static org.mockito.Mockito.any;
3030
import static org.mockito.Mockito.mock;

runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/TestDataflowRunnerTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,8 +24,8 @@
2424
import static org.junit.Assert.assertNull;
2525
import static org.junit.Assert.assertSame;
2626
import static org.junit.Assert.fail;
27-
import static org.mockito.ArgumentMatchers.anyString;
2827
import static org.mockito.ArgumentMatchers.any;
28+
import static org.mockito.ArgumentMatchers.anyString;
2929
import static org.mockito.Mockito.doReturn;
3030
import static org.mockito.Mockito.never;
3131
import static org.mockito.Mockito.spy;

runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcher.java

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@
3030
import java.util.function.Function;
3131
import javax.annotation.concurrent.NotThreadSafe;
3232
import org.apache.beam.runners.core.InMemoryMultimapSideInputView;
33+
import org.apache.beam.runners.core.LateDataUtils;
3334
import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils;
3435
import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
3536
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalData;
@@ -48,6 +49,8 @@
4849
import org.apache.beam.sdk.values.WindowingStrategy;
4950
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier;
5051
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
52+
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Ordering;
53+
import org.joda.time.Instant;
5154
import org.slf4j.Logger;
5255
import org.slf4j.LoggerFactory;
5356

@@ -207,6 +210,11 @@ private <T, SideWindowT extends BoundedWindow> GlobalData fetchGlobalDataFromWin
207210
ByteStringOutputStream windowStream = new ByteStringOutputStream();
208211
windowCoder.encode(sideWindow, windowStream);
209212

213+
Instant firingDeadline =
214+
Ordering.natural()
215+
.min(
216+
sideWindowStrategy.getTrigger().getWatermarkThatGuaranteesFiring(sideWindow),
217+
LateDataUtils.garbageCollectionTime(sideWindow, sideWindowStrategy));
210218
GlobalDataRequest request =
211219
GlobalDataRequest.newBuilder()
212220
.setDataId(
@@ -216,8 +224,7 @@ private <T, SideWindowT extends BoundedWindow> GlobalData fetchGlobalDataFromWin
216224
.build())
217225
.setStateFamily(stateFamily)
218226
.setExistenceWatermarkDeadline(
219-
WindmillTimeUtils.harnessToWindmillTimestamp(
220-
sideWindowStrategy.getTrigger().getWatermarkThatGuaranteesFiring(sideWindow)))
227+
WindmillTimeUtils.harnessToWindmillTimestamp(firingDeadline))
221228
.build();
222229

223230
try (Closeable ignored = scopedReadStateSupplier.get()) {

runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFnsTest.java

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -248,7 +248,8 @@ public void testPartialGroupByKeyWithCombinerAndSideInputs() throws Exception {
248248
ImmutableList.of(
249249
WindowedValue.valueInGlobalWindow(KV.of("hi", 4)),
250250
WindowedValue.valueInGlobalWindow(KV.of("there", 5))));
251-
when(mockSideInputFetcher.storeIfBlocked(ArgumentMatchers.<WindowedValue<KV<String, Integer>>>any()))
251+
when(mockSideInputFetcher.storeIfBlocked(
252+
ArgumentMatchers.<WindowedValue<KV<String, Integer>>>any()))
252253
.thenReturn(false, false, false, true);
253254

254255
pgbkParDoFn.startBundle(receiver);
@@ -360,7 +361,8 @@ public void testCreateWithCombinerAndStreamingSideInputs() throws Exception {
360361

361362
when(mockSideInputReader.isEmpty()).thenReturn(false);
362363
when(mockStreamingStepContext.stateInternals()).thenReturn((StateInternals) mockStateInternals);
363-
when(mockStateInternals.state(ArgumentMatchers.<StateNamespace>any(), ArgumentMatchers.<StateTag>any()))
364+
when(mockStateInternals.state(
365+
ArgumentMatchers.<StateNamespace>any(), ArgumentMatchers.<StateTag>any()))
364366
.thenReturn(mockState);
365367
when(mockState.read()).thenReturn(Maps.newHashMap());
366368

@@ -394,7 +396,8 @@ public void testCoderSizeEstimationWithNonLazyObserver() throws Exception {
394396
return null;
395397
})
396398
.when(mockCoder)
397-
.registerByteSizeObserver(ArgumentMatchers.eq("apple"), ArgumentMatchers.<ElementByteSizeObserver>any());
399+
.registerByteSizeObserver(
400+
ArgumentMatchers.eq("apple"), ArgumentMatchers.<ElementByteSizeObserver>any());
398401
CoderSizeEstimator<String> estimator = new CoderSizeEstimator(mockCoder);
399402
assertEquals(5, estimator.estimateSize("apple"));
400403
}
@@ -410,7 +413,8 @@ public void testCoderSizeEstimationWithLazyObserver() throws Exception {
410413
return null;
411414
})
412415
.when(mockCoder)
413-
.registerByteSizeObserver(ArgumentMatchers.eq("apple"), ArgumentMatchers.<ElementByteSizeObserver>any());
416+
.registerByteSizeObserver(
417+
ArgumentMatchers.eq("apple"), ArgumentMatchers.<ElementByteSizeObserver>any());
414418

415419
// Encode the input to the output stream
416420
doAnswer(

runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -33,9 +33,9 @@
3333
import static org.junit.Assert.assertSame;
3434
import static org.junit.Assert.assertTrue;
3535
import static org.junit.Assert.fail;
36+
import static org.mockito.ArgumentMatchers.any;
3637
import static org.mockito.ArgumentMatchers.eq;
3738
import static org.mockito.ArgumentMatchers.nullable;
38-
import static org.mockito.ArgumentMatchers.any;
3939
import static org.mockito.Mockito.atLeast;
4040
import static org.mockito.Mockito.atLeastOnce;
4141
import static org.mockito.Mockito.mock;

runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactory.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -125,7 +125,11 @@ public RemoteEnvironment createEnvironment(Environment environment, String worke
125125
String containerId = null;
126126
InstructionRequestHandler instructionHandler = null;
127127
try {
128-
LOG.info("Running Docker command: image={}, opts={}, args={}", containerImage, dockerOptsBuilder.build(), argsBuilder.build());
128+
LOG.info(
129+
"Running Docker command: image={}, opts={}, args={}",
130+
containerImage,
131+
dockerOptsBuilder.build(),
132+
argsBuilder.build());
129133
containerId = docker.runImage(containerImage, dockerOptsBuilder.build(), argsBuilder.build());
130134
LOG.debug("Created Docker Container with Container ID {}", containerId);
131135
// Wait on a client from the gRPC server.

sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java

Lines changed: 4 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -587,19 +587,11 @@ public class BigQueryIO {
587587
private static final String DATASET_REGEXP = "[-\\w.]{1,1024}";
588588

589589
/**
590-
* Regular expression that matches BigQuery Table IDs.
591-
* Supports Unicode characters in categories:
592-
* - L (letter)
593-
* - M (mark)
594-
* - N (number)
595-
* As well as:
596-
* - Underscore (_)
597-
* - Dash (-)
598-
* - Dollar sign ($)
599-
* - At sign (@)
600-
* - Space
590+
* Regular expression that matches BigQuery Table IDs. Supports Unicode characters in categories:
591+
* - L (letter) - M (mark) - N (number) As well as: - Underscore (_) - Dash (-) - Dollar sign ($)
592+
* - At sign (@) - Space
601593
*
602-
* The pattern requires 1-1024 characters matching these categories.
594+
* <p>The pattern requires 1-1024 characters matching these categories.
603595
*/
604596
private static final String TABLE_REGEXP = "[-_\\p{L}\\p{N}\\p{M}$@ ]{1,1024}";
605597

sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,8 +29,8 @@
2929
import static org.junit.Assert.assertSame;
3030
import static org.junit.Assert.assertThrows;
3131
import static org.junit.Assert.assertTrue;
32-
import static org.mockito.ArgumentMatchers.anyString;
3332
import static org.mockito.ArgumentMatchers.any;
33+
import static org.mockito.ArgumentMatchers.anyString;
3434
import static org.mockito.Mockito.mock;
3535
import static org.mockito.Mockito.times;
3636
import static org.mockito.Mockito.verify;

0 commit comments

Comments
 (0)