-
Notifications
You must be signed in to change notification settings - Fork 4.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Change UnboundedSourceAsSdfWrapperFn to share the cache across instan…
…ces. (#33901) add a utility class to enable sharing across all deserialized instances of a DoFn and use it in UnboundedSourceAsSdfWrapperFn to cache Readers across dofn instances
- Loading branch information
Showing
4 changed files
with
229 additions
and
31 deletions.
There are no files selected for viewing
This file contains 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
60 changes: 60 additions & 0 deletions
60
...src/main/java/org/apache/beam/sdk/util/MemoizingPerInstantiationSerializableSupplier.java
This file contains 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,60 @@ | ||
/* | ||
* 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.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; | ||
|
||
/** | ||
* A supplier that memoizes within an instantiation across serialization/deserialization. | ||
* | ||
* <p>Specifically the wrapped supplier will be called once and the result memoized per group | ||
* consisting of an instance and all instances deserialized from its serialized state. | ||
* | ||
* <p>A particular use for this is within a DoFn class to maintain shared state across all instances | ||
* of the DoFn that correspond to same step in the graph but separate from other steps in the graph | ||
* using the same DoFn. 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 MemoizingPerInstantiationSerializableSupplier<T> implements SerializableSupplier<T> { | ||
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 MemoizingPerInstantiationSerializableSupplier(SerializableSupplier<@NonNull T> supplier) { | ||
id = idGenerator.incrementAndGet(); | ||
this.supplier = supplier; | ||
} | ||
|
||
@Override | ||
@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 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
130 changes: 130 additions & 0 deletions
130
...test/java/org/apache/beam/sdk/util/MemoizingPerInstantiationSerializableSupplierTest.java
This file contains 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,130 @@ | ||
/* | ||
* 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 MemoizingPerInstantiationSerializableSupplierTest { | ||
|
||
@SuppressWarnings("unchecked") | ||
@Test | ||
public void testSharedAcrossDeserialize() throws Exception { | ||
MemoizingPerInstantiationSerializableSupplier<AtomicInteger> instance = | ||
new MemoizingPerInstantiationSerializableSupplier<>(AtomicInteger::new); | ||
SerializableUtils.ensureSerializable(instance); | ||
|
||
AtomicInteger i = instance.get(); | ||
i.set(10); | ||
assertSame(i, instance.get()); | ||
|
||
byte[] serialized = SerializableUtils.serializeToByteArray(instance); | ||
MemoizingPerInstantiationSerializableSupplier<AtomicInteger> deserialized1 = | ||
(MemoizingPerInstantiationSerializableSupplier<AtomicInteger>) | ||
SerializableUtils.deserializeFromByteArray(serialized, "instance"); | ||
assertSame(i, deserialized1.get()); | ||
|
||
MemoizingPerInstantiationSerializableSupplier<AtomicInteger> deserialized2 = | ||
(MemoizingPerInstantiationSerializableSupplier<AtomicInteger>) | ||
SerializableUtils.deserializeFromByteArray(serialized, "instance"); | ||
assertSame(i, deserialized2.get()); | ||
assertEquals(10, i.get()); | ||
} | ||
|
||
@Test | ||
public void testDifferentInstancesSeparate() throws Exception { | ||
MemoizingPerInstantiationSerializableSupplier<AtomicInteger> instance = | ||
new MemoizingPerInstantiationSerializableSupplier<>(AtomicInteger::new); | ||
SerializableUtils.ensureSerializable(instance); | ||
AtomicInteger i = instance.get(); | ||
i.set(10); | ||
assertSame(i, instance.get()); | ||
|
||
MemoizingPerInstantiationSerializableSupplier<AtomicInteger> instance2 = | ||
new MemoizingPerInstantiationSerializableSupplier<>(AtomicInteger::new); | ||
SerializableUtils.ensureSerializable(instance2); | ||
AtomicInteger j = instance2.get(); | ||
j.set(20); | ||
assertSame(j, instance2.get()); | ||
assertNotSame(j, i); | ||
|
||
MemoizingPerInstantiationSerializableSupplier<AtomicInteger> instance1clone = | ||
SerializableUtils.clone(instance); | ||
assertSame(instance1clone.get(), i); | ||
MemoizingPerInstantiationSerializableSupplier<AtomicInteger> instance2clone = | ||
SerializableUtils.clone(instance2); | ||
assertSame(instance2clone.get(), j); | ||
} | ||
|
||
@SuppressWarnings("unchecked") | ||
@Test | ||
public void testDifferentInstancesSeparateNoGetBeforeSerialization() throws Exception { | ||
MemoizingPerInstantiationSerializableSupplier<AtomicInteger> instance = | ||
new MemoizingPerInstantiationSerializableSupplier<>(AtomicInteger::new); | ||
SerializableUtils.ensureSerializable(instance); | ||
|
||
MemoizingPerInstantiationSerializableSupplier<AtomicInteger> instance2 = | ||
new MemoizingPerInstantiationSerializableSupplier<>(AtomicInteger::new); | ||
SerializableUtils.ensureSerializable(instance2); | ||
|
||
byte[] serialized = SerializableUtils.serializeToByteArray(instance); | ||
MemoizingPerInstantiationSerializableSupplier<AtomicInteger> deserialized1 = | ||
(MemoizingPerInstantiationSerializableSupplier<AtomicInteger>) | ||
SerializableUtils.deserializeFromByteArray(serialized, "instance"); | ||
MemoizingPerInstantiationSerializableSupplier<AtomicInteger> deserialized2 = | ||
(MemoizingPerInstantiationSerializableSupplier<AtomicInteger>) | ||
SerializableUtils.deserializeFromByteArray(serialized, "instance"); | ||
assertSame(deserialized1.get(), deserialized2.get()); | ||
|
||
MemoizingPerInstantiationSerializableSupplier<AtomicInteger> instance2clone = | ||
SerializableUtils.clone(instance2); | ||
assertNotSame(instance2clone.get(), deserialized1.get()); | ||
} | ||
|
||
@Test | ||
public void testDifferentTypes() throws Exception { | ||
MemoizingPerInstantiationSerializableSupplier<AtomicInteger> instance = | ||
new MemoizingPerInstantiationSerializableSupplier<>(AtomicInteger::new); | ||
SerializableUtils.ensureSerializable(instance); | ||
AtomicInteger i = instance.get(); | ||
i.set(10); | ||
assertSame(i, instance.get()); | ||
|
||
MemoizingPerInstantiationSerializableSupplier<ConcurrentHashMap<Integer, Integer>> instance2 = | ||
new MemoizingPerInstantiationSerializableSupplier<>(ConcurrentHashMap::new); | ||
SerializableUtils.ensureSerializable(instance2); | ||
ConcurrentHashMap<Integer, Integer> j = instance2.get(); | ||
j.put(1, 100); | ||
assertSame(j, instance2.get()); | ||
|
||
MemoizingPerInstantiationSerializableSupplier<AtomicInteger> instance1clone = | ||
SerializableUtils.clone(instance); | ||
assertSame(instance1clone.get(), i); | ||
MemoizingPerInstantiationSerializableSupplier<ConcurrentHashMap<Integer, Integer>> | ||
instance2clone = SerializableUtils.clone(instance2); | ||
assertSame(instance2clone.get(), j); | ||
} | ||
} |