Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
41 changes: 23 additions & 18 deletions sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@
import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.util.NameUtils;
import org.apache.beam.sdk.util.PerSerializationStatic;
import org.apache.beam.sdk.util.SerializableUtils;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
Expand Down Expand Up @@ -481,12 +482,31 @@ static class UnboundedSourceAsSDFWrapperFn<OutputT, CheckpointT extends Checkpoi
private static final Logger LOG = LoggerFactory.getLogger(UnboundedSourceAsSDFWrapperFn.class);
private static final int DEFAULT_BUNDLE_FINALIZATION_LIMIT_MINS = 10;
private final Coder<CheckpointT> checkpointCoder;
private @Nullable Cache<Object, UnboundedReader<OutputT>> cachedReaders;
private final PerSerializationStatic<Cache<Object, UnboundedReader<OutputT>>> cachedReaders;
private @Nullable Coder<UnboundedSourceRestriction<OutputT, CheckpointT>> restrictionCoder;

@VisibleForTesting
UnboundedSourceAsSDFWrapperFn(Coder<CheckpointT> checkpointCoder) {
this.checkpointCoder = checkpointCoder;
cachedReaders =
new PerSerializationStatic<>(
() ->
CacheBuilder.newBuilder()
.expireAfterWrite(1, TimeUnit.MINUTES)
.maximumSize(100)
.removalListener(
(RemovalListener<Object, UnboundedReader<OutputT>>)
removalNotification -> {
if (removalNotification.wasEvicted()) {
try {
Preconditions.checkNotNull(removalNotification.getValue())
.close();
} catch (IOException e) {
LOG.warn("Failed to close UnboundedReader.", e);
}
}
})
.build());
}

@GetInitialRestriction
Expand All @@ -498,22 +518,6 @@ public UnboundedSourceRestriction<OutputT, CheckpointT> initialRestriction(
@Setup
public void setUp() throws Exception {
restrictionCoder = restrictionCoder();
cachedReaders =
CacheBuilder.newBuilder()
.expireAfterWrite(1, TimeUnit.MINUTES)
.maximumSize(100)
.removalListener(
(RemovalListener<Object, UnboundedReader<OutputT>>)
removalNotification -> {
if (removalNotification.wasEvicted()) {
try {
Preconditions.checkNotNull(removalNotification.getValue()).close();
} catch (IOException e) {
LOG.warn("Failed to close UnboundedReader.", e);
}
}
})
.build();
}

@SplitRestriction
Expand Down Expand Up @@ -556,7 +560,8 @@ public void splitRestriction(
PipelineOptions pipelineOptions) {
Coder<UnboundedSourceRestriction<OutputT, CheckpointT>> restrictionCoder =
checkStateNotNull(this.restrictionCoder);
Cache<Object, UnboundedReader<OutputT>> cachedReaders = checkStateNotNull(this.cachedReaders);
Cache<Object, UnboundedReader<OutputT>> cachedReaders =
checkStateNotNull(this.cachedReaders.get());
return new UnboundedSourceAsSDFRestrictionTracker<>(
restriction, pipelineOptions, cachedReaders, restrictionCoder);
}
Expand Down
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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -179,12 +179,7 @@ public void testUnboundedSdfWrapperCacheStartedReaders() {
// read is default.
ExperimentalOptions.addExperiment(
pipeline.getOptions().as(ExperimentalOptions.class), "use_sdf_read");
// Force the pipeline to run with one thread to ensure the reader will be reused on one DoFn
// instance.
// We are not able to use DirectOptions because of circular dependency.
pipeline
.runWithAdditionalOptionArgs(ImmutableList.of("--targetParallelism=1"))
.waitUntilFinish();
pipeline.run().waitUntilFinish();
}

@Test
Expand Down
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);
}
}
Loading