scwhittle commented on code in PR #28755:
URL: https://github.com/apache/beam/pull/28755#discussion_r1362080628
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java:
##########
@@ -292,41 +213,54 @@ protected SideInputReader getSideInputReaderForViews(
* Fetches the requested sideInput, and maintains a view of the cache that
doesn't remove items
* until the active work item is finished.
*
- * <p>If the side input was not ready, throws {@code IllegalStateException}
if the state is
- * {@literal CACHED_IN_WORKITEM} or returns null otherwise.
- *
- * <p>If the side input was ready and null, returns {@literal
Optional.absent()}. If the side
- * input was ready and non-null returns {@literal Optional.present(...)}.
+ * <p>If the side input cached, throws {@code IllegalStateException} if the
state is {@literal
+ * CACHED_IN_WORK_ITEM} or returns {@link SideInput<T>} which contains
{@link Optional<T>}.
*/
- private @Nullable <T> Optional<T> fetchSideInput(
+ @SuppressWarnings({"deprecation", "unchecked"})
Review Comment:
can you move the unchecked suppression to where cast is made (similar to
before)?
what is deprecation suppression required for?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java:
##########
@@ -292,41 +213,54 @@ protected SideInputReader getSideInputReaderForViews(
* Fetches the requested sideInput, and maintains a view of the cache that
doesn't remove items
* until the active work item is finished.
*
- * <p>If the side input was not ready, throws {@code IllegalStateException}
if the state is
- * {@literal CACHED_IN_WORKITEM} or returns null otherwise.
- *
- * <p>If the side input was ready and null, returns {@literal
Optional.absent()}. If the side
- * input was ready and non-null returns {@literal Optional.present(...)}.
+ * <p>If the side input cached, throws {@code IllegalStateException} if the
state is {@literal
+ * CACHED_IN_WORK_ITEM} or returns {@link SideInput<T>} which contains
{@link Optional<T>}.
*/
- private @Nullable <T> Optional<T> fetchSideInput(
+ @SuppressWarnings({"deprecation", "unchecked"})
+ private <T> SideInput<T> fetchSideInput(
+ PCollectionView<T> view,
+ BoundedWindow sideInputWindow,
+ @Nullable String stateFamily,
+ SideInputState state,
+ @Nullable Supplier<Closeable> scopedReadStateSupplier) {
+ Map<BoundedWindow, SideInput<?>> tagCache =
+ sideInputCache.computeIfAbsent(view.getTagInternal(), k -> new
HashMap<>());
+
+ Optional<SideInput<T>> seenSideInput =
Review Comment:
nit: cachedSideInput?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInput.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.streaming.sideinput;
+
+import com.google.auto.value.AutoValue;
+import java.util.Optional;
+import javax.annotation.Nullable;
+
+/**
+ * Entry in the side input cache that stores the value (null if not ready),
and the encoded size of
+ * the value.
+ */
+@AutoValue
+public abstract class SideInput<T> {
+ static <T> SideInput<T> ready(@Nullable T value, int encodedSize) {
Review Comment:
should this take Optional instead?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputCache.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.streaming.sideinput;
+
+import com.google.auto.value.AutoValue;
+import com.google.errorprone.annotations.CheckReturnValue;
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Weigher;
+
+/**
+ * Wrapper around {@code Cache<SideInputId, SideInput>} that mostly delegates
to the underlying
+ * cache, but adds threadsafe functionality to invalidate and load entries
that are not ready.
+ *
+ * @implNote Returned values are explicitly cast, because the {@link
#sideInputCache} holds wildcard
+ * types of all objects.
+ */
+@CheckReturnValue
+final class SideInputCache {
+
+ private static final long MAXIMUM_CACHE_WEIGHT = 100000000; /* 100 MB */
+ private static final long CACHE_ENTRY_EXPIRY_MINUTES = 1L;
+
+ private final Cache<Key, SideInput<?>> sideInputCache;
+
+ SideInputCache(Cache<Key, SideInput<?>> sideInputCache) {
+ this.sideInputCache = sideInputCache;
+ }
+
+ static SideInputCache create() {
+ return new SideInputCache(
+ CacheBuilder.newBuilder()
+ .maximumWeight(MAXIMUM_CACHE_WEIGHT)
+ .expireAfterWrite(CACHE_ENTRY_EXPIRY_MINUTES, TimeUnit.MINUTES)
+ .weigher((Weigher<Key, SideInput<?>>) (id, entry) -> entry.size())
+ .build());
+ }
+
+ synchronized <T> SideInput<T> invalidateThenLoadNewEntry(
+ Key key, Callable<SideInput<T>> cacheLoaderFn) throws ExecutionException
{
+ // Invalidate the existing not-ready entry. This must be done atomically
+ // so that another thread doesn't replace the entry with a ready entry,
which
+ // would then be deleted here.
+ SideInput<?> newEntry = sideInputCache.getIfPresent(key);
+ if (newEntry != null && !newEntry.isReady()) {
+ sideInputCache.invalidate(key);
+ }
+
+ @SuppressWarnings({
+ "unchecked" // cacheLoaderFn loads SideInput<T>, so value for Key is
always SideInput<T>.
Review Comment:
the comment here and elsewhere doesn't entirely explain why the cast is safe.
If we're using the cacheLoaderFn from this invocation then it's safe, but if
there was a previous call to get for the same key with a different type and
that was cached, the cast would be incorrect.
So I think the comment should be that the runner enforces that the same type
is used for the same key. (we could perhaps enforce this with some instanceof
check or making type part of the cache key?)
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInput.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.streaming.sideinput;
+
+import com.google.auto.value.AutoValue;
+import java.util.Optional;
+import javax.annotation.Nullable;
+
+/**
+ * Entry in the side input cache that stores the value (null if not ready),
and the encoded size of
Review Comment:
update comment since there can be optional.empty() + ready/notready?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java:
##########
@@ -292,41 +213,54 @@ protected SideInputReader getSideInputReaderForViews(
* Fetches the requested sideInput, and maintains a view of the cache that
doesn't remove items
* until the active work item is finished.
*
- * <p>If the side input was not ready, throws {@code IllegalStateException}
if the state is
- * {@literal CACHED_IN_WORKITEM} or returns null otherwise.
- *
- * <p>If the side input was ready and null, returns {@literal
Optional.absent()}. If the side
- * input was ready and non-null returns {@literal Optional.present(...)}.
+ * <p>If the side input cached, throws {@code IllegalStateException} if the
state is {@literal
+ * CACHED_IN_WORK_ITEM} or returns {@link SideInput<T>} which contains
{@link Optional<T>}.
*/
- private @Nullable <T> Optional<T> fetchSideInput(
+ @SuppressWarnings({"deprecation", "unchecked"})
+ private <T> SideInput<T> fetchSideInput(
+ PCollectionView<T> view,
+ BoundedWindow sideInputWindow,
+ @Nullable String stateFamily,
+ SideInputState state,
+ @Nullable Supplier<Closeable> scopedReadStateSupplier) {
+ Map<BoundedWindow, SideInput<?>> tagCache =
+ sideInputCache.computeIfAbsent(view.getTagInternal(), k -> new
HashMap<>());
+
+ Optional<SideInput<T>> seenSideInput =
+ Optional.ofNullable((SideInput<T>) tagCache.get(sideInputWindow));
+
+ if (state == SideInputState.CACHED_IN_WORK_ITEM &&
!seenSideInput.isPresent()) {
+ throw new IllegalStateException(
+ "Expected side input to be cached. Tag: " +
view.getTagInternal().getId());
+ }
+
+ return seenSideInput.orElseGet(
Review Comment:
the orelseget seems more confusing than just structuring like:
if (seenSideInput.isPresent()) {
return seenSideInput;
}
if (state == CACHED_IN_WORK_ITEM) {
throw ...
}
return fetchSideInputFromWindmill();
// or just inline that method
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java:
##########
@@ -292,41 +213,54 @@ protected SideInputReader getSideInputReaderForViews(
* Fetches the requested sideInput, and maintains a view of the cache that
doesn't remove items
* until the active work item is finished.
*
- * <p>If the side input was not ready, throws {@code IllegalStateException}
if the state is
- * {@literal CACHED_IN_WORKITEM} or returns null otherwise.
- *
- * <p>If the side input was ready and null, returns {@literal
Optional.absent()}. If the side
- * input was ready and non-null returns {@literal Optional.present(...)}.
+ * <p>If the side input cached, throws {@code IllegalStateException} if the
state is {@literal
Review Comment:
should this be: If the side input was *not* cached?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java:
##########
@@ -478,8 +415,84 @@ <T, W extends BoundedWindow> void writePCollectionViewData(
throws IOException;
}
- String getStateFamily(NameContext nameContext) {
- return nameContext.userName() == null ? null :
stateNameMap.get(nameContext.userName());
+ /**
+ * Execution states in Streaming are shared between multiple map-task
executors. Thus this class
+ * needs to be thread safe for multiple writers. A single stage could have
have multiple executors
+ * running concurrently.
+ */
+ public static class StreamingModeExecutionState
+ extends DataflowOperationContext.DataflowExecutionState {
+
+ // AtomicLong is used because this value is written in two places:
+ // 1. The sampling thread calls takeSample to increment the time spent in
this state
+ // 2. The reporting thread calls extractUpdate which reads the current sum
*AND* sets it to 0.
+ private final AtomicLong totalMillisInState = new AtomicLong();
+
+ // The worker that created this state. Used to report lulls back to the
worker.
+ @SuppressWarnings("unused") // Affects a public api
Review Comment:
can we remove the member variable and put the suppresion on the public
constructor parameter instead?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcher.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.streaming.sideinput;
+
+import static
org.apache.beam.sdk.transforms.Materializations.ITERABLE_MATERIALIZATION_URN;
+import static
org.apache.beam.sdk.transforms.Materializations.MULTIMAP_MATERIALIZATION_URN;
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.beam.runners.core.InMemoryMultimapSideInputView;
+import
org.apache.beam.runners.dataflow.worker.MetricTrackingWindmillServerStub;
+import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalData;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.Materializations.IterableView;
+import org.apache.beam.sdk.transforms.Materializations.MultimapView;
+import org.apache.beam.sdk.transforms.ViewFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.ByteStringOutputStream;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Class responsible for fetching state from the windmill server. */
+@NotThreadSafe
+public class SideInputStateFetcher {
+ private static final Logger LOG =
LoggerFactory.getLogger(SideInputStateFetcher.class);
+
+ private static final Set<String> SUPPORTED_MATERIALIZATIONS =
+ ImmutableSet.of(ITERABLE_MATERIALIZATION_URN,
MULTIMAP_MATERIALIZATION_URN);
+
+ private final SideInputCache sideInputCache;
+ private final MetricTrackingWindmillServerStub server;
+ private long bytesRead = 0L;
+
+ public SideInputStateFetcher(MetricTrackingWindmillServerStub server) {
+ this(server, SideInputCache.create());
+ }
+
+ SideInputStateFetcher(MetricTrackingWindmillServerStub server,
SideInputCache sideInputCache) {
+ this.server = server;
+ this.sideInputCache = sideInputCache;
+ }
+
+ private static <T> Iterable<?> decodeRawData(PCollectionView<T> view,
GlobalData data)
+ throws IOException {
+ @SuppressWarnings({
+ "deprecation" // The view's internal coder is required to decode the raw
data.
+ })
+ Iterable<?> decodedData =
+ !data.getData().isEmpty()
+ ?
IterableCoder.of(view.getCoderInternal()).decode(data.getData().newInput())
+ : Collections.emptyList();
+
+ return decodedData;
+ }
+
+ /** Returns a view of the underlying cache that keeps track of bytes read
separately. */
+ public SideInputStateFetcher byteTrackingView() {
+ return new SideInputStateFetcher(server, sideInputCache);
+ }
+
+ public long getBytesRead() {
+ return bytesRead;
+ }
+
+ /**
+ * Fetch the given side input, storing it in a process-level cache.
+ *
+ * <p>If state is KNOWN_READY, attempt to fetch the data regardless of
whether a not-ready entry
+ * was cached.
+ */
+ public <T> SideInput<T> fetchSideInput(
+ PCollectionView<T> view,
+ BoundedWindow sideWindow,
+ String stateFamily,
+ SideInputState state,
+ Supplier<Closeable> scopedReadStateSupplier) {
+ Callable<SideInput<T>> loadSideInputFromWindmill =
+ () -> loadSideInputFromWindmill(view, sideWindow, stateFamily,
scopedReadStateSupplier);
+
+ @SuppressWarnings({
+ "deprecation" // Required as part of the SideInputCacheKey, and not
exposed.
+ })
+ SideInputCache.Key sideInputCacheKey =
+ SideInputCache.Key.create(view.getTagInternal(), sideWindow);
+
+ try {
+ if (state == SideInputState.KNOWN_READY) {
+ Optional<SideInput<T>> existingCacheEntry =
sideInputCache.get(sideInputCacheKey);
+ if (!existingCacheEntry.isPresent()) {
+ return sideInputCache.getOrLoad(sideInputCacheKey,
loadSideInputFromWindmill);
+ }
+
+ if (!existingCacheEntry.get().isReady()) {
+ return sideInputCache.invalidateThenLoadNewEntry(
+ sideInputCacheKey, loadSideInputFromWindmill);
+ }
+
+ return existingCacheEntry.get();
+ }
+
+ return sideInputCache.getOrLoad(sideInputCacheKey,
loadSideInputFromWindmill);
+ } catch (Exception e) {
+ LOG.error("Fetch failed: ", e);
+ throw new RuntimeException("Exception while fetching side input: ", e);
+ }
+ }
+
+ private <T, SideWindowT extends BoundedWindow> GlobalData
fetchGlobalDataFromWindmill(
+ PCollectionView<T> view,
+ SideWindowT sideWindow,
+ String stateFamily,
+ Supplier<Closeable> scopedReadStateSupplier)
+ throws IOException {
+ @SuppressWarnings({
+ "deprecation", // Internal windowStrategy is required to fetch side
input data from Windmill.
+ "unchecked" // Internal windowing strategy matches WindowingStrategy<?,
SideWindowT>.
+ })
+ WindowingStrategy<?, SideWindowT> sideWindowStrategy =
+ (WindowingStrategy<?, SideWindowT>)
view.getWindowingStrategyInternal();
+
+ Coder<SideWindowT> windowCoder =
sideWindowStrategy.getWindowFn().windowCoder();
+
+ ByteStringOutputStream windowStream = new ByteStringOutputStream();
+ windowCoder.encode(sideWindow, windowStream);
+
+ @SuppressWarnings({
+ "deprecation" // PCollectionView's internal tag needed to create a
GlobalDataId.
+ })
+ Windmill.GlobalDataRequest request =
+ Windmill.GlobalDataRequest.newBuilder()
+ .setDataId(
+ Windmill.GlobalDataId.newBuilder()
+ .setTag(view.getTagInternal().getId())
+ .setVersion(windowStream.toByteString())
+ .build())
+ .setStateFamily(stateFamily)
+ .setExistenceWatermarkDeadline(
+ WindmillTimeUtils.harnessToWindmillTimestamp(
+
sideWindowStrategy.getTrigger().getWatermarkThatGuaranteesFiring(sideWindow)))
+ .build();
+
+ try (Closeable ignored = scopedReadStateSupplier.get()) {
+ return server.getSideInputData(request);
+ }
+ }
+
+ private <T> SideInput<T> loadSideInputFromWindmill(
+ PCollectionView<T> view,
+ BoundedWindow sideWindow,
+ String stateFamily,
+ Supplier<Closeable> scopedReadStateSupplier)
+ throws IOException {
+ validateViewMaterialization(view);
+ GlobalData data =
+ fetchGlobalDataFromWindmill(view, sideWindow, stateFamily,
scopedReadStateSupplier);
+ bytesRead += data.getSerializedSize();
+ return data.getIsReady() ? createSideInputCacheEntry(view, data) :
SideInput.notReady();
+ }
+
+ private <T> void validateViewMaterialization(PCollectionView<T> view) {
+ @SuppressWarnings({
+ "deprecation" // Underlying ViewFn is needed to validate the
materialization URN.
+ })
+ String materializationUrn = view.getViewFn().getMaterialization().getUrn();
+
+ checkState(
+ SUPPORTED_MATERIALIZATIONS.contains(materializationUrn),
+ "Only materialization's of type %s supported, received %s",
+ SUPPORTED_MATERIALIZATIONS,
+ materializationUrn);
+ }
+
+ private <T> SideInput<T> createSideInputCacheEntry(PCollectionView<T> view,
GlobalData data)
+ throws IOException {
+ @SuppressWarnings({
+ "deprecation" // Materialization URN is required to create the
SideInputCacheEntry
+ })
+ String viewMaterializationUrn =
view.getViewFn().getMaterialization().getUrn();
+ Iterable<?> rawData = decodeRawData(view, data);
+
+ switch (viewMaterializationUrn) {
+ case ITERABLE_MATERIALIZATION_URN:
+ {
+ @SuppressWarnings({
+ "unchecked", // ITERABLE_MATERIALIZATION_URN has
ViewFn<IterableView, T>.
+ "deprecation", // Underlying ViewFn is required to convert the
rawData
+ "rawtypes" // TODO(https://github.com/apache/beam/issues/20447)
+ })
+ ViewFn<IterableView, T> viewFn = (ViewFn<IterableView, T>)
view.getViewFn();
+ return SideInput.ready(viewFn.apply(() -> rawData),
data.getData().size());
+ }
+ case MULTIMAP_MATERIALIZATION_URN:
+ {
+ @SuppressWarnings({
+ "unchecked", // MULTIMAP_MATERIALIZATION_URN has
ViewFn<MultimapView, T>.
+ "deprecation", // Underlying ViewFn is required to convert the
rawData
+ "rawtypes" // TODO(https://github.com/apache/beam/issues/20447)
+ })
+ ViewFn<MultimapView, T> viewFn = (ViewFn<MultimapView, T>)
view.getViewFn();
+
+ @SuppressWarnings({
+ "deprecation" // internal coder's key coder is required to convert
the rawData into T
+ })
+ Coder<?> keyCoder = ((KvCoder<?, ?>)
view.getCoderInternal()).getKeyCoder();
+
+ @SuppressWarnings({
+ "unchecked", // Safe since multimap rawData is of type
Iterable<KV<K, V>>
+ "rawtypes", // TODO(https://github.com/apache/beam/issues/20447)
+ })
+ T multimapSideInputValue =
+ viewFn.apply(
+ InMemoryMultimapSideInputView.fromIterable(keyCoder,
(Iterable) rawData));
+ return SideInput.ready(multimapSideInputValue,
data.getData().size());
+ }
+ default:
+ {
+ @SuppressWarnings({
+ "deprecation" // ViewFn is required to get the underlying
materialization URN.
+ })
+ String unknownMaterializationFormatErrorMessage =
+ String.format(
+ "Unknown side input materialization format requested '%s'",
+ view.getViewFn().getMaterialization().getUrn());
+ throw new
IllegalStateException(unknownMaterializationFormatErrorMessage);
Review Comment:
nit: just inline String.format or just inline +
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcher.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.streaming.sideinput;
+
+import static
org.apache.beam.sdk.transforms.Materializations.ITERABLE_MATERIALIZATION_URN;
+import static
org.apache.beam.sdk.transforms.Materializations.MULTIMAP_MATERIALIZATION_URN;
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.beam.runners.core.InMemoryMultimapSideInputView;
+import
org.apache.beam.runners.dataflow.worker.MetricTrackingWindmillServerStub;
+import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalData;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.Materializations.IterableView;
+import org.apache.beam.sdk.transforms.Materializations.MultimapView;
+import org.apache.beam.sdk.transforms.ViewFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.ByteStringOutputStream;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Class responsible for fetching state from the windmill server. */
+@NotThreadSafe
+public class SideInputStateFetcher {
+ private static final Logger LOG =
LoggerFactory.getLogger(SideInputStateFetcher.class);
+
+ private static final Set<String> SUPPORTED_MATERIALIZATIONS =
+ ImmutableSet.of(ITERABLE_MATERIALIZATION_URN,
MULTIMAP_MATERIALIZATION_URN);
+
+ private final SideInputCache sideInputCache;
+ private final MetricTrackingWindmillServerStub server;
+ private long bytesRead = 0L;
+
+ public SideInputStateFetcher(MetricTrackingWindmillServerStub server) {
+ this(server, SideInputCache.create());
+ }
+
+ SideInputStateFetcher(MetricTrackingWindmillServerStub server,
SideInputCache sideInputCache) {
+ this.server = server;
+ this.sideInputCache = sideInputCache;
+ }
+
+ private static <T> Iterable<?> decodeRawData(PCollectionView<T> view,
GlobalData data)
+ throws IOException {
+ @SuppressWarnings({
+ "deprecation" // The view's internal coder is required to decode the raw
data.
+ })
+ Iterable<?> decodedData =
+ !data.getData().isEmpty()
+ ?
IterableCoder.of(view.getCoderInternal()).decode(data.getData().newInput())
+ : Collections.emptyList();
+
+ return decodedData;
+ }
+
+ /** Returns a view of the underlying cache that keeps track of bytes read
separately. */
+ public SideInputStateFetcher byteTrackingView() {
+ return new SideInputStateFetcher(server, sideInputCache);
+ }
+
+ public long getBytesRead() {
+ return bytesRead;
+ }
+
+ /**
+ * Fetch the given side input, storing it in a process-level cache.
+ *
+ * <p>If state is KNOWN_READY, attempt to fetch the data regardless of
whether a not-ready entry
+ * was cached.
+ */
+ public <T> SideInput<T> fetchSideInput(
+ PCollectionView<T> view,
+ BoundedWindow sideWindow,
+ String stateFamily,
+ SideInputState state,
+ Supplier<Closeable> scopedReadStateSupplier) {
+ Callable<SideInput<T>> loadSideInputFromWindmill =
+ () -> loadSideInputFromWindmill(view, sideWindow, stateFamily,
scopedReadStateSupplier);
+
+ @SuppressWarnings({
+ "deprecation" // Required as part of the SideInputCacheKey, and not
exposed.
+ })
+ SideInputCache.Key sideInputCacheKey =
+ SideInputCache.Key.create(view.getTagInternal(), sideWindow);
+
+ try {
+ if (state == SideInputState.KNOWN_READY) {
+ Optional<SideInput<T>> existingCacheEntry =
sideInputCache.get(sideInputCacheKey);
+ if (!existingCacheEntry.isPresent()) {
+ return sideInputCache.getOrLoad(sideInputCacheKey,
loadSideInputFromWindmill);
+ }
+
+ if (!existingCacheEntry.get().isReady()) {
+ return sideInputCache.invalidateThenLoadNewEntry(
+ sideInputCacheKey, loadSideInputFromWindmill);
+ }
+
+ return existingCacheEntry.get();
+ }
+
+ return sideInputCache.getOrLoad(sideInputCacheKey,
loadSideInputFromWindmill);
+ } catch (Exception e) {
+ LOG.error("Fetch failed: ", e);
+ throw new RuntimeException("Exception while fetching side input: ", e);
+ }
+ }
+
+ private <T, SideWindowT extends BoundedWindow> GlobalData
fetchGlobalDataFromWindmill(
+ PCollectionView<T> view,
+ SideWindowT sideWindow,
+ String stateFamily,
+ Supplier<Closeable> scopedReadStateSupplier)
+ throws IOException {
+ @SuppressWarnings({
+ "deprecation", // Internal windowStrategy is required to fetch side
input data from Windmill.
+ "unchecked" // Internal windowing strategy matches WindowingStrategy<?,
SideWindowT>.
+ })
+ WindowingStrategy<?, SideWindowT> sideWindowStrategy =
+ (WindowingStrategy<?, SideWindowT>)
view.getWindowingStrategyInternal();
+
+ Coder<SideWindowT> windowCoder =
sideWindowStrategy.getWindowFn().windowCoder();
+
+ ByteStringOutputStream windowStream = new ByteStringOutputStream();
+ windowCoder.encode(sideWindow, windowStream);
+
+ @SuppressWarnings({
+ "deprecation" // PCollectionView's internal tag needed to create a
GlobalDataId.
+ })
+ Windmill.GlobalDataRequest request =
+ Windmill.GlobalDataRequest.newBuilder()
+ .setDataId(
+ Windmill.GlobalDataId.newBuilder()
+ .setTag(view.getTagInternal().getId())
+ .setVersion(windowStream.toByteString())
+ .build())
+ .setStateFamily(stateFamily)
+ .setExistenceWatermarkDeadline(
+ WindmillTimeUtils.harnessToWindmillTimestamp(
+
sideWindowStrategy.getTrigger().getWatermarkThatGuaranteesFiring(sideWindow)))
+ .build();
+
+ try (Closeable ignored = scopedReadStateSupplier.get()) {
+ return server.getSideInputData(request);
+ }
+ }
+
+ private <T> SideInput<T> loadSideInputFromWindmill(
+ PCollectionView<T> view,
+ BoundedWindow sideWindow,
+ String stateFamily,
+ Supplier<Closeable> scopedReadStateSupplier)
+ throws IOException {
+ validateViewMaterialization(view);
+ GlobalData data =
+ fetchGlobalDataFromWindmill(view, sideWindow, stateFamily,
scopedReadStateSupplier);
+ bytesRead += data.getSerializedSize();
+ return data.getIsReady() ? createSideInputCacheEntry(view, data) :
SideInput.notReady();
+ }
+
+ private <T> void validateViewMaterialization(PCollectionView<T> view) {
+ @SuppressWarnings({
+ "deprecation" // Underlying ViewFn is needed to validate the
materialization URN.
+ })
+ String materializationUrn = view.getViewFn().getMaterialization().getUrn();
+
+ checkState(
+ SUPPORTED_MATERIALIZATIONS.contains(materializationUrn),
+ "Only materialization's of type %s supported, received %s",
+ SUPPORTED_MATERIALIZATIONS,
+ materializationUrn);
+ }
+
+ private <T> SideInput<T> createSideInputCacheEntry(PCollectionView<T> view,
GlobalData data)
+ throws IOException {
+ @SuppressWarnings({
+ "deprecation" // Materialization URN is required to create the
SideInputCacheEntry
+ })
+ String viewMaterializationUrn =
view.getViewFn().getMaterialization().getUrn();
+ Iterable<?> rawData = decodeRawData(view, data);
+
+ switch (viewMaterializationUrn) {
+ case ITERABLE_MATERIALIZATION_URN:
+ {
+ @SuppressWarnings({
+ "unchecked", // ITERABLE_MATERIALIZATION_URN has
ViewFn<IterableView, T>.
+ "deprecation", // Underlying ViewFn is required to convert the
rawData
+ "rawtypes" // TODO(https://github.com/apache/beam/issues/20447)
+ })
+ ViewFn<IterableView, T> viewFn = (ViewFn<IterableView, T>)
view.getViewFn();
+ return SideInput.ready(viewFn.apply(() -> rawData),
data.getData().size());
+ }
+ case MULTIMAP_MATERIALIZATION_URN:
+ {
+ @SuppressWarnings({
+ "unchecked", // MULTIMAP_MATERIALIZATION_URN has
ViewFn<MultimapView, T>.
+ "deprecation", // Underlying ViewFn is required to convert the
rawData
+ "rawtypes" // TODO(https://github.com/apache/beam/issues/20447)
+ })
+ ViewFn<MultimapView, T> viewFn = (ViewFn<MultimapView, T>)
view.getViewFn();
+
+ @SuppressWarnings({
+ "deprecation" // internal coder's key coder is required to convert
the rawData into T
+ })
+ Coder<?> keyCoder = ((KvCoder<?, ?>)
view.getCoderInternal()).getKeyCoder();
+
+ @SuppressWarnings({
+ "unchecked", // Safe since multimap rawData is of type
Iterable<KV<K, V>>
+ "rawtypes", // TODO(https://github.com/apache/beam/issues/20447)
+ })
+ T multimapSideInputValue =
+ viewFn.apply(
+ InMemoryMultimapSideInputView.fromIterable(keyCoder,
(Iterable) rawData));
+ return SideInput.ready(multimapSideInputValue,
data.getData().size());
+ }
+ default:
+ {
+ @SuppressWarnings({
+ "deprecation" // ViewFn is required to get the underlying
materialization URN.
+ })
+ String unknownMaterializationFormatErrorMessage =
+ String.format(
+ "Unknown side input materialization format requested '%s'",
+ view.getViewFn().getMaterialization().getUrn());
Review Comment:
just use viewMaterializationUrn and remove suppression here.
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcher.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.streaming.sideinput;
+
+import static
org.apache.beam.sdk.transforms.Materializations.ITERABLE_MATERIALIZATION_URN;
+import static
org.apache.beam.sdk.transforms.Materializations.MULTIMAP_MATERIALIZATION_URN;
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.beam.runners.core.InMemoryMultimapSideInputView;
+import
org.apache.beam.runners.dataflow.worker.MetricTrackingWindmillServerStub;
+import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalData;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.Materializations.IterableView;
+import org.apache.beam.sdk.transforms.Materializations.MultimapView;
+import org.apache.beam.sdk.transforms.ViewFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.ByteStringOutputStream;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Class responsible for fetching state from the windmill server. */
+@NotThreadSafe
+public class SideInputStateFetcher {
+ private static final Logger LOG =
LoggerFactory.getLogger(SideInputStateFetcher.class);
+
+ private static final Set<String> SUPPORTED_MATERIALIZATIONS =
+ ImmutableSet.of(ITERABLE_MATERIALIZATION_URN,
MULTIMAP_MATERIALIZATION_URN);
+
+ private final SideInputCache sideInputCache;
+ private final MetricTrackingWindmillServerStub server;
+ private long bytesRead = 0L;
+
+ public SideInputStateFetcher(MetricTrackingWindmillServerStub server) {
+ this(server, SideInputCache.create());
+ }
+
+ SideInputStateFetcher(MetricTrackingWindmillServerStub server,
SideInputCache sideInputCache) {
+ this.server = server;
+ this.sideInputCache = sideInputCache;
+ }
+
+ private static <T> Iterable<?> decodeRawData(PCollectionView<T> view,
GlobalData data)
+ throws IOException {
+ @SuppressWarnings({
+ "deprecation" // The view's internal coder is required to decode the raw
data.
+ })
+ Iterable<?> decodedData =
+ !data.getData().isEmpty()
+ ?
IterableCoder.of(view.getCoderInternal()).decode(data.getData().newInput())
+ : Collections.emptyList();
+
+ return decodedData;
+ }
+
+ /** Returns a view of the underlying cache that keeps track of bytes read
separately. */
+ public SideInputStateFetcher byteTrackingView() {
+ return new SideInputStateFetcher(server, sideInputCache);
+ }
+
+ public long getBytesRead() {
+ return bytesRead;
+ }
+
+ /**
+ * Fetch the given side input, storing it in a process-level cache.
+ *
+ * <p>If state is KNOWN_READY, attempt to fetch the data regardless of
whether a not-ready entry
+ * was cached.
+ */
+ public <T> SideInput<T> fetchSideInput(
+ PCollectionView<T> view,
+ BoundedWindow sideWindow,
+ String stateFamily,
+ SideInputState state,
+ Supplier<Closeable> scopedReadStateSupplier) {
+ Callable<SideInput<T>> loadSideInputFromWindmill =
+ () -> loadSideInputFromWindmill(view, sideWindow, stateFamily,
scopedReadStateSupplier);
+
+ @SuppressWarnings({
+ "deprecation" // Required as part of the SideInputCacheKey, and not
exposed.
+ })
+ SideInputCache.Key sideInputCacheKey =
+ SideInputCache.Key.create(view.getTagInternal(), sideWindow);
+
+ try {
+ if (state == SideInputState.KNOWN_READY) {
+ Optional<SideInput<T>> existingCacheEntry =
sideInputCache.get(sideInputCacheKey);
+ if (!existingCacheEntry.isPresent()) {
+ return sideInputCache.getOrLoad(sideInputCacheKey,
loadSideInputFromWindmill);
+ }
+
+ if (!existingCacheEntry.get().isReady()) {
+ return sideInputCache.invalidateThenLoadNewEntry(
+ sideInputCacheKey, loadSideInputFromWindmill);
+ }
+
+ return existingCacheEntry.get();
+ }
+
+ return sideInputCache.getOrLoad(sideInputCacheKey,
loadSideInputFromWindmill);
+ } catch (Exception e) {
+ LOG.error("Fetch failed: ", e);
+ throw new RuntimeException("Exception while fetching side input: ", e);
+ }
+ }
+
+ private <T, SideWindowT extends BoundedWindow> GlobalData
fetchGlobalDataFromWindmill(
+ PCollectionView<T> view,
+ SideWindowT sideWindow,
+ String stateFamily,
+ Supplier<Closeable> scopedReadStateSupplier)
+ throws IOException {
+ @SuppressWarnings({
+ "deprecation", // Internal windowStrategy is required to fetch side
input data from Windmill.
+ "unchecked" // Internal windowing strategy matches WindowingStrategy<?,
SideWindowT>.
+ })
+ WindowingStrategy<?, SideWindowT> sideWindowStrategy =
+ (WindowingStrategy<?, SideWindowT>)
view.getWindowingStrategyInternal();
+
+ Coder<SideWindowT> windowCoder =
sideWindowStrategy.getWindowFn().windowCoder();
+
+ ByteStringOutputStream windowStream = new ByteStringOutputStream();
+ windowCoder.encode(sideWindow, windowStream);
+
+ @SuppressWarnings({
+ "deprecation" // PCollectionView's internal tag needed to create a
GlobalDataId.
+ })
+ Windmill.GlobalDataRequest request =
+ Windmill.GlobalDataRequest.newBuilder()
+ .setDataId(
+ Windmill.GlobalDataId.newBuilder()
+ .setTag(view.getTagInternal().getId())
+ .setVersion(windowStream.toByteString())
+ .build())
+ .setStateFamily(stateFamily)
+ .setExistenceWatermarkDeadline(
+ WindmillTimeUtils.harnessToWindmillTimestamp(
+
sideWindowStrategy.getTrigger().getWatermarkThatGuaranteesFiring(sideWindow)))
+ .build();
+
+ try (Closeable ignored = scopedReadStateSupplier.get()) {
+ return server.getSideInputData(request);
+ }
+ }
+
+ private <T> SideInput<T> loadSideInputFromWindmill(
+ PCollectionView<T> view,
+ BoundedWindow sideWindow,
+ String stateFamily,
+ Supplier<Closeable> scopedReadStateSupplier)
+ throws IOException {
+ validateViewMaterialization(view);
+ GlobalData data =
+ fetchGlobalDataFromWindmill(view, sideWindow, stateFamily,
scopedReadStateSupplier);
+ bytesRead += data.getSerializedSize();
+ return data.getIsReady() ? createSideInputCacheEntry(view, data) :
SideInput.notReady();
+ }
+
+ private <T> void validateViewMaterialization(PCollectionView<T> view) {
+ @SuppressWarnings({
+ "deprecation" // Underlying ViewFn is needed to validate the
materialization URN.
+ })
+ String materializationUrn = view.getViewFn().getMaterialization().getUrn();
Review Comment:
how about a private helper method to get urn from view so you don't have to
repeat suppression and explanation a lot
--
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]