-
Notifications
You must be signed in to change notification settings - Fork 4.5k
[KafkaIO] Extract expiring memoizing supplier as a separate utility and run end offset refreshes in the current thread #36075
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
jrmccluskey
merged 3 commits into
apache:master
from
sjvanrossum:kafkaio-expiring-memoizing-serializable-supplier
Nov 5, 2025
Merged
Changes from all commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
489e17b
Extract expiring memoizing supplier as a separate utility and run end…
sjvanrossum c27b94c
Merge remote-tracking branch 'origin/master' into kafkaio-expiring-me…
sjvanrossum b65b747
Merge remote-tracking branch 'origin/master' into kafkaio-expiring-me…
sjvanrossum 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
101 changes: 101 additions & 0 deletions
101
...va/core/src/main/java/org/apache/beam/sdk/util/ExpiringMemoizingSerializableSupplier.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,101 @@ | ||
| /* | ||
| * 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.IOException; | ||
| import java.io.ObjectInputStream; | ||
| import java.time.Duration; | ||
| import java.util.concurrent.atomic.AtomicLongFieldUpdater; | ||
| import org.checkerframework.checker.nullness.qual.Nullable; | ||
|
|
||
| /** | ||
| * A thread-safe {@link SerializableSupplier} that wraps a {@link SerializableSupplier} and retains | ||
| * the supplier's result for the provided period. Lightweight locking and synchronization is used to | ||
| * guarantee mutual exclusivity and visibility of updates at the expense of single nanosecond | ||
| * precision. | ||
| * | ||
| * <p>The initial value and subsequently retained values are considered transient and will not be | ||
| * serialized. | ||
| */ | ||
| public final class ExpiringMemoizingSerializableSupplier<T extends @Nullable Object> | ||
| implements SerializableSupplier<T> { | ||
| // TODO(sjvanrossum): Replace with VarHandle after JDK 8 support is dropped. | ||
| @SuppressWarnings("rawtypes") | ||
| private static final AtomicLongFieldUpdater<ExpiringMemoizingSerializableSupplier> | ||
| DEADLINE_NANOS = | ||
| AtomicLongFieldUpdater.newUpdater( | ||
| ExpiringMemoizingSerializableSupplier.class, "deadlineNanos"); | ||
|
|
||
| private final SerializableSupplier<T> supplier; | ||
| private final long periodNanos; | ||
| private transient T value; | ||
| private transient volatile long deadlineNanos; | ||
|
|
||
| public ExpiringMemoizingSerializableSupplier( | ||
| SerializableSupplier<T> supplier, Duration period, T initialValue, Duration initialDelay) { | ||
| this.supplier = supplier; // final store | ||
| this.periodNanos = period.toNanos(); // final store | ||
| this.value = initialValue; // normal store | ||
|
|
||
| // Ordered stores may be reordered with subsequent loads. | ||
| // The default value of deadlineNanos permits an indefinite initial expiration depending on the | ||
| // clock's state. | ||
| this.deadlineNanos = | ||
| System.nanoTime() + initialDelay.toNanos() | ||
| & ~1L; // volatile store (sequentially consistent release) | ||
| } | ||
|
|
||
| @Override | ||
| public T get() { | ||
| final long deadlineNanos = this.deadlineNanos; // volatile load (acquire) | ||
| final long nowNanos; | ||
| final T result; | ||
|
|
||
| /* | ||
| * Sacrificing 1ns precision to pack the lock state into the low bit of deadlineNanos is deemed acceptable. | ||
| * Subsequent loads and stores are prevented from reordering before a volatile load. | ||
| * Preceeding loads and stores are prevented from reordering after an ordered store. | ||
| * A store to value can't be reordered after a store to deadlineNanos | ||
| * A store to deadlineNanos can be reordered after a load of deadlineNanos. | ||
| * The returned value will be as old as or younger than deadlineNanos. | ||
| */ | ||
| if ((deadlineNanos & 1L) == 0 | ||
| && deadlineNanos - (nowNanos = System.nanoTime()) <= 0L | ||
| && DEADLINE_NANOS | ||
| .compareAndSet( // volatile load/store (sequentially consistent acquire/release) | ||
| this, deadlineNanos, deadlineNanos | 1L)) { | ||
| try { | ||
| this.value = result = supplier.get(); // normal store | ||
| } finally { | ||
| DEADLINE_NANOS.lazySet(this, (nowNanos + periodNanos) & ~1L); // ordered store (release) | ||
| } | ||
| } else { | ||
| result = this.value; // normal load | ||
| } | ||
|
|
||
| return result; | ||
| } | ||
|
|
||
| private void readObject(ObjectInputStream is) throws IOException, ClassNotFoundException { | ||
| is.defaultReadObject(); | ||
|
|
||
| // Immediate initial expiration prevents a load of value before it is initialized. | ||
| this.deadlineNanos = | ||
| System.nanoTime() & ~1L; // volatile store (sequentially consistent release) | ||
| } | ||
| } | ||
166 changes: 166 additions & 0 deletions
166
...ore/src/test/java/org/apache/beam/sdk/util/ExpiringMemoizingSerializableSupplierTest.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,166 @@ | ||
| /* | ||
| * 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.assertNotNull; | ||
| import static org.junit.Assert.assertNotSame; | ||
| import static org.junit.Assert.assertSame; | ||
| import static org.junit.Assert.assertThrows; | ||
|
|
||
| import java.time.Duration; | ||
| import java.util.Arrays; | ||
| import java.util.Iterator; | ||
| import org.junit.Test; | ||
| import org.junit.runner.RunWith; | ||
| import org.junit.runners.JUnit4; | ||
|
|
||
| @RunWith(JUnit4.class) | ||
| public class ExpiringMemoizingSerializableSupplierTest { | ||
|
|
||
| @Test | ||
| public void testSupplierIsSerializable() { | ||
| final ExpiringMemoizingSerializableSupplier<?> instance = | ||
| new ExpiringMemoizingSerializableSupplier<>( | ||
| Object::new, Duration.ZERO, null, Duration.ZERO); | ||
|
|
||
| // Instances must be serializable. | ||
| SerializableUtils.ensureSerializable(instance); | ||
| } | ||
|
|
||
| @Test | ||
| public void testSameValueAfterConstruction() { | ||
| final Object initialValue = new Object(); | ||
| final ExpiringMemoizingSerializableSupplier<Object> instance = | ||
| new ExpiringMemoizingSerializableSupplier<>( | ||
| Object::new, Duration.ofHours(1), initialValue, Duration.ofHours(1)); | ||
|
|
||
| // Construction initializes deadlineNanos for delayed expiration. | ||
| // The supplied value must not be observed as uninitialized | ||
| // The supplied value is referentially equal to initialValue. | ||
| final Object instanceValue = instance.get(); | ||
| assertNotNull(instanceValue); | ||
| assertSame(initialValue, instanceValue); | ||
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| @Test | ||
| public void testDistinctValuesAfterDeserialization() throws Exception { | ||
| final Object initialValue = new Object(); | ||
| final ExpiringMemoizingSerializableSupplier<Object> instance = | ||
| new ExpiringMemoizingSerializableSupplier<>( | ||
| Object::new, Duration.ofHours(1), initialValue, Duration.ofHours(1)); | ||
|
|
||
| // Deserialized instances must be referentially distinct for the purpose of this test. | ||
| final byte[] serialized = SerializableUtils.serializeToByteArray(instance); | ||
| final ExpiringMemoizingSerializableSupplier<Object> deserialized1 = | ||
| (ExpiringMemoizingSerializableSupplier<Object>) | ||
| SerializableUtils.deserializeFromByteArray(serialized, "instance"); | ||
| final ExpiringMemoizingSerializableSupplier<Object> deserialized2 = | ||
| (ExpiringMemoizingSerializableSupplier<Object>) | ||
| SerializableUtils.deserializeFromByteArray(serialized, "instance"); | ||
| assertNotSame(instance, deserialized1); | ||
| assertNotSame(instance, deserialized2); | ||
| assertNotSame(deserialized1, deserialized2); | ||
|
|
||
| // Deserialization initializes deadlineNanos for immediate expiration. | ||
| // Supplied values must not be observed as uninitialized. | ||
| // The initial and supplied values are all referentially distinct. | ||
| final Object deserialized1Value = deserialized1.get(); | ||
| final Object deserialized2Value = deserialized2.get(); | ||
| assertNotNull(deserialized1Value); | ||
| assertNotNull(deserialized2Value); | ||
| assertNotSame(initialValue, deserialized1Value); | ||
| assertNotSame(initialValue, deserialized2Value); | ||
| assertNotSame(deserialized1Value, deserialized2Value); | ||
| } | ||
|
|
||
| @Test | ||
| public void testProgressAfterException() throws Exception { | ||
| final Object initialValue = new Object(); | ||
| final Object terminalValue = new Object(); | ||
| final Iterator<?> suppliedValues = | ||
| Arrays.asList(new Object(), new RuntimeException(), new Object()).iterator(); | ||
| final ExpiringMemoizingSerializableSupplier<?> instance = | ||
| new ExpiringMemoizingSerializableSupplier<>( | ||
| () -> { | ||
| if (!suppliedValues.hasNext()) { | ||
| return terminalValue; | ||
| } | ||
| final Object value = suppliedValues.next(); | ||
| if (value instanceof RuntimeException) { | ||
| throw (RuntimeException) value; | ||
| } | ||
| return value; | ||
| }, | ||
| Duration.ZERO, | ||
| initialValue, | ||
| Duration.ZERO); | ||
|
|
||
| // The initial value expires immediately and must not be observed. | ||
| final Object instanceValue = instance.get(); | ||
| assertNotSame(initialValue, instanceValue); | ||
|
|
||
| // An exception must be thrown for the purpose of this test. | ||
| assertThrows(RuntimeException.class, instance::get); | ||
|
|
||
| // Exceptions must not lock the instance state. | ||
| // The supplied value is referentially distinct from instanceValue for the purpose of this test. | ||
| // Note that parallelly observed supplied values may be referentially equal to instanceValue. | ||
| final Object intermediateValue = instance.get(); | ||
| assertNotSame(instanceValue, intermediateValue); | ||
|
|
||
| // The supplied value is referentially equal to terminalValue for the purpose of this test. | ||
| assertSame(terminalValue, instance.get()); | ||
| } | ||
|
|
||
| @Test | ||
| public void testInitialValueVisibilityOnDifferentThread() throws Exception { | ||
| final Object initialValue = new Object(); | ||
| final Object[] valueHolder = new Object[] {new Object()}; | ||
| final ExpiringMemoizingSerializableSupplier<Object> instance = | ||
| new ExpiringMemoizingSerializableSupplier<>( | ||
| Object::new, Duration.ZERO, initialValue, Duration.ofHours(1)); | ||
|
|
||
| // Initialization of value and deadlineNanos must be visible on other threads. | ||
| // The initial value must be supplied for delayed expiration. | ||
| final Thread t = new Thread(() -> valueHolder[0] = instance.get()); | ||
| t.start(); | ||
| t.join(); | ||
| final Object observedValue = valueHolder[0]; | ||
| assertNotNull(observedValue); | ||
| assertSame(initialValue, observedValue); | ||
| } | ||
|
|
||
| @Test | ||
| public void testIntermediateValueVisibilityOnDifferentThread() throws Exception { | ||
| final Object intermediateValue = new Object(); | ||
| final Object[] valueHolder = new Object[] {new Object()}; | ||
| final ExpiringMemoizingSerializableSupplier<Object> instance = | ||
| new ExpiringMemoizingSerializableSupplier<>( | ||
| () -> intermediateValue, Duration.ofHours(1), new Object(), Duration.ZERO); | ||
|
|
||
| // Initialization of value and deadlineNanos must be visible on other threads. | ||
| // The intermediate value must be supplied for immediate expiration. | ||
| final Thread t = new Thread(() -> valueHolder[0] = instance.get()); | ||
| t.start(); | ||
| t.join(); | ||
| final Object observedValue = valueHolder[0]; | ||
| assertNotNull(observedValue); | ||
| assertSame(intermediateValue, observedValue); | ||
| } | ||
| } |
Oops, something went wrong.
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.
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.
Are we packing the state here so that we can use a single atomic set?