-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Change UnboundedSourceAsSdfWrapperFn to share the cache across instances. #33901
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
Merged
Merged
Changes from 1 commit
Commits
Show all changes
6 commits
Select commit
Hold shift + click to select a range
a420191
Add a utility class to enable sharing across all deserialized instanc…
scwhittle 755170c
rename and remove size limit
scwhittle 43c67cc
rename member, use specific interface
scwhittle cd927ac
reword comment
scwhittle 9d19ec2
Use remove to avoid possible shared unbounded reader.
scwhittle 8126dd1
address comment
scwhittle 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
58 changes: 58 additions & 0 deletions
58
sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerSerializationStatic.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,58 @@ | ||
| /* | ||
| * 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.sdk.util; | ||
|
|
||
| import java.io.Serializable; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import javax.annotation.Nullable; | ||
| import org.checkerframework.checker.nullness.qual.MonotonicNonNull; | ||
| import org.checkerframework.checker.nullness.qual.NonNull; | ||
|
|
||
| /** | ||
| * An object that simplifies having a variable that behaves like a static object but which is scoped | ||
| * to deserialized instances. | ||
| * | ||
| * <p>In particular this can be useful for use within a DoFn class to maintain shared state across | ||
| * all instances of the DoFn that are the same step in the graph. This differs from a static | ||
| * variable which would be shared across all instances of the DoFn and a non-static variable which | ||
| * is per instance. | ||
| */ | ||
| public class PerSerializationStatic<T> implements Serializable { | ||
| private static final AtomicInteger idGenerator = new AtomicInteger(); | ||
| private final int id; | ||
|
|
||
| private static final ConcurrentHashMap<Integer, Object> staticCache = new ConcurrentHashMap<>(); | ||
| private final SerializableSupplier<@NonNull T> supplier; | ||
| private transient volatile @MonotonicNonNull T value; | ||
|
|
||
| public PerSerializationStatic(SerializableSupplier<@NonNull T> supplier) { | ||
| id = idGenerator.incrementAndGet(); | ||
| this.supplier = supplier; | ||
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| public T get() { | ||
| @Nullable T result = value; | ||
| if (result != null) { | ||
| return result; | ||
| } | ||
| @Nullable T mapValue = (T) staticCache.computeIfAbsent(id, ignored -> supplier.get()); | ||
| return value = Preconditions.checkStateNotNull(mapValue); | ||
| } | ||
| } |
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
126 changes: 126 additions & 0 deletions
126
sdks/java/core/src/test/java/org/apache/beam/sdk/util/PerSerializationStaticTest.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,126 @@ | ||
| /* | ||
| * 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.sdk.util; | ||
|
|
||
| import static org.junit.Assert.assertEquals; | ||
| import static org.junit.Assert.assertNotSame; | ||
| import static org.junit.Assert.assertSame; | ||
|
|
||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import org.junit.Test; | ||
| import org.junit.runner.RunWith; | ||
| import org.junit.runners.JUnit4; | ||
|
|
||
| @RunWith(JUnit4.class) | ||
| public class PerSerializationStaticTest { | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| @Test | ||
| public void testSharedAcrossDeserialize() throws Exception { | ||
| PerSerializationStatic<AtomicInteger> instance = | ||
| new PerSerializationStatic<>(AtomicInteger::new); | ||
| SerializableUtils.ensureSerializable(instance); | ||
|
|
||
| AtomicInteger i = instance.get(); | ||
| i.set(10); | ||
| assertSame(i, instance.get()); | ||
|
|
||
| byte[] serialized = SerializableUtils.serializeToByteArray(instance); | ||
| PerSerializationStatic<AtomicInteger> deserialized1 = | ||
| (PerSerializationStatic<AtomicInteger>) | ||
| SerializableUtils.deserializeFromByteArray(serialized, "instance"); | ||
| assertSame(i, deserialized1.get()); | ||
|
|
||
| PerSerializationStatic<AtomicInteger> deserialized2 = | ||
| (PerSerializationStatic<AtomicInteger>) | ||
| SerializableUtils.deserializeFromByteArray(serialized, "instance"); | ||
| assertSame(i, deserialized2.get()); | ||
| assertEquals(10, i.get()); | ||
| } | ||
|
|
||
| @Test | ||
| public void testDifferentInstancesSeparate() throws Exception { | ||
| PerSerializationStatic<AtomicInteger> instance = | ||
| new PerSerializationStatic<>(AtomicInteger::new); | ||
| SerializableUtils.ensureSerializable(instance); | ||
| AtomicInteger i = instance.get(); | ||
| i.set(10); | ||
| assertSame(i, instance.get()); | ||
|
|
||
| PerSerializationStatic<AtomicInteger> instance2 = | ||
| new PerSerializationStatic<>(AtomicInteger::new); | ||
| SerializableUtils.ensureSerializable(instance2); | ||
| AtomicInteger j = instance2.get(); | ||
| j.set(20); | ||
| assertSame(j, instance2.get()); | ||
| assertNotSame(j, i); | ||
|
|
||
| PerSerializationStatic<AtomicInteger> instance1clone = SerializableUtils.clone(instance); | ||
| assertSame(instance1clone.get(), i); | ||
| PerSerializationStatic<AtomicInteger> instance2clone = SerializableUtils.clone(instance2); | ||
| assertSame(instance2clone.get(), j); | ||
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| @Test | ||
| public void testDifferentInstancesSeparateNoGetBeforeSerialization() throws Exception { | ||
| PerSerializationStatic<AtomicInteger> instance = | ||
| new PerSerializationStatic<>(AtomicInteger::new); | ||
| SerializableUtils.ensureSerializable(instance); | ||
|
|
||
| PerSerializationStatic<AtomicInteger> instance2 = | ||
| new PerSerializationStatic<>(AtomicInteger::new); | ||
| SerializableUtils.ensureSerializable(instance2); | ||
|
|
||
| byte[] serialized = SerializableUtils.serializeToByteArray(instance); | ||
| PerSerializationStatic<AtomicInteger> deserialized1 = | ||
| (PerSerializationStatic<AtomicInteger>) | ||
| SerializableUtils.deserializeFromByteArray(serialized, "instance"); | ||
| PerSerializationStatic<AtomicInteger> deserialized2 = | ||
| (PerSerializationStatic<AtomicInteger>) | ||
| SerializableUtils.deserializeFromByteArray(serialized, "instance"); | ||
| assertSame(deserialized1.get(), deserialized2.get()); | ||
|
|
||
| PerSerializationStatic<AtomicInteger> instance2clone = SerializableUtils.clone(instance2); | ||
| assertNotSame(instance2clone.get(), deserialized1.get()); | ||
| } | ||
|
|
||
| @Test | ||
| public void testDifferentTypes() throws Exception { | ||
| PerSerializationStatic<AtomicInteger> instance = | ||
| new PerSerializationStatic<>(AtomicInteger::new); | ||
| SerializableUtils.ensureSerializable(instance); | ||
| AtomicInteger i = instance.get(); | ||
| i.set(10); | ||
| assertSame(i, instance.get()); | ||
|
|
||
| PerSerializationStatic<ConcurrentHashMap<Integer, Integer>> instance2 = | ||
| new PerSerializationStatic<>(ConcurrentHashMap::new); | ||
| SerializableUtils.ensureSerializable(instance2); | ||
| ConcurrentHashMap<Integer, Integer> j = instance2.get(); | ||
| j.put(1, 100); | ||
| assertSame(j, instance2.get()); | ||
|
|
||
| PerSerializationStatic<AtomicInteger> instance1clone = SerializableUtils.clone(instance); | ||
| assertSame(instance1clone.get(), i); | ||
| PerSerializationStatic<ConcurrentHashMap<Integer, Integer>> instance2clone = | ||
| SerializableUtils.clone(instance2); | ||
| assertSame(instance2clone.get(), j); | ||
| } | ||
| } |
Oops, something went wrong.
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.