-
Notifications
You must be signed in to change notification settings - Fork 4.5k
[BEAM-5987] Cache and share materialized side inputs between Spark tasks #7091
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
daca4cd
[BEAM-5987] Spark: Share cached side inputs between tasks.
55df568
[BEAM-5987] deserialized sideInputs are cached in executor
mareksimunek c72eee9
[BEAM-5987] cached spark side inputs are evicted based on time access
mareksimunek 6a71b50
[BEAM-5987] added more logging about broadcast
mareksimunek ddfe7dd
[BEAM-5987] fix for sideInputWithNull
mareksimunek File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
104 changes: 104 additions & 0 deletions
104
runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputStorage.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,104 @@ | ||
| /* | ||
| * 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.spark.util; | ||
|
|
||
| import java.util.Objects; | ||
| import java.util.concurrent.TimeUnit; | ||
| import org.apache.beam.sdk.transforms.windowing.BoundedWindow; | ||
| import org.apache.beam.sdk.values.PCollectionView; | ||
| import org.apache.beam.vendor.guava.v20_0.com.google.common.cache.Cache; | ||
| import org.apache.beam.vendor.guava.v20_0.com.google.common.cache.CacheBuilder; | ||
|
|
||
| /** | ||
| * Cache deserialized side inputs for executor so every task doesn't need to deserialize them again. | ||
| * Side inputs are stored in {@link Cache} with 5 minutes expireAfterAccess. | ||
| */ | ||
| class SideInputStorage { | ||
|
|
||
| /** JVM deserialized side input cache. */ | ||
| private static final Cache<Key<?>, Value<?>> materializedSideInputs = | ||
| CacheBuilder.newBuilder().expireAfterAccess(5, TimeUnit.MINUTES).build(); | ||
|
|
||
| static Cache<Key<?>, Value<?>> getMaterializedSideInputs() { | ||
| return materializedSideInputs; | ||
| } | ||
|
|
||
| /** | ||
| * Composite key of {@link PCollectionView} and {@link BoundedWindow} used to identify | ||
| * materialized results. | ||
| * | ||
| * @param <T> type of result | ||
| */ | ||
| public static class Key<T> { | ||
|
|
||
| private final PCollectionView<T> view; | ||
| private final BoundedWindow window; | ||
|
|
||
| Key(PCollectionView<T> view, BoundedWindow window) { | ||
| this.view = view; | ||
| this.window = window; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean equals(Object o) { | ||
| if (this == o) { | ||
| return true; | ||
| } | ||
| if (o == null || getClass() != o.getClass()) { | ||
| return false; | ||
| } | ||
| Key<?> key = (Key<?>) o; | ||
| return Objects.equals(view, key.view) && Objects.equals(window, key.window); | ||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() { | ||
| return Objects.hash(view, window); | ||
| } | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| String pName = view.getPCollection() != null ? view.getPCollection().getName() : "Unknown"; | ||
| return "Key{" | ||
| + "view=" | ||
| + view.getTagInternal() | ||
| + " of Pcollection[" | ||
| + pName | ||
| + "], window=" | ||
| + window | ||
| + '}'; | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Null value is not allowed in guava's Cache and is valid in SideInput so we use wrapper for | ||
| * cache value. | ||
| */ | ||
| public static class Value<T> { | ||
|
|
||
| T value; | ||
|
|
||
| Value(T value) { | ||
| this.value = value; | ||
| } | ||
|
|
||
| public T getValue() { | ||
| return value; | ||
| } | ||
| } | ||
| } |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am worried on the possible semantics consequences of
CachedSideInputReader.get()returning anullvalue when it is not in the Cache. Wouldn't it imply that a window could get an empty side input assigned?The documentation on this is not really clear (pinging @kennknowles to see if I am misreading it).
Wonder if there is a test to validate that this cannot happen or if we can create one somehow?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah, the meaning of that comment is that
nullis a value. You can have aPCollection<@Nullable Foo>that contains just one copy ofnulland useView.asSingleton()and the side input returns thenull.In other words,
getmust return a value of typeT. But the typeTmay itself be@Nullable Something. The annotation onSideInputReadershould be removed. It is is incorrect if we use a static analysis that understands this. Findbugs does not understand this but we should aspire for our annotations to be correct so the documentation is clear.