reuvenlax commented on code in PR #38363: URL: https://github.com/apache/beam/pull/38363#discussion_r3250248417
########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputProcessorTest.java: ########## @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.api.client.util.Lists; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; +import org.apache.beam.runners.core.TimerInternals.TimerData; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +/** Unit tests for {@link StreamingSideInputProcessor}. */ +@RunWith(JUnit4.class) +public class StreamingSideInputProcessorTest { + + @Mock private StreamingSideInputFetcher<String, IntervalWindow> mockFetcher; + private StreamingSideInputProcessor<String, IntervalWindow> processor; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + processor = new StreamingSideInputProcessor<>(mockFetcher); + } + + @Test + public void testTryUnblockElementsNoReadyWindows() { + // Given + doNothing().when(mockFetcher).prefetchBlockedMap(); + when(mockFetcher.getReadyWindows()).thenReturn(Collections.emptySet()); + + // When + Iterator<WindowedValue<String>> unblocked = processor.tryUnblockElements(); + + // Then + assertFalse(unblocked.hasNext()); + verify(mockFetcher).prefetchBlockedMap(); + verify(mockFetcher).getReadyWindows(); + } + + @Test + public void testTryUnblockElementsWithReadyWindows() { + // Given + IntervalWindow window1 = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow window2 = new IntervalWindow(new Instant(10), new Instant(20)); + Set<IntervalWindow> readyWindows = new HashSet<>(Arrays.asList(window1, window2)); + + WindowedValue<String> element1 = + WindowedValues.of("e1", new Instant(5), Arrays.asList(window1), PaneInfo.NO_FIRING); + WindowedValue<String> element2 = + WindowedValues.of("e2", new Instant(15), Arrays.asList(window2), PaneInfo.NO_FIRING); + + @SuppressWarnings("unchecked") + BagState<WindowedValue<String>> mockBag1 = mock(BagState.class); + @SuppressWarnings("unchecked") + BagState<WindowedValue<String>> mockBag2 = mock(BagState.class); + + when(mockBag1.read()).thenReturn(Arrays.asList(element1)); + when(mockBag2.read()).thenReturn(Arrays.asList(element2)); + + doNothing().when(mockFetcher).prefetchBlockedMap(); + when(mockFetcher.getReadyWindows()).thenReturn(readyWindows); + when(mockFetcher.prefetchElements(readyWindows)).thenReturn(Arrays.asList(mockBag1, mockBag2)); + doNothing().when(mockFetcher).releaseBlockedWindows(readyWindows); + + // When + Iterable<WindowedValue<String>> unblocked = () -> processor.tryUnblockElements(); + + // Then Review Comment: Done ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputProcessor.java: ########## @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker; + +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.Set; +import org.apache.beam.runners.core.TimerInternals; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** Helper class for handling elements blocked on side inputs. */ +class StreamingSideInputProcessor<InputT, W extends BoundedWindow> { + private final StreamingSideInputFetcher<InputT, W> sideInputFetcher; + + public StreamingSideInputProcessor(StreamingSideInputFetcher<InputT, W> sideInputFetcher) { + this.sideInputFetcher = sideInputFetcher; + } + + /** + * Handle's startBundle. If there are unblocked elements, process them and then return the set of + * windows that were unblocked. + */ + Iterator<WindowedValue<InputT>> tryUnblockElements() { + sideInputFetcher.prefetchBlockedMap(); + + // Find the set of ready windows. + Set<W> readyWindows = sideInputFetcher.getReadyWindows(); + + Iterable<BagState<WindowedValue<InputT>>> elementsBags = + sideInputFetcher.prefetchElements(readyWindows); + + // Return a lazy iterator to the released elements. This is a destructive iterator - it clears + // the bags after reading them. Bags can be paged in from the service, so we try to avoid + // materializing the whole + // bag into memory here. + Iterator<WindowedValue<InputT>> releasedElements = + new Iterator<WindowedValue<InputT>>() { + Iterator<BagState<WindowedValue<InputT>>> bagsIterator = elementsBags.iterator(); + @Nullable Iterator<WindowedValue<InputT>> currentBagElements; + @Nullable BagState<WindowedValue<InputT>> currentBag; + + @Override + public boolean hasNext() { + do { + if (currentBagElements == null || !currentBagElements.hasNext()) { + if (!advanceBag()) { + return false; + } + } + } while (!org.apache.beam.sdk.util.Preconditions.checkStateNotNull(currentBagElements) + .hasNext()); + return true; + } + + boolean advanceBag() { + // Once we finish reading a bag, clear it. + clearCurrentBag(); + if (bagsIterator.hasNext()) { + currentBag = bagsIterator.next(); + currentBagElements = currentBag.read().iterator(); + return true; + } else { + return false; + } + } + + void clearCurrentBag() { + if (currentBag != null) { + currentBag.clear(); + currentBag = null; + currentBagElements = null; + } + } + + @Override + public WindowedValue<InputT> next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return org.apache.beam.sdk.util.Preconditions.checkStateNotNull(currentBagElements) + .next(); + } + }; + + sideInputFetcher.releaseBlockedWindows(readyWindows); Review Comment: It should be safe. However I moved this to the end of the iterator just to be safe. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
