From 45c3eaeab7a93c2c4d4d1a28b112e140111b4327 Mon Sep 17 00:00:00 2001 From: Bryce Anderson Date: Tue, 4 Feb 2025 16:18:59 -0700 Subject: [PATCH 01/14] concurrent-api: make context capture more generic Motivation: Why is this change being made? Modifications: - List the changes Result: What is the result of this change? --- .../api/AbstractAsyncContextProvider.java | 265 +++++++++++++++ ...stractAsynchronousCompletableOperator.java | 8 +- ...AbstractAsynchronousPublisherOperator.java | 8 +- .../AbstractAsynchronousSingleOperator.java | 8 +- ...tractCompletableAndSingleConcatenated.java | 9 +- .../api/AbstractMergeCompletableOperator.java | 8 +- .../api/AbstractPubToCompletable.java | 7 +- .../concurrent/api/AbstractPubToSingle.java | 7 +- .../api/AbstractPublisherGroupBy.java | 25 +- .../api/AbstractSynchronousCompletable.java | 6 +- ...bstractSynchronousCompletableOperator.java | 6 +- .../api/AbstractSynchronousPublisher.java | 6 +- .../AbstractSynchronousPublisherOperator.java | 6 +- .../api/AbstractSynchronousSingle.java | 6 +- .../AbstractSynchronousSingleOperator.java | 6 +- .../concurrent/api/AsyncContext.java | 26 +- .../concurrent/api/AsyncContextProvider.java | 83 ++--- .../concurrent/api/CacheSingle.java | 9 +- .../concurrent/api/CapturedContext.java | 30 ++ .../concurrent/api/Completable.java | 28 +- .../api/CompletableConcatWithCompletable.java | 7 +- .../CompletableConcatWithCompletables.java | 7 +- .../api/CompletableConcatWithPublisher.java | 5 +- .../api/CompletableConcatWithSingle.java | 6 +- .../api/CompletableMergeWithPublisher.java | 27 +- .../api/CompletableSetContextOnSubscribe.java | 16 +- .../CompletableShareContextOnSubscribe.java | 8 +- .../api/CompletableToPublisher.java | 15 +- .../concurrent/api/CompletableToSingle.java | 5 +- .../api/ContextAwareExecutorUtils.java | 6 +- .../api/ContextCaptureProvider.java | 15 + .../concurrent/api/ContextMapThreadLocal.java | 80 +++++ .../api/ContextPreservingBiConsumer.java | 11 +- .../api/ContextPreservingBiFunction.java | 10 +- .../api/ContextPreservingCallable.java | 8 +- .../api/ContextPreservingCancellable.java | 9 +- ...rvingCancellableCompletableSubscriber.java | 5 +- ...PreservingCancellableSingleSubscriber.java | 5 +- .../ContextPreservingCompletableFuture.java | 11 +- ...ontextPreservingCompletableSubscriber.java | 11 +- ...ngCompletableSubscriberAndCancellable.java | 3 +- .../api/ContextPreservingConsumer.java | 8 +- .../api/ContextPreservingFunction.java | 10 +- .../api/ContextPreservingRunnable.java | 8 +- .../ContextPreservingSingleSubscriber.java | 11 +- ...servingSingleSubscriberAndCancellable.java | 3 +- .../api/ContextPreservingSubscriber.java | 13 +- ...xtPreservingSubscriberAndSubscription.java | 3 +- .../api/ContextPreservingSubscription.java | 11 +- ...ntextPreservingSubscriptionSubscriber.java | 5 +- .../CustomCaptureAsyncContextProvider.java | 22 ++ .../api/DefaultAsyncContextProvider.java | 312 +----------------- .../api/LiftSynchronousPublisherToSingle.java | 5 +- .../concurrent/api/MulticastPublisher.java | 47 +-- .../api/NoopAsyncContextProvider.java | 44 +-- .../api/OnErrorResumeCompletable.java | 15 +- .../api/OnErrorResumePublisher.java | 16 +- .../concurrent/api/OnErrorResumeSingle.java | 16 +- ...scribeIgnoringSubscriberForOffloading.java | 5 +- .../PublishAndSubscribeOnCompletables.java | 15 +- .../api/PublishAndSubscribeOnPublishers.java | 15 +- .../api/PublishAndSubscribeOnSingles.java | 15 +- .../servicetalk/concurrent/api/Publisher.java | 28 +- .../concurrent/api/PublisherGroupBy.java | 12 +- .../concurrent/api/PublisherGroupToMany.java | 10 +- .../api/PublisherSetContextOnSubscribe.java | 16 +- .../api/PublisherShareContextOnSubscribe.java | 8 +- .../concurrent/api/RedoPublisher.java | 14 +- .../concurrent/api/RedoWhenPublisher.java | 15 +- .../concurrent/api/ReduceSingle.java | 9 +- .../concurrent/api/RepeatWhenSingle.java | 15 +- .../concurrent/api/RetrySingle.java | 15 +- .../concurrent/api/RetryWhenSingle.java | 15 +- .../api/ScanWithLifetimePublisher.java | 11 +- .../concurrent/api/ScanWithPublisher.java | 15 +- .../io/servicetalk/concurrent/api/Single.java | 32 +- .../concurrent/api/SingleAmbWith.java | 7 +- .../api/SingleConcatWithCompletable.java | 6 +- .../api/SingleConcatWithPublisher.java | 5 +- .../api/SingleFlatMapCompletable.java | 15 +- .../api/SingleFlatMapPublisher.java | 13 +- .../api/SingleSetContextOnSubscribe.java | 16 +- .../api/SingleShareContextOnSubscribe.java | 8 +- .../concurrent/api/SingleToCompletable.java | 5 +- .../api/SingleToCompletableFuture.java | 1 + .../concurrent/api/SingleToPublisher.java | 17 +- .../TaskBasedAsyncCompletableOperator.java | 5 +- .../api/TaskBasedAsyncPublisherOperator.java | 5 +- .../api/TaskBasedAsyncSingleOperator.java | 5 +- .../concurrent/api/TimeoutCompletable.java | 11 +- .../api/TimeoutDemandPublisher.java | 11 +- .../concurrent/api/TimeoutPublisher.java | 19 +- .../concurrent/api/TimeoutSingle.java | 11 +- .../api/DefaultAsyncContextProviderTest.java | 8 +- .../concurrent/api/SubscribeThrowsTest.java | 2 +- 95 files changed, 931 insertions(+), 880 deletions(-) create mode 100644 servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsyncContextProvider.java create mode 100644 servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java create mode 100644 servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java create mode 100644 servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextMapThreadLocal.java create mode 100644 servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsyncContextProvider.java new file mode 100644 index 0000000000..cdc1152a5a --- /dev/null +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsyncContextProvider.java @@ -0,0 +1,265 @@ +/* + * Copyright © 2018-2019, 2021 Apple Inc. and the ServiceTalk project authors + * + * Licensed 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 io.servicetalk.concurrent.api; + +import io.servicetalk.concurrent.CompletableSource; +import io.servicetalk.concurrent.PublisherSource.Subscriber; +import io.servicetalk.concurrent.SingleSource; +import io.servicetalk.context.api.ContextMap; + +import javax.annotation.Nonnull; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; +import java.util.function.BiConsumer; +import java.util.function.BiFunction; +import java.util.function.Consumer; +import java.util.function.Function; + +abstract class AbstractAsyncContextProvider implements AsyncContextProvider { + + AbstractAsyncContextProvider() { + // singleton + } + + @Nonnull + @Override + public final ContextMap context() { + return ContextMapThreadLocal.context(); + } + + @Override + public final CompletableSource.Subscriber wrapCancellable(final CompletableSource.Subscriber subscriber, + final CapturedContext context) { + if (subscriber instanceof ContextPreservingCompletableSubscriber) { + final ContextPreservingCompletableSubscriber s = (ContextPreservingCompletableSubscriber) subscriber; + if (s.saved == context) { + return subscriber instanceof ContextPreservingCompletableSubscriberAndCancellable ? subscriber : + new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber && + ((ContextPreservingCancellableCompletableSubscriber) subscriber).saved == context) { + // no need to check for instanceof ContextPreservingCompletableSubscriberAndCancellable, because + // it extends from ContextPreservingSingleSubscriber. + return subscriber; + } + return new ContextPreservingCancellableCompletableSubscriber(subscriber, context); + } + + @Override + public final CompletableSource.Subscriber wrapCompletableSubscriber(final CompletableSource.Subscriber subscriber, + final CapturedContext context) { + if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber) { + final ContextPreservingCancellableCompletableSubscriber s = + (ContextPreservingCancellableCompletableSubscriber) subscriber; + if (s.saved == context) { + // replace current wrapper with wrapper that includes Subscriber and Cancellable + return new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingCompletableSubscriber && + ((ContextPreservingCompletableSubscriber) subscriber).saved == context) { + // no need to check for instanceof ContextPreservingCompletableSubscriberAndCancellable, because + // it extends from ContextPreservingCompletableSubscriber. + return subscriber; + } + return new ContextPreservingCompletableSubscriber(subscriber, context); + } + + @Override + public final CompletableSource.Subscriber wrapCompletableSubscriberAndCancellable( + final CompletableSource.Subscriber subscriber, final CapturedContext context) { + if (subscriber instanceof ContextPreservingCompletableSubscriber) { + final ContextPreservingCompletableSubscriber s = (ContextPreservingCompletableSubscriber) subscriber; + if (s.saved == context) { + return subscriber instanceof ContextPreservingCompletableSubscriberAndCancellable ? subscriber : + new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber) { + final ContextPreservingCancellableCompletableSubscriber s = + (ContextPreservingCancellableCompletableSubscriber) subscriber; + if (s.saved == context) { + return new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); + } + } + return new ContextPreservingCompletableSubscriberAndCancellable(subscriber, context); + } + + @Override + public final SingleSource.Subscriber wrapCancellable(final SingleSource.Subscriber subscriber, + final CapturedContext context) { + if (subscriber instanceof ContextPreservingSingleSubscriber) { + final ContextPreservingSingleSubscriber s = (ContextPreservingSingleSubscriber) subscriber; + if (s.saved == context) { + return subscriber instanceof ContextPreservingSingleSubscriberAndCancellable ? subscriber : + new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingCancellableSingleSubscriber && + ((ContextPreservingCancellableSingleSubscriber) subscriber).saved == context) { + // no need to check for instanceof ContextPreservingSingleSubscriberAndCancellable, because + // it extends from ContextPreservingSingleSubscriber. + return subscriber; + } + return new ContextPreservingCancellableSingleSubscriber<>(subscriber, context); + } + + @Override + public final SingleSource.Subscriber wrapSingleSubscriber(final SingleSource.Subscriber subscriber, + final CapturedContext context) { + if (subscriber instanceof ContextPreservingCancellableSingleSubscriber) { + final ContextPreservingCancellableSingleSubscriber s = + (ContextPreservingCancellableSingleSubscriber) subscriber; + if (s.saved == context) { + return new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingSingleSubscriber && + ((ContextPreservingSingleSubscriber) subscriber).saved == context) { + // no need to check for instanceof ContextPreservingSingleSubscriberAndCancellable, because + // it extends from ContextPreservingSingleSubscriber. + return subscriber; + } + return new ContextPreservingSingleSubscriber<>(subscriber, context); + } + + @Override + public final SingleSource.Subscriber wrapSingleSubscriberAndCancellable( + final SingleSource.Subscriber subscriber, final CapturedContext context) { + if (subscriber instanceof ContextPreservingSingleSubscriber) { + final ContextPreservingSingleSubscriber s = (ContextPreservingSingleSubscriber) subscriber; + if (s.saved == context) { + return subscriber instanceof ContextPreservingSingleSubscriberAndCancellable ? subscriber : + new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingCancellableSingleSubscriber) { + final ContextPreservingCancellableSingleSubscriber s = + (ContextPreservingCancellableSingleSubscriber) subscriber; + if (s.saved == context) { + return new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); + } + } + return new ContextPreservingSingleSubscriberAndCancellable<>(subscriber, context); + } + + @Override + public final Subscriber wrapSubscription(final Subscriber subscriber, final CapturedContext context) { + if (subscriber instanceof ContextPreservingSubscriber) { + final ContextPreservingSubscriber s = (ContextPreservingSubscriber) subscriber; + if (s.saved == context) { + return subscriber instanceof ContextPreservingSubscriberAndSubscription ? subscriber : + new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingSubscriptionSubscriber && + ((ContextPreservingSubscriptionSubscriber) subscriber).saved == context) { + // no need to check for instanceof ContextPreservingSubscriberAndSubscription, because + // it extends from ContextPreservingSubscriptionSubscriber. + return subscriber; + } + return new ContextPreservingSubscriptionSubscriber<>(subscriber, context); + } + + @Override + public final Subscriber wrapPublisherSubscriber(final Subscriber subscriber, final CapturedContext context) { + if (subscriber instanceof ContextPreservingSubscriptionSubscriber) { + final ContextPreservingSubscriptionSubscriber s = + (ContextPreservingSubscriptionSubscriber) subscriber; + if (s.saved == context) { + return new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingSubscriber && + ((ContextPreservingSubscriber) subscriber).saved == context) { + // no need to check for instanceof ContextPreservingSubscriberAndSubscription, because + // it extends from ContextPreservingSubscriptionSubscriber. + return subscriber; + } + return new ContextPreservingSubscriber<>(subscriber, context); + } + + @Override + public final Subscriber wrapPublisherSubscriberAndSubscription(final Subscriber subscriber, + final CapturedContext context) { + if (subscriber instanceof ContextPreservingSubscriber) { + final ContextPreservingSubscriber s = (ContextPreservingSubscriber) subscriber; + if (s.saved == context) { + return subscriber instanceof ContextPreservingSubscriberAndSubscription ? subscriber : + new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingSubscriptionSubscriber) { + final ContextPreservingSubscriptionSubscriber s = + (ContextPreservingSubscriptionSubscriber) subscriber; + if (s.saved == context) { + return new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); + } + } + return new ContextPreservingSubscriberAndSubscription<>(subscriber, context); + } + + @Override + public final Executor wrapJdkExecutor(final Executor executor) { + return ContextPreservingExecutor.of(executor); + } + + @Override + public final ExecutorService wrapJdkExecutorService(final ExecutorService executor) { + return ContextPreservingExecutorService.of(executor); + } + + @Override + public final io.servicetalk.concurrent.api.Executor wrapExecutor(final io.servicetalk.concurrent.api.Executor executor) { + return ContextPreservingStExecutor.of(executor); + } + + @Override + public final ScheduledExecutorService wrapJdkScheduledExecutorService(final ScheduledExecutorService executor) { + return ContextPreservingScheduledExecutorService.of(executor); + } + + @Override + public final CompletableFuture wrapCompletableFuture(final CompletableFuture future, + final CapturedContext context) { + return ContextPreservingCompletableFuture.newContextPreservingFuture(future, context); + } + + @Override + public final Runnable wrapRunnable(final Runnable runnable, final CapturedContext context) { + return new ContextPreservingRunnable(runnable, context); + } + + @Override + public final Callable wrapCallable(final Callable callable, final CapturedContext context) { + return new ContextPreservingCallable<>(callable, context); + } + + @Override + public final Consumer wrapConsumer(final Consumer consumer, final CapturedContext context) { + return new ContextPreservingConsumer<>(consumer, context); + } + + @Override + public final Function wrapFunction(final Function func, final CapturedContext context) { + return new ContextPreservingFunction<>(func, context); + } + + @Override + public final BiConsumer wrapBiConsumer(final BiConsumer consumer, final CapturedContext context) { + return new ContextPreservingBiConsumer<>(consumer, context); + } + + @Override + public final BiFunction wrapBiFunction(final BiFunction func, final CapturedContext context) { + return new ContextPreservingBiFunction<>(func, context); + } +} diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsynchronousCompletableOperator.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsynchronousCompletableOperator.java index 85105ccbb7..a883c6c754 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsynchronousCompletableOperator.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsynchronousCompletableOperator.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import static java.util.Objects.requireNonNull; /** @@ -38,13 +36,13 @@ abstract class AbstractAsynchronousCompletableOperator extends AbstractNoHandleS @Override final void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { // The AsyncContext needs to be preserved when ever we interact with the original Subscriber, so we wrap it here // with the original contextMap. Otherwise some other context may leak into this subscriber chain from the other // side of the asynchronous boundary. final Subscriber operatorSubscriber = - contextProvider.wrapCompletableSubscriberAndCancellable(subscriber, contextMap); + contextProvider.wrapCompletableSubscriberAndCancellable(subscriber, capturedContext); final Subscriber upstreamSubscriber = apply(operatorSubscriber); - original.delegateSubscribe(upstreamSubscriber, contextMap, contextProvider); + original.delegateSubscribe(upstreamSubscriber, capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsynchronousPublisherOperator.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsynchronousPublisherOperator.java index b9121eb818..24e112dde6 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsynchronousPublisherOperator.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsynchronousPublisherOperator.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import static java.util.Objects.requireNonNull; /** @@ -41,13 +39,13 @@ abstract class AbstractAsynchronousPublisherOperator extends AbstractNoHan @Override final void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { // The AsyncContext needs to be preserved when ever we interact with the original Subscriber, so we wrap it here // with the original contextMap. Otherwise some other context may leak into this subscriber chain from the other // side of the asynchronous boundary. final Subscriber operatorSubscriber = - contextProvider.wrapPublisherSubscriberAndSubscription(subscriber, contextMap); + contextProvider.wrapPublisherSubscriberAndSubscription(subscriber, capturedContext); final Subscriber upstreamSubscriber = apply(operatorSubscriber); - original.delegateSubscribe(upstreamSubscriber, contextMap, contextProvider); + original.delegateSubscribe(upstreamSubscriber, capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsynchronousSingleOperator.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsynchronousSingleOperator.java index 2f81fdc402..f5f6528ec8 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsynchronousSingleOperator.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsynchronousSingleOperator.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import static java.util.Objects.requireNonNull; /** @@ -41,13 +39,13 @@ abstract class AbstractAsynchronousSingleOperator extends AbstractNoHandle @Override final void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { // The AsyncContext needs to be preserved when ever we interact with the original Subscriber, so we wrap it here // with the original contextMap. Otherwise some other context may leak into this subscriber chain from the other // side of the asynchronous boundary. final Subscriber operatorSubscriber = - contextProvider.wrapSingleSubscriberAndCancellable(subscriber, contextMap); + contextProvider.wrapSingleSubscriberAndCancellable(subscriber, capturedContext); final Subscriber upstreamSubscriber = apply(operatorSubscriber); - original.delegateSubscribe(upstreamSubscriber, contextMap, contextProvider); + original.delegateSubscribe(upstreamSubscriber, capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractCompletableAndSingleConcatenated.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractCompletableAndSingleConcatenated.java index 49984237a0..990ed7bfa7 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractCompletableAndSingleConcatenated.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractCompletableAndSingleConcatenated.java @@ -18,7 +18,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.CompletableSource; import io.servicetalk.concurrent.internal.SequentialCancellable; -import io.servicetalk.context.api.ContextMap; import javax.annotation.Nullable; @@ -26,13 +25,13 @@ abstract class AbstractCompletableAndSingleConcatenated extends AbstractNoHan @Override protected void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { - final Subscriber wrappedSubscriber = contextProvider.wrapSingleSubscriber(subscriber, contextMap); - delegateSubscribeToOriginal(wrappedSubscriber, contextMap, contextProvider); + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + final Subscriber wrappedSubscriber = contextProvider.wrapSingleSubscriber(subscriber, capturedContext); + delegateSubscribeToOriginal(wrappedSubscriber, capturedContext, contextProvider); } abstract void delegateSubscribeToOriginal(Subscriber offloadSubscriber, - ContextMap contextMap, AsyncContextProvider contextProvider); + CapturedContext capturedContext, AsyncContextProvider contextProvider); abstract static class AbstractConcatWithSubscriber implements Subscriber, CompletableSource.Subscriber { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractMergeCompletableOperator.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractMergeCompletableOperator.java index b0fb8eb08e..609d40876f 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractMergeCompletableOperator.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractMergeCompletableOperator.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import static java.util.Objects.requireNonNull; abstract class AbstractMergeCompletableOperator @@ -30,15 +28,15 @@ abstract class AbstractMergeCompletableOperator extends AbstractNoHandleSubscribeComp @Override final void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { // We are now subscribing to the original Publisher chain for the first time, wrap Subscription to preserve the // context. PublisherSource.Subscriber wrappedSubscriber = - contextProvider.wrapSubscription(newSubscriber(subscriber), contextMap); - source.delegateSubscribe(wrappedSubscriber, contextMap, contextProvider); + contextProvider.wrapSubscription(newSubscriber(subscriber), capturedContext); + source.delegateSubscribe(wrappedSubscriber, capturedContext, contextProvider); } abstract static class AbstractPubToCompletableSubscriber extends DelayedCancellable diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractPubToSingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractPubToSingle.java index 11239c2430..313b8d11e4 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractPubToSingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractPubToSingle.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.PublisherSource; import io.servicetalk.concurrent.PublisherSource.Subscription; -import io.servicetalk.context.api.ContextMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,12 +37,12 @@ abstract class AbstractPubToSingle extends AbstractNoHandleSubscribeSingle @Override final void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { // We are now subscribing to the original Publisher chain for the first time, wrap Subscription to preserve the // context. PublisherSource.Subscriber wrappedSubscription = - contextProvider.wrapSubscription(newSubscriber(subscriber), contextMap); - source.delegateSubscribe(wrappedSubscription, contextMap, contextProvider); + contextProvider.wrapSubscription(newSubscriber(subscriber), capturedContext); + source.delegateSubscribe(wrappedSubscription, capturedContext, contextProvider); } abstract PublisherSource.Subscriber newSubscriber(Subscriber original); diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractPublisherGroupBy.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractPublisherGroupBy.java index 14546f3c3e..3abcb5e868 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractPublisherGroupBy.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractPublisherGroupBy.java @@ -18,7 +18,6 @@ import io.servicetalk.concurrent.PublisherSource; import io.servicetalk.concurrent.internal.ConcurrentSubscription; import io.servicetalk.concurrent.internal.DuplicateSubscribeException; -import io.servicetalk.context.api.ContextMap; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -49,7 +48,7 @@ abstract class AbstractPublisherGroupBy extends AbstractNoHandleSubscrib abstract static class AbstractGroupBySubscriber implements Subscriber { private boolean rootCancelled; private final int queueLimit; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; private final Map> groups; private final GroupMulticastSubscriber> target; @@ -57,13 +56,13 @@ abstract static class AbstractGroupBySubscriber implements Subscriber private Subscription subscription; AbstractGroupBySubscriber(final Subscriber> target, final int queueLimit, - final int initialCapacityForGroups, final ContextMap contextMap, + final int initialCapacityForGroups, final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { this.queueLimit = queueLimit; - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; this.target = new GroupMulticastSubscriber<>(this, "root"); - this.target.subscriber(target, false, contextMap, contextProvider); + this.target.subscriber(target, false, capturedContext, contextProvider); groups = new ConcurrentHashMap<>(initialCapacityForGroups); } @@ -98,7 +97,7 @@ final void onNext(Key key, @Nullable T t) { } else { groupSub = new GroupMulticastSubscriber<>(this, key); GroupedPublisher groupedPublisher = new DefaultGroupedPublisher<>(key, groupSub, - contextMap, contextProvider); + capturedContext, contextProvider); final GroupMulticastSubscriber oldVal = groups.put(key, groupSub); assert oldVal == null; // concurrent onNext not allowed, collision not expected. groupSub.onNext(t); // deliver to group first to avoid re-entry creating ordering issues. @@ -165,16 +164,16 @@ public String toString() { } void subscriber(final Subscriber subscriber, final boolean triggerOnSubscribe, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { // The root Subscriber's downstream subscriber is set internally, so no need for atomic operation to filter // duplicates. if (!triggerOnSubscribe) { assert this.subscriber == null && ctxSubscriber == null; this.subscriber = subscriber; - ctxSubscriber = contextProvider.wrapPublisherSubscriber(subscriber, contextMap); + ctxSubscriber = contextProvider.wrapPublisherSubscriber(subscriber, capturedContext); } else if (subscriberStateUpdater.compareAndSet(this, 0, 1)) { this.subscriber = subscriber; - ctxSubscriber = contextProvider.wrapPublisherSubscriber(subscriber, contextMap); + ctxSubscriber = contextProvider.wrapPublisherSubscriber(subscriber, capturedContext); triggerOnSubscribe(); } else { // this.subscriber may be null (we set the subscriber variable after subscriberStateUpdater), @@ -215,14 +214,14 @@ int outstandingDemandLimit() { private static final class DefaultGroupedPublisher extends GroupedPublisher implements PublisherSource { private final GroupMulticastSubscriber groupSink; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; DefaultGroupedPublisher(final Key key, final GroupMulticastSubscriber groupSink, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { super(key); this.groupSink = groupSink; - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; } @@ -233,7 +232,7 @@ public void subscribe(final Subscriber subscriber) { @Override protected void handleSubscribe(Subscriber sub) { - groupSink.subscriber(sub, true, contextMap, contextProvider); + groupSink.subscriber(sub, true, capturedContext, contextProvider); } } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousCompletable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousCompletable.java index 15e2ecc003..ea5a54af7b 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousCompletable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousCompletable.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - /** * Base class for all {@link Completable}s that are created with already realized result and does not generate result * asynchronously. @@ -25,10 +23,10 @@ abstract class AbstractSynchronousCompletable extends AbstractNoHandleSubscribeC @Override final void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { // We need to wrap the Subscriber to save/restore the AsyncContext on each operation or else the AsyncContext // may leak from another thread. - doSubscribe(contextProvider.wrapCompletableSubscriber(subscriber, contextMap)); + doSubscribe(contextProvider.wrapCompletableSubscriber(subscriber, capturedContext)); } /** diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousCompletableOperator.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousCompletableOperator.java index 9bf92ea62f..f261782661 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousCompletableOperator.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousCompletableOperator.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import static java.util.Objects.requireNonNull; /** @@ -41,7 +39,7 @@ abstract class AbstractSynchronousCompletableOperator extends AbstractNoHandleSu @Override final void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { - original.delegateSubscribe(apply(subscriber), contextMap, contextProvider); + CapturedContext capturedContext, AsyncContextProvider contextProvider) { + original.delegateSubscribe(apply(subscriber), capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousPublisher.java index 5b04e58352..c21667268c 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousPublisher.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - /** * Base class for all {@link Publisher}s that are created with already realized values and do not generate values * asynchronously. @@ -27,10 +25,10 @@ abstract class AbstractSynchronousPublisher extends AbstractNoHandleSubscribe @Override final void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { // We need to wrap the Subscriber to save/restore the AsyncContext on each operation or else the AsyncContext // may leak from another thread. - doSubscribe(contextProvider.wrapPublisherSubscriber(subscriber, contextMap)); + doSubscribe(contextProvider.wrapPublisherSubscriber(subscriber, capturedContext)); } /** diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousPublisherOperator.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousPublisherOperator.java index 1a8191b8aa..42c17b3878 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousPublisherOperator.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousPublisherOperator.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import static java.util.Objects.requireNonNull; /** @@ -43,7 +41,7 @@ abstract class AbstractSynchronousPublisherOperator extends AbstractNoHand @Override final void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { - original.delegateSubscribe(apply(subscriber), contextMap, contextProvider); + CapturedContext capturedContext, AsyncContextProvider contextProvider) { + original.delegateSubscribe(apply(subscriber), capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousSingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousSingle.java index 408a5adbfd..a073d8a59d 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousSingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousSingle.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - /** * Base class for all {@link Single}s that are created with already realized result and does not generate result * asynchronously. @@ -27,10 +25,10 @@ abstract class AbstractSynchronousSingle extends AbstractNoHandleSubscribeSin @Override final void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { // We need to wrap the Subscriber to save/restore the AsyncContext on each operation or else the AsyncContext // may leak from another thread. - doSubscribe(contextProvider.wrapSingleSubscriber(subscriber, contextMap)); + doSubscribe(contextProvider.wrapSingleSubscriber(subscriber, capturedContext)); } /** diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousSingleOperator.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousSingleOperator.java index b73a826f2b..e2ca6590e1 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousSingleOperator.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractSynchronousSingleOperator.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import static java.util.Objects.requireNonNull; /** @@ -43,7 +41,7 @@ abstract class AbstractSynchronousSingleOperator extends AbstractNoHandleS @Override final void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { - original.delegateSubscribe(apply(subscriber), contextMap, contextProvider); + CapturedContext capturedContext, AsyncContextProvider contextProvider) { + original.delegateSubscribe(apply(subscriber), capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContext.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContext.java index 52475600a9..efe9ca0ab2 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContext.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContext.java @@ -20,7 +20,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Collections; import java.util.ConcurrentModificationException; +import java.util.List; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; @@ -31,6 +33,7 @@ import java.util.function.BiPredicate; import java.util.function.Consumer; import java.util.function.Function; +import java.util.function.UnaryOperator; import javax.annotation.Nullable; import static io.servicetalk.concurrent.api.AsyncContextExecutorPlugin.EXECUTOR_PLUGIN; @@ -52,6 +55,7 @@ public final class AsyncContext { private static final int STATE_AUTO_ENABLED = 1; private static final int STATE_ENABLED = 2; + private static final AsyncContextProvider DEFAULT_ENABLED_PROVIDER; /** * Note this mechanism is racy. Currently only the {@link #disable()} method is exposed publicly and * {@link #STATE_DISABLED} is a terminal state. Because we favor going to the disabled state we don't have to worry @@ -63,7 +67,21 @@ public final class AsyncContext { * use case for this is a "once at start up" to {@link #disable()} this mechanism completely. This is currently a * best effort mechanism for performance reasons, and we can re-evaluate later if more strict behavior is required. */ - private static AsyncContextProvider provider = DefaultAsyncContextProvider.INSTANCE; + private static AsyncContextProvider provider; + + static { + AsyncContextProvider result = DefaultAsyncContextProvider.INSTANCE; + List> wrappers = asyncProviderWrappers(); + if (!wrappers.isEmpty()) { + ContextCaptureProvider contextCaptureProvider = DefaultAsyncContextProvider.INSTANCE; + for (UnaryOperator wrapper : wrappers) { + contextCaptureProvider = wrapper.apply(result); + } + result = new CustomCaptureAsyncContextProvider(contextCaptureProvider); + } + DEFAULT_ENABLED_PROVIDER = result; + provider = DEFAULT_ENABLED_PROVIDER; + } private AsyncContext() { // no instances @@ -548,7 +566,7 @@ static void autoEnable() { } private static void enable0() { - provider = DefaultAsyncContextProvider.INSTANCE; + provider = DEFAULT_ENABLED_PROVIDER; EXECUTOR_PLUGINS.add(EXECUTOR_PLUGIN); LOGGER.debug("Enabled."); @@ -562,4 +580,8 @@ private static void disable0() { EXECUTOR_PLUGINS.remove(EXECUTOR_PLUGIN); LOGGER.info("Disabled. Features that depend on AsyncContext will stop working."); } + + private static List> asyncProviderWrappers() { + return Collections.emptyList(); + } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java index 4d4b3909ac..af455c6917 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java @@ -36,7 +36,7 @@ /** * Implementation that backs the {@link AsyncContext}. */ -interface AsyncContextProvider { +interface AsyncContextProvider extends ContextCaptureProvider { /** * Get the current context. * @@ -47,42 +47,25 @@ interface AsyncContextProvider { */ ContextMap context(); - /** - * Capture existing context in preparation for an asynchronous thread jump. - * - * Note that this can do more than just package up the ServiceTalk {@link AsyncContext} and could be enhanced or - * wrapped to bundle up additional contexts such as the OpenTelemetry or grpc contexts. - * @return the saved context state that may be restored later. - */ - ContextMap captureContext(); - - /** - * Restore the previously saved {@link ContextMap} to the local state. - * @param contextMap representing the state previously saved via {@link AsyncContextProvider#captureContext()} and - * that is intended to be restored. - * @return a {@link Scope} that must be closed at the end of the attachment. - */ - Scope attachContext(ContextMap contextMap); - /** * Wrap the {@link Cancellable} to ensure it is able to track {@link AsyncContext} correctly. * * @param subscriber The {@link CompletableSource.Subscriber} for which to wrap the corresponding * {@link Cancellable}. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @return The wrapped {@link CompletableSource.Subscriber}. */ - CompletableSource.Subscriber wrapCancellable(CompletableSource.Subscriber subscriber, ContextMap context); + CompletableSource.Subscriber wrapCancellable(CompletableSource.Subscriber subscriber, CapturedContext context); /** * Wrap an {@link CompletableSource.Subscriber} to ensure it is able to track {@link AsyncContext} correctly. * * @param subscriber The subscriber to wrap. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @return The wrapped subscriber. */ CompletableSource.Subscriber wrapCompletableSubscriber(CompletableSource.Subscriber subscriber, - ContextMap context); + CapturedContext context); /** * Wrap an {@link CompletableSource.Subscriber} and any {@link Cancellable}s from @@ -90,32 +73,32 @@ CompletableSource.Subscriber wrapCompletableSubscriber(CompletableSource.Subscri * correctly. * * @param subscriber The subscriber to wrap. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @return The wrapped subscriber. */ CompletableSource.Subscriber wrapCompletableSubscriberAndCancellable(CompletableSource.Subscriber subscriber, - ContextMap context); + CapturedContext context); /** * Wrap the {@link Cancellable} to ensure it is able to track {@link AsyncContext} correctly. * * @param subscriber The {@link SingleSource.Subscriber} for which to wrap the corresponding * {@link Cancellable}. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @param Type of the {@link Single}. * @return The wrapped {@link SingleSource.Subscriber}. */ - SingleSource.Subscriber wrapCancellable(SingleSource.Subscriber subscriber, ContextMap context); + SingleSource.Subscriber wrapCancellable(SingleSource.Subscriber subscriber, CapturedContext context); /** * Wrap an {@link SingleSource.Subscriber} to ensure it is able to track {@link AsyncContext} correctly. * * @param subscriber subscriber to wrap. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @param Type of the {@link Single}. * @return The wrapped subscriber. */ - SingleSource.Subscriber wrapSingleSubscriber(SingleSource.Subscriber subscriber, ContextMap context); + SingleSource.Subscriber wrapSingleSubscriber(SingleSource.Subscriber subscriber, CapturedContext context); /** * Wrap an {@link SingleSource.Subscriber} and any {@link Cancellable}s from @@ -123,46 +106,46 @@ CompletableSource.Subscriber wrapCompletableSubscriberAndCancellable(Completable * correctly. * * @param subscriber subscriber to wrap. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @param Type of the {@link Single}. * @return The wrapped subscriber. */ SingleSource.Subscriber wrapSingleSubscriberAndCancellable(SingleSource.Subscriber subscriber, - ContextMap context); + CapturedContext context); /** * Wrap an {@link Subscription} to ensure it is able to track {@link AsyncContext} correctly. * * @param subscriber The {@link Subscriber} for which to wrap the corresponding * {@link Subscription}. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @return The wrapped {@link Subscriber}. */ PublisherSource.Subscriber wrapSubscription(PublisherSource.Subscriber subscriber, - ContextMap context); + CapturedContext context); /** * Wrap an {@link Subscriber} to ensure it is able to track {@link AsyncContext} correctly. * * @param subscriber The subscriber to wrap. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @param the type of element signaled to the {@link Subscriber}. * @return The wrapped subscriber. */ PublisherSource.Subscriber wrapPublisherSubscriber(PublisherSource.Subscriber subscriber, - ContextMap context); + CapturedContext context); /** * Wrap an {@link Subscriber} and any {@link Subscription}s from {@link Subscriber#onSubscribe(Subscription)} to * ensure it is able to track {@link AsyncContext} correctly. * * @param subscriber The subscriber to wrap. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @param the type of element signaled to the {@link Subscriber}. * @return The wrapped subscriber. */ PublisherSource.Subscriber wrapPublisherSubscriberAndSubscription(PublisherSource.Subscriber subscriber, - ContextMap context); + CapturedContext context); /** * Wrap an {@link Executor} to ensure it is able to track {@link AsyncContext} correctly. @@ -199,72 +182,72 @@ PublisherSource.Subscriber wrapPublisherSubscriberAndSubscription(Publish * Wrap a {@link CompletableFuture} so that {@link AsyncContext} is preserved from listener methods. * * @param future The future to wrap. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @param The type of data for {@link CompletableFuture}. * @return the wrapped {@link CompletableFuture}. */ - CompletableFuture wrapCompletableFuture(CompletableFuture future, ContextMap context); + CompletableFuture wrapCompletableFuture(CompletableFuture future, CapturedContext context); /** * Wrap a {@link Runnable} to ensure it is able to track {@link AsyncContext} correctly. * * @param runnable The runnable to wrap. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @return The wrapped {@link Runnable}. */ - Runnable wrapRunnable(Runnable runnable, ContextMap context); + Runnable wrapRunnable(Runnable runnable, CapturedContext context); /** * Wrap a {@link Callable} to ensure it is able to track {@link AsyncContext} correctly. * * @param callable The callable to wrap. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @param The type of data returned by {@code callable}. * @return The wrapped {@link Callable}. */ - Callable wrapCallable(Callable callable, ContextMap context); + Callable wrapCallable(Callable callable, CapturedContext context); /** * Wrap a {@link Consumer} to ensure it is able to track {@link AsyncContext} correctly. * * @param consumer The consumer to wrap. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @param The type of data consumed by {@code consumer}. * @return The wrapped {@link Consumer}. */ - Consumer wrapConsumer(Consumer consumer, ContextMap context); + Consumer wrapConsumer(Consumer consumer, CapturedContext context); /** * Wrap a {@link Function} to ensure it is able to track {@link AsyncContext} correctly. * * @param func The function to wrap. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @param The type of data consumed by {@code func}. * @param The type of data returned by {@code func}. * @return The wrapped {@link Function}. */ - Function wrapFunction(Function func, ContextMap context); + Function wrapFunction(Function func, CapturedContext context); /** * Wrap a {@link BiFunction} to ensure it is able to track {@link AsyncContext} correctly. * * @param consumer The consumer to wrap. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @param The type of data consumed by {@code func}. * @param The type of data consumed by {@code func}. * @return The wrapped {@link BiConsumer}. */ - BiConsumer wrapBiConsumer(BiConsumer consumer, ContextMap context); + BiConsumer wrapBiConsumer(BiConsumer consumer, CapturedContext context); /** * Wrap a {@link BiFunction} to ensure it is able to track {@link AsyncContext} correctly. * * @param func The function to wrap. - * @param context The current {@link ContextMap}. + * @param context The current {@link CapturedContext}. * @param The type of data consumed by {@code func}. * @param The type of data consumed by {@code func}. * @param The type of data returned by {@code func}. * @return The wrapped {@link BiFunction}. */ - BiFunction wrapBiFunction(BiFunction func, ContextMap context); + BiFunction wrapBiFunction(BiFunction func, CapturedContext context); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CacheSingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CacheSingle.java index 6597f734f7..7a9f732756 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CacheSingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CacheSingle.java @@ -20,7 +20,6 @@ import io.servicetalk.concurrent.internal.DelayedCancellable; import io.servicetalk.concurrent.internal.SubscriberUtils; import io.servicetalk.concurrent.internal.TerminalNotification; -import io.servicetalk.context.api.ContextMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,9 +67,9 @@ final class CacheSingle extends AbstractNoHandleSubscribeSingle { } @Override - void handleSubscribe(Subscriber subscriber, ContextMap contextMap, + void handleSubscribe(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { - state.addSubscriber(subscriber, contextMap, contextProvider); + state.addSubscriber(subscriber, capturedContext, contextProvider); } private final class State implements Subscriber { @@ -79,7 +78,7 @@ private final class State implements Subscriber { volatile int subscribeCount; private final DelayedCancellable delayedCancellable = new DelayedCancellable(); - void addSubscriber(Subscriber subscriber, ContextMap contextMap, + void addSubscriber(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { final int sCount = subscribeCountUpdater.incrementAndGet(this); final ConcurrentOnSubscribeSubscriber multiSubscriber = @@ -107,7 +106,7 @@ void addSubscriber(Subscriber subscriber, ContextMap contextMap, if (sCount == minSubscribers) { // This operator has special behavior where it chooses to use the AsyncContext and signal // offloader from the last subscribe operation. - original.delegateSubscribe(this, contextMap, contextProvider); + original.delegateSubscribe(this, capturedContext, contextProvider); } break; } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java new file mode 100644 index 0000000000..a1f2d43487 --- /dev/null +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java @@ -0,0 +1,30 @@ +/* + * Copyright © 2025 Apple Inc. and the ServiceTalk project authors + * + * Licensed 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 io.servicetalk.concurrent.api; + +/** + * An interface representing the restoration of the thread-local like context that can be restored later + * during an async operation. + */ +interface CapturedContext { + + /** + * Restore the thread-local like context. + * @return a {@link Scope} that will revert the restoration and return the thread-local like state to the state + * that it had before restoring this context. + */ + Scope restoreContext(); +} diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Completable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Completable.java index 0786ced0c2..b8d31a9534 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Completable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Completable.java @@ -1728,9 +1728,9 @@ public final Future toFuture() { * @param provider The {@link AsyncContextProvider} which is the source of the map * @return {@link ContextMap} for this subscribe operation. */ - ContextMap contextForSubscribe(AsyncContextProvider provider) { + CapturedContext contextForSubscribe(AsyncContextProvider provider) { // the default behavior is to copy the map. Some operators may want to use shared map - return provider.captureContext().copy(); + return provider.captureContextCopy(); } /** @@ -1742,15 +1742,15 @@ ContextMap contextForSubscribe(AsyncContextProvider provider) { protected final void subscribeInternal(Subscriber subscriber) { requireNonNull(subscriber); AsyncContextProvider contextProvider = AsyncContext.provider(); - ContextMap contextMap = contextForSubscribe(contextProvider); - Subscriber wrapped = contextProvider.wrapCancellable(subscriber, contextMap); - if (contextProvider.context() == contextMap) { + CapturedContext capturedContext = contextForSubscribe(contextProvider); + Subscriber wrapped = contextProvider.wrapCancellable(subscriber, capturedContext); + if (contextProvider.context() == capturedContext) { // No need to wrap as we are sharing the AsyncContext - handleSubscribe(wrapped, contextMap, contextProvider); + handleSubscribe(wrapped, capturedContext, contextProvider); } else { // Ensure that AsyncContext used for handleSubscribe() is the contextMap for the subscribe() - try (Scope unused = contextProvider.attachContext(contextMap)) { - handleSubscribe(wrapped, contextMap, contextProvider); + try (Scope unused = capturedContext.restoreContext()) { + handleSubscribe(wrapped, capturedContext, contextProvider); } } } @@ -2264,12 +2264,12 @@ public static Completable anyOf(final Iterable completables) { * source. * * @param subscriber the subscriber. - * @param contextMap the {@link ContextMap} to use for this {@link Subscriber}. + * @param capturedContext the {@link ContextMap} to use for this {@link Subscriber}. * @param contextProvider the {@link AsyncContextProvider} used to wrap any objects to preserve {@link ContextMap}. */ final void delegateSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { - handleSubscribe(subscriber, contextMap, contextProvider); + CapturedContext capturedContext, AsyncContextProvider contextProvider) { + handleSubscribe(subscriber, capturedContext, contextProvider); } /** @@ -2277,12 +2277,12 @@ final void delegateSubscribe(Subscriber subscriber, *

* Operators that do not wish to wrap the passed {@link Subscriber} can override this method and omit the wrapping. * @param subscriber the subscriber. - * @param contextMap the {@link ContextMap} to use for this {@link Subscriber}. + * @param capturedContext the {@link ContextMap} to use for this {@link Subscriber}. * @param contextProvider the {@link AsyncContextProvider} used to wrap any objects to preserve {@link ContextMap}. */ - void handleSubscribe(Subscriber subscriber, ContextMap contextMap, AsyncContextProvider contextProvider) { + void handleSubscribe(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { try { - Subscriber wrapped = contextProvider.wrapCompletableSubscriber(subscriber, contextMap); + Subscriber wrapped = contextProvider.wrapCompletableSubscriber(subscriber, capturedContext); handleSubscribe(wrapped); } catch (Throwable t) { LOGGER.warn("Unexpected exception from subscribe(), assuming no interaction with the Subscriber.", t); diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithCompletable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithCompletable.java index 46bb0a575d..53eb8043bc 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithCompletable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithCompletable.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.internal.SequentialCancellable; -import io.servicetalk.context.api.ContextMap; import javax.annotation.Nullable; @@ -36,10 +35,10 @@ final class CompletableConcatWithCompletable extends AbstractNoHandleSubscribeCo } @Override - protected void handleSubscribe(Subscriber subscriber, ContextMap contextMap, + protected void handleSubscribe(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { - Subscriber wrappedSubscriber = contextProvider.wrapCompletableSubscriber(subscriber, contextMap); - original.delegateSubscribe(new ConcatWithSubscriber(wrappedSubscriber, next), contextMap, contextProvider); + Subscriber wrappedSubscriber = contextProvider.wrapCompletableSubscriber(subscriber, capturedContext); + original.delegateSubscribe(new ConcatWithSubscriber(wrappedSubscriber, next), capturedContext, contextProvider); } private static final class ConcatWithSubscriber implements Subscriber { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithCompletables.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithCompletables.java index 380761fa71..bcd1a5b821 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithCompletables.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithCompletables.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.internal.SequentialCancellable; -import io.servicetalk.context.api.ContextMap; import javax.annotation.Nullable; @@ -37,10 +36,10 @@ final class CompletableConcatWithCompletables extends AbstractNoHandleSubscribeC } @Override - protected void handleSubscribe(Subscriber subscriber, ContextMap contextMap, + protected void handleSubscribe(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { - Subscriber wrappedSubscriber = contextProvider.wrapCompletableSubscriber(subscriber, contextMap); - original.delegateSubscribe(new ConcatWithSubscriber(wrappedSubscriber, nexts), contextMap, contextProvider); + Subscriber wrappedSubscriber = contextProvider.wrapCompletableSubscriber(subscriber, capturedContext); + original.delegateSubscribe(new ConcatWithSubscriber(wrappedSubscriber, nexts), capturedContext, contextProvider); } private static final class ConcatWithSubscriber implements Subscriber { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithPublisher.java index bfcd18386e..de1eac13ab 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithPublisher.java @@ -18,7 +18,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.CompletableSource; import io.servicetalk.concurrent.PublisherSource; -import io.servicetalk.context.api.ContextMap; import javax.annotation.Nullable; @@ -33,8 +32,8 @@ final class CompletableConcatWithPublisher extends AbstractNoHandleSubscribeP @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { - original.delegateSubscribe(new ConcatSubscriber<>(subscriber, next), contextMap, contextProvider); + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + original.delegateSubscribe(new ConcatSubscriber<>(subscriber, next), capturedContext, contextProvider); } private static final class ConcatSubscriber extends DelayedCancellableThenSubscription diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithSingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithSingle.java index c43f90fcdb..54b922c7e9 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithSingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithSingle.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import javax.annotation.Nullable; import static java.util.Objects.requireNonNull; @@ -37,8 +35,8 @@ final class CompletableConcatWithSingle extends AbstractCompletableAndSingleC @Override void delegateSubscribeToOriginal(final Subscriber offloadSubscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { - original.delegateSubscribe(new ConcatWithSubscriber<>(offloadSubscriber, next), contextMap, contextProvider); + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + original.delegateSubscribe(new ConcatWithSubscriber<>(offloadSubscriber, next), capturedContext, contextProvider); } private static final class ConcatWithSubscriber extends AbstractConcatWithSubscriber { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableMergeWithPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableMergeWithPublisher.java index 596967ca1b..2dc991c6cb 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableMergeWithPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableMergeWithPublisher.java @@ -20,7 +20,6 @@ import io.servicetalk.concurrent.internal.ConcurrentSubscription; import io.servicetalk.concurrent.internal.DelayedCancellable; import io.servicetalk.concurrent.internal.DelayedSubscription; -import io.servicetalk.context.api.ContextMap; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; @@ -47,13 +46,13 @@ final class CompletableMergeWithPublisher extends AbstractNoHandleSubscribePu @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { if (delayError) { - new MergerDelayError<>(subscriber, contextMap, contextProvider) - .merge(original, mergeWith, contextMap, contextProvider); + new MergerDelayError<>(subscriber, capturedContext, contextProvider) + .merge(original, mergeWith, capturedContext, contextProvider); } else { - new Merger<>(subscriber, contextMap, contextProvider) - .merge(original, mergeWith, contextMap, contextProvider); + new Merger<>(subscriber, capturedContext, contextProvider) + .merge(original, mergeWith, capturedContext, contextProvider); } } @@ -67,18 +66,18 @@ private static final class MergerDelayError implements Subscriber { @Nullable private volatile TerminalSignal terminal; - MergerDelayError(Subscriber subscriber, ContextMap contextMap, + MergerDelayError(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { // This is used only to deliver signals that originate from the mergeWith Publisher. - this.wrappedSubscriber = contextProvider.wrapPublisherSubscriber(subscriber, contextMap); + this.wrappedSubscriber = contextProvider.wrapPublisherSubscriber(subscriber, capturedContext); completableSubscriber = new CompletableSubscriber(); } void merge(Completable original, Publisher mergeWith, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { wrappedSubscriber.onSubscribe( new MergedCancellableWithSubscription(subscription, completableSubscriber)); - original.delegateSubscribe(completableSubscriber, contextMap, contextProvider); + original.delegateSubscribe(completableSubscriber, capturedContext, contextProvider); mergeWith.subscribeInternal(this); } @@ -185,16 +184,16 @@ private static final class Merger implements Subscriber { private Throwable completableError; private volatile int state; - Merger(Subscriber subscriber, ContextMap contextMap, AsyncContextProvider contextProvider) { - this.wrappedSubscriber = contextProvider.wrapPublisherSubscriber(subscriber, contextMap); + Merger(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { + this.wrappedSubscriber = contextProvider.wrapPublisherSubscriber(subscriber, capturedContext); completableSubscriber = new CompletableSubscriber(); } void merge(Completable original, Publisher mergeWith, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { wrappedSubscriber.onSubscribe( new MergedCancellableWithSubscription(subscription, completableSubscriber)); - original.delegateSubscribe(completableSubscriber, contextMap, contextProvider); + original.delegateSubscribe(completableSubscriber, capturedContext, contextProvider); mergeWith.subscribeInternal(this); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java index 823ab38026..a812dca3b2 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java @@ -29,16 +29,24 @@ final class CompletableSetContextOnSubscribe extends AbstractNoHandleSubscribeCo } @Override - ContextMap contextForSubscribe(AsyncContextProvider provider) { - return context; + CapturedContext contextForSubscribe(AsyncContextProvider provider) { + CapturedContext parentContext = super.contextForSubscribe(provider); + return () -> { + Scope outer = parentContext.restoreContext(); + Scope inner = ContextMapThreadLocal.attachContext(context); + return () -> { + inner.close(); + outer.close(); + }; + }; } @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { // This operator currently only targets the subscribe method. Given this limitation if we try to change the // ContextMap now it is possible that operators downstream in the subscribe call stack may have modified // the ContextMap and we don't want to discard those changes by using a different ContextMap. - original.handleSubscribe(subscriber, contextMap, contextProvider); + original.handleSubscribe(subscriber, capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableShareContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableShareContextOnSubscribe.java index 2f8711f3fd..84dd669fbb 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableShareContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableShareContextOnSubscribe.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - final class CompletableShareContextOnSubscribe extends AbstractNoHandleSubscribeCompletable { private final Completable original; @@ -25,16 +23,16 @@ final class CompletableShareContextOnSubscribe extends AbstractNoHandleSubscribe } @Override - ContextMap contextForSubscribe(AsyncContextProvider provider) { + CapturedContext contextForSubscribe(AsyncContextProvider provider) { return provider.captureContext(); } @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { // This operator currently only targets the subscribe method. Given this limitation if we try to change the // ContextMap now it is possible that operators downstream in the subscribe call stack may have modified // the ContextMap and we don't want to discard those changes by using a different ContextMap. - original.handleSubscribe(subscriber, contextMap, contextProvider); + original.handleSubscribe(subscriber, capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableToPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableToPublisher.java index 196635591b..6f145befc5 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableToPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableToPublisher.java @@ -18,7 +18,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.CompletableSource; import io.servicetalk.concurrent.internal.SequentialCancellable; -import io.servicetalk.context.api.ContextMap; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; @@ -40,9 +39,9 @@ final class CompletableToPublisher extends AbstractNoHandleSubscribePublisher @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { - original.delegateSubscribe(new ConversionSubscriber<>(subscriber, contextMap, contextProvider), - contextMap, contextProvider); + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + original.delegateSubscribe(new ConversionSubscriber<>(subscriber, capturedContext, contextProvider), + capturedContext, contextProvider); } private static final class ConversionSubscriber extends SequentialCancellable @@ -51,15 +50,15 @@ private static final class ConversionSubscriber extends SequentialCancellable private static final AtomicIntegerFieldUpdater terminatedUpdater = newUpdater(ConversionSubscriber.class, "terminated"); private final Subscriber subscriber; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; private volatile int terminated; private ConversionSubscriber(Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { this.subscriber = subscriber; - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; } @@ -89,7 +88,7 @@ public void request(long n) { // We have not wrapped the Subscriber as we generally emit to the Subscriber from the Completable // Subscriber methods which are correctly wrapped. This is the only case where we invoke the // Subscriber directly, hence we explicitly wrap it. - Subscriber wrapped = wrapWithDummyOnSubscribe(subscriber, contextMap, contextProvider); + Subscriber wrapped = wrapWithDummyOnSubscribe(subscriber, capturedContext, contextProvider); try { cancel(); } catch (Throwable t) { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableToSingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableToSingle.java index 47e3456e78..dec97e1122 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableToSingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableToSingle.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.CompletableSource; -import io.servicetalk.context.api.ContextMap; final class CompletableToSingle extends AbstractNoHandleSubscribeSingle { private final Completable original; @@ -28,7 +27,7 @@ final class CompletableToSingle extends AbstractNoHandleSubscribeSingle { @Override protected void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { // We are not modifying the Cancellable between sources, so we do not need to take care of offloading between // the sources (in this operator). If the Cancellable is configured to be offloaded, it will be done when the // resulting Completable is subscribed. Since, it is the same source, just viewed as a Single, there is no @@ -48,6 +47,6 @@ public void onComplete() { public void onError(Throwable t) { subscriber.onError(t); } - }, contextMap, contextProvider); + }, capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextAwareExecutorUtils.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextAwareExecutorUtils.java index e422174b84..66501605f8 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextAwareExecutorUtils.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextAwareExecutorUtils.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -30,9 +28,9 @@ private ContextAwareExecutorUtils() { static Collection> wrap(Collection> tasks) { List> wrappedTasks = new ArrayList<>(tasks.size()); - ContextMap contextMap = AsyncContext.provider().captureContext(); + CapturedContext capturedContext = AsyncContext.provider().captureContext(); for (Callable task : tasks) { - wrappedTasks.add(new ContextPreservingCallable<>(task, contextMap)); + wrappedTasks.add(new ContextPreservingCallable<>(task, capturedContext)); } return wrappedTasks; } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java new file mode 100644 index 0000000000..bcf2b4a862 --- /dev/null +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java @@ -0,0 +1,15 @@ +package io.servicetalk.concurrent.api; + +interface ContextCaptureProvider { + + /** + * Save existing context in preparation for an asynchronous thread jump. + * + * Note that this can do more than just package up the ServiceTalk {@link AsyncContext} and could be enhanced or + * wrapped to bundle up additional contexts such as the OpenTelemetry or grpc contexts. + * @return the saved context state that may be restored later. + */ + CapturedContext captureContext(); + + CapturedContext captureContextCopy(); +} diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextMapThreadLocal.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextMapThreadLocal.java new file mode 100644 index 0000000000..23bc9a6d4e --- /dev/null +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextMapThreadLocal.java @@ -0,0 +1,80 @@ +package io.servicetalk.concurrent.api; + +import io.servicetalk.context.api.ContextMap; +import io.servicetalk.context.api.ContextMapHolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import static java.lang.ThreadLocal.withInitial; + +final class ContextMapThreadLocal { + + private static final Logger LOGGER = LoggerFactory.getLogger(ContextMapThreadLocal.class); + + protected static final ThreadLocal CONTEXT_THREAD_LOCAL = + withInitial(ContextMapThreadLocal::newContextMap); + + @Nonnull + static ContextMap context() { + final Thread t = Thread.currentThread(); + if (t instanceof ContextMapHolder) { + final ContextMapHolder contextMapHolder = (ContextMapHolder) t; + ContextMap map = contextMapHolder.context(); + if (map == null) { + map = newContextMap(); + contextMapHolder.context(map); + } + return map; + } else { + return CONTEXT_THREAD_LOCAL.get(); + } + } + + static Scope attachContext(ContextMap contextMap) { + final Thread currentThread = Thread.currentThread(); + if (currentThread instanceof ContextMapHolder) { + final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; + ContextMap prev = asyncContextMapHolder.context(); + asyncContextMapHolder.context(contextMap); + return prev == null ? Scope.NOOP : () -> detachContext(contextMap, prev); + } else { + return slowPathSetContext(contextMap); + } + } + + private static Scope slowPathSetContext(ContextMap contextMap) { + ContextMap prev = CONTEXT_THREAD_LOCAL.get(); + CONTEXT_THREAD_LOCAL.set(contextMap); + return () -> detachContext(contextMap, prev); + } + + private static void detachContext(ContextMap expectedContext, ContextMap toRestore) { + final Thread currentThread = Thread.currentThread(); + if (currentThread instanceof ContextMapHolder) { + final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; + ContextMap current = asyncContextMapHolder.context(); + if (current != expectedContext) { + LOGGER.warn("Current context didn't match the expected context. current: {}, expected: {}", + current, expectedContext); + } + asyncContextMapHolder.context(toRestore); + } else { + slowPathDetachContext(expectedContext, toRestore); + } + } + + private static void slowPathDetachContext(ContextMap expectedContext, ContextMap toRestore) { + ContextMap current = CONTEXT_THREAD_LOCAL.get(); + if (current != expectedContext) { + LOGGER.warn("Current context didn't match the expected context. current: {}, expected: {}", + current, expectedContext); + } + CONTEXT_THREAD_LOCAL.set(toRestore); + } + + private static ContextMap newContextMap() { + return new CopyOnWriteContextMap(); + } +} diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiConsumer.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiConsumer.java index 76181e66dd..f6c0be9a9f 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiConsumer.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiConsumer.java @@ -15,24 +15,23 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import java.util.function.BiConsumer; import static java.util.Objects.requireNonNull; final class ContextPreservingBiConsumer implements BiConsumer { - private final ContextMap saved; + + private final CapturedContext capturedContext; private final BiConsumer delegate; - ContextPreservingBiConsumer(BiConsumer delegate, ContextMap contextMap) { - this.saved = requireNonNull(contextMap); + ContextPreservingBiConsumer(BiConsumer delegate, CapturedContext capturedContext) { + this.capturedContext = requireNonNull(capturedContext); this.delegate = requireNonNull(delegate); } @Override public void accept(T t, U u) { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = capturedContext.restoreContext()) { delegate.accept(t, u); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java index 0e1d7e7dbe..b24555f831 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java @@ -15,24 +15,22 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import java.util.function.BiFunction; import static java.util.Objects.requireNonNull; final class ContextPreservingBiFunction implements BiFunction { - private final ContextMap saved; + private final CapturedContext saved; private final BiFunction delegate; - ContextPreservingBiFunction(BiFunction delegate, ContextMap contextMap) { - this.saved = requireNonNull(contextMap); + ContextPreservingBiFunction(BiFunction delegate, CapturedContext capturedContext) { + this.saved = requireNonNull(capturedContext); this.delegate = requireNonNull(delegate); } @Override public V apply(T t, U u) { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { return delegate.apply(t, u); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCallable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCallable.java index 4cb3df9232..79fc4994ce 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCallable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCallable.java @@ -15,28 +15,26 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import java.util.concurrent.Callable; import static java.util.Objects.requireNonNull; final class ContextPreservingCallable implements Callable { - private final ContextMap saved; + private final CapturedContext saved; private final Callable delegate; ContextPreservingCallable(Callable delegate) { this(delegate, AsyncContext.provider().captureContext()); } - ContextPreservingCallable(Callable delegate, ContextMap current) { + ContextPreservingCallable(Callable delegate, CapturedContext current) { this.saved = requireNonNull(current); this.delegate = requireNonNull(delegate); } @Override public V call() throws Exception { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { return delegate.call(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellable.java index a16dbbb025..77722f96c4 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellable.java @@ -16,20 +16,19 @@ package io.servicetalk.concurrent.api; import io.servicetalk.concurrent.Cancellable; -import io.servicetalk.context.api.ContextMap; import static java.util.Objects.requireNonNull; final class ContextPreservingCancellable implements Cancellable { - private final ContextMap saved; + private final CapturedContext saved; private final Cancellable delegate; - private ContextPreservingCancellable(Cancellable delegate, ContextMap current) { + private ContextPreservingCancellable(Cancellable delegate, CapturedContext current) { this.saved = requireNonNull(current); this.delegate = requireNonNull(delegate); } - static Cancellable wrap(Cancellable delegate, ContextMap current) { + static Cancellable wrap(Cancellable delegate, CapturedContext current) { // The double wrapping can be observed when folks manually create a Single/Completable and directly call the // onSubscribe method. return delegate instanceof ContextPreservingCancellable && @@ -39,7 +38,7 @@ static Cancellable wrap(Cancellable delegate, ContextMap current) { @Override public void cancel() { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { delegate.cancel(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableCompletableSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableCompletableSubscriber.java index dd7d09f2da..77789f37d1 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableCompletableSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableCompletableSubscriber.java @@ -16,16 +16,15 @@ package io.servicetalk.concurrent.api; import io.servicetalk.concurrent.Cancellable; -import io.servicetalk.context.api.ContextMap; import static io.servicetalk.concurrent.CompletableSource.Subscriber; import static java.util.Objects.requireNonNull; final class ContextPreservingCancellableCompletableSubscriber implements Subscriber { - final ContextMap saved; + final CapturedContext saved; final Subscriber subscriber; - ContextPreservingCancellableCompletableSubscriber(Subscriber subscriber, ContextMap current) { + ContextPreservingCancellableCompletableSubscriber(Subscriber subscriber, CapturedContext current) { this.subscriber = requireNonNull(subscriber); this.saved = requireNonNull(current); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableSingleSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableSingleSubscriber.java index aa79a249b7..200fb4e518 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableSingleSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableSingleSubscriber.java @@ -18,17 +18,16 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.SingleSource; import io.servicetalk.concurrent.SingleSource.Subscriber; -import io.servicetalk.context.api.ContextMap; import javax.annotation.Nullable; import static java.util.Objects.requireNonNull; final class ContextPreservingCancellableSingleSubscriber implements Subscriber { - final ContextMap saved; + final CapturedContext saved; final SingleSource.Subscriber subscriber; - ContextPreservingCancellableSingleSubscriber(Subscriber subscriber, ContextMap current) { + ContextPreservingCancellableSingleSubscriber(Subscriber subscriber, CapturedContext current) { this.subscriber = requireNonNull(subscriber); this.saved = requireNonNull(current); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableFuture.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableFuture.java index 1c50631e0d..b9cf40fd1d 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableFuture.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableFuture.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; @@ -34,16 +32,17 @@ final class ContextPreservingCompletableFuture extends CompletableFuture { private final CompletableFuture delegate; - private final ContextMap saved; + private final CapturedContext saved; - private ContextPreservingCompletableFuture(CompletableFuture delegate, ContextMap current) { + private ContextPreservingCompletableFuture(CompletableFuture delegate, CapturedContext current) { this.delegate = requireNonNull(delegate); this.saved = requireNonNull(current); } static ContextPreservingCompletableFuture newContextPreservingFuture(CompletableFuture original, - ContextMap contextMap) { - ContextPreservingCompletableFuture future = new ContextPreservingCompletableFuture<>(original, contextMap); + CapturedContext capturedContext) { + ContextPreservingCompletableFuture future = + new ContextPreservingCompletableFuture<>(original, capturedContext); cascadeTermination(original, future); return future; } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriber.java index f7c6676168..fa24a3b4b3 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriber.java @@ -17,15 +17,14 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.CompletableSource.Subscriber; -import io.servicetalk.context.api.ContextMap; import static java.util.Objects.requireNonNull; class ContextPreservingCompletableSubscriber implements Subscriber { - final ContextMap saved; + final CapturedContext saved; final Subscriber subscriber; - ContextPreservingCompletableSubscriber(Subscriber subscriber, ContextMap current) { + ContextPreservingCompletableSubscriber(Subscriber subscriber, CapturedContext current) { this.subscriber = requireNonNull(subscriber); this.saved = requireNonNull(current); } @@ -36,21 +35,21 @@ void invokeOnSubscribe(Cancellable cancellable) { @Override public final void onSubscribe(final Cancellable cancellable) { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { invokeOnSubscribe(cancellable); } } @Override public final void onComplete() { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { subscriber.onComplete(); } } @Override public final void onError(Throwable t) { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { subscriber.onError(t); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriberAndCancellable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriberAndCancellable.java index 0626fe9790..3476c4edde 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriberAndCancellable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriberAndCancellable.java @@ -17,10 +17,9 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.CompletableSource.Subscriber; -import io.servicetalk.context.api.ContextMap; final class ContextPreservingCompletableSubscriberAndCancellable extends ContextPreservingCompletableSubscriber { - ContextPreservingCompletableSubscriberAndCancellable(Subscriber subscriber, ContextMap current) { + ContextPreservingCompletableSubscriberAndCancellable(Subscriber subscriber, CapturedContext current) { super(subscriber, current); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingConsumer.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingConsumer.java index c56eb9423f..4ad5703be8 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingConsumer.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingConsumer.java @@ -15,24 +15,22 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import java.util.function.Consumer; import static java.util.Objects.requireNonNull; final class ContextPreservingConsumer implements Consumer { - private final ContextMap saved; + private final CapturedContext saved; private final Consumer delegate; - ContextPreservingConsumer(Consumer delegate, ContextMap current) { + ContextPreservingConsumer(Consumer delegate, CapturedContext current) { this.saved = requireNonNull(current); this.delegate = requireNonNull(delegate); } @Override public void accept(T t) { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { delegate.accept(t); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingFunction.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingFunction.java index dc8bb4bf5f..9ed8bdf9e8 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingFunction.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingFunction.java @@ -15,24 +15,22 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import java.util.function.Function; import static java.util.Objects.requireNonNull; final class ContextPreservingFunction implements Function { - private final ContextMap saved; + private final CapturedContext saved; private final Function delegate; - ContextPreservingFunction(Function delegate, ContextMap contextMap) { - this.saved = requireNonNull(contextMap); + ContextPreservingFunction(Function delegate, CapturedContext capturedContext) { + this.saved = requireNonNull(capturedContext); this.delegate = requireNonNull(delegate); } @Override public U apply(T t) { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { return delegate.apply(t); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingRunnable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingRunnable.java index 9d13254dd1..d8ba8b2027 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingRunnable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingRunnable.java @@ -15,26 +15,24 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import static java.util.Objects.requireNonNull; final class ContextPreservingRunnable implements Runnable { - private final ContextMap saved; + private final CapturedContext saved; private final Runnable delegate; ContextPreservingRunnable(Runnable delegate) { this(delegate, AsyncContext.provider().captureContext()); } - ContextPreservingRunnable(Runnable delegate, ContextMap current) { + ContextPreservingRunnable(Runnable delegate, CapturedContext current) { this.saved = requireNonNull(current); this.delegate = requireNonNull(delegate); } @Override public void run() { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { delegate.run(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriber.java index 57c89d2a04..9ab5b07610 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriber.java @@ -18,17 +18,16 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.SingleSource; import io.servicetalk.concurrent.SingleSource.Subscriber; -import io.servicetalk.context.api.ContextMap; import javax.annotation.Nullable; import static java.util.Objects.requireNonNull; class ContextPreservingSingleSubscriber implements Subscriber { - final ContextMap saved; + final CapturedContext saved; final SingleSource.Subscriber subscriber; - ContextPreservingSingleSubscriber(Subscriber subscriber, ContextMap current) { + ContextPreservingSingleSubscriber(Subscriber subscriber, CapturedContext current) { this.subscriber = requireNonNull(subscriber); this.saved = requireNonNull(current); } @@ -39,21 +38,21 @@ void invokeOnSubscribe(Cancellable cancellable) { @Override public final void onSubscribe(Cancellable cancellable) { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { invokeOnSubscribe(cancellable); } } @Override public final void onSuccess(@Nullable T result) { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { subscriber.onSuccess(result); } } @Override public final void onError(Throwable t) { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { subscriber.onError(t); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriberAndCancellable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriberAndCancellable.java index 288ba5aca5..c2d3c00e9e 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriberAndCancellable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriberAndCancellable.java @@ -17,10 +17,9 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.SingleSource.Subscriber; -import io.servicetalk.context.api.ContextMap; final class ContextPreservingSingleSubscriberAndCancellable extends ContextPreservingSingleSubscriber { - ContextPreservingSingleSubscriberAndCancellable(Subscriber subscriber, ContextMap current) { + ContextPreservingSingleSubscriberAndCancellable(Subscriber subscriber, CapturedContext current) { super(subscriber, current); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriber.java index 70c6030b22..2f39570b62 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriber.java @@ -17,15 +17,14 @@ import io.servicetalk.concurrent.PublisherSource.Subscriber; import io.servicetalk.concurrent.PublisherSource.Subscription; -import io.servicetalk.context.api.ContextMap; import static java.util.Objects.requireNonNull; class ContextPreservingSubscriber implements Subscriber { - final ContextMap saved; + final CapturedContext saved; final Subscriber subscriber; - ContextPreservingSubscriber(Subscriber subscriber, ContextMap current) { + ContextPreservingSubscriber(Subscriber subscriber, CapturedContext current) { this.subscriber = requireNonNull(subscriber); this.saved = requireNonNull(current); } @@ -36,28 +35,28 @@ void invokeOnSubscribe(Subscription s) { @Override public final void onSubscribe(Subscription s) { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { invokeOnSubscribe(s); } } @Override public final void onNext(T t) { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { subscriber.onNext(t); } } @Override public final void onError(Throwable t) { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { subscriber.onError(t); } } @Override public final void onComplete() { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { subscriber.onComplete(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriberAndSubscription.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriberAndSubscription.java index b5ffff3315..c86e57901c 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriberAndSubscription.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriberAndSubscription.java @@ -17,10 +17,9 @@ import io.servicetalk.concurrent.PublisherSource; import io.servicetalk.concurrent.PublisherSource.Subscriber; -import io.servicetalk.context.api.ContextMap; final class ContextPreservingSubscriberAndSubscription extends ContextPreservingSubscriber { - ContextPreservingSubscriberAndSubscription(Subscriber subscriber, ContextMap current) { + ContextPreservingSubscriberAndSubscription(Subscriber subscriber, CapturedContext current) { super(subscriber, current); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscription.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscription.java index 55d2c3bb61..e26232cccb 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscription.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscription.java @@ -16,20 +16,19 @@ package io.servicetalk.concurrent.api; import io.servicetalk.concurrent.PublisherSource.Subscription; -import io.servicetalk.context.api.ContextMap; import static java.util.Objects.requireNonNull; final class ContextPreservingSubscription implements Subscription { - private final ContextMap saved; + private final CapturedContext saved; private final Subscription subscription; - private ContextPreservingSubscription(Subscription subscription, ContextMap current) { + private ContextPreservingSubscription(Subscription subscription, CapturedContext current) { this.subscription = requireNonNull(subscription); this.saved = requireNonNull(current); } - static Subscription wrap(Subscription subscription, ContextMap current) { + static Subscription wrap(Subscription subscription, CapturedContext current) { return subscription instanceof ContextPreservingSubscription && ((ContextPreservingSubscription) subscription).saved == current ? subscription : new ContextPreservingSubscription(subscription, current); @@ -37,14 +36,14 @@ static Subscription wrap(Subscription subscription, ContextMap current) { @Override public void request(long l) { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { subscription.request(l); } } @Override public void cancel() { - try (Scope ignored = AsyncContext.provider().attachContext(saved)) { + try (Scope ignored = saved.restoreContext()) { subscription.cancel(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriptionSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriptionSubscriber.java index 69bf8d01f8..099f899561 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriptionSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriptionSubscriber.java @@ -17,15 +17,14 @@ import io.servicetalk.concurrent.PublisherSource.Subscriber; import io.servicetalk.concurrent.PublisherSource.Subscription; -import io.servicetalk.context.api.ContextMap; import static java.util.Objects.requireNonNull; final class ContextPreservingSubscriptionSubscriber implements Subscriber { - final ContextMap saved; + final CapturedContext saved; final Subscriber subscriber; - ContextPreservingSubscriptionSubscriber(Subscriber subscriber, ContextMap current) { + ContextPreservingSubscriptionSubscriber(Subscriber subscriber, CapturedContext current) { this.subscriber = requireNonNull(subscriber); this.saved = requireNonNull(current); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java new file mode 100644 index 0000000000..cbee6bd520 --- /dev/null +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java @@ -0,0 +1,22 @@ +package io.servicetalk.concurrent.api; + +import static java.util.Objects.requireNonNull; + +final class CustomCaptureAsyncContextProvider extends AbstractAsyncContextProvider { + + private final ContextCaptureProvider contextCaptureProvider; + + CustomCaptureAsyncContextProvider(ContextCaptureProvider contextCaptureProvider) { + this.contextCaptureProvider = requireNonNull(contextCaptureProvider); + } + + @Override + public CapturedContext captureContext() { + return contextCaptureProvider.captureContext(); + } + + @Override + public CapturedContext captureContextCopy() { + return contextCaptureProvider.captureContextCopy(); + } +} diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java index 9251225294..1117a6970b 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java @@ -15,322 +15,26 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.concurrent.CompletableSource; -import io.servicetalk.concurrent.PublisherSource.Subscriber; -import io.servicetalk.concurrent.SingleSource; import io.servicetalk.context.api.ContextMap; -import io.servicetalk.context.api.ContextMapHolder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +final class DefaultAsyncContextProvider extends AbstractAsyncContextProvider { -import java.util.concurrent.Callable; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ScheduledExecutorService; -import java.util.function.BiConsumer; -import java.util.function.BiFunction; -import java.util.function.Consumer; -import java.util.function.Function; -import javax.annotation.Nonnull; - -import static java.lang.ThreadLocal.withInitial; - -final class DefaultAsyncContextProvider implements AsyncContextProvider { - - private static final Logger LOGGER = LoggerFactory.getLogger(DefaultAsyncContextProvider.class); - private static final ThreadLocal CONTEXT_THREAD_LOCAL = - withInitial(DefaultAsyncContextProvider::newContextMap); - - static final AsyncContextProvider INSTANCE = new DefaultAsyncContextProvider(); + static final DefaultAsyncContextProvider INSTANCE = new DefaultAsyncContextProvider(); private DefaultAsyncContextProvider() { // singleton } - @Nonnull - @Override - public ContextMap context() { - final Thread t = Thread.currentThread(); - if (t instanceof ContextMapHolder) { - final ContextMapHolder contextMapHolder = (ContextMapHolder) t; - ContextMap map = contextMapHolder.context(); - if (map == null) { - map = newContextMap(); - contextMapHolder.context(map); - } - return map; - } else { - return CONTEXT_THREAD_LOCAL.get(); - } - } - - @Override - public ContextMap captureContext() { - return context(); - } - - @Override - public Scope attachContext(ContextMap contextMap) { - final Thread currentThread = Thread.currentThread(); - if (currentThread instanceof ContextMapHolder) { - final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; - ContextMap prev = asyncContextMapHolder.context(); - asyncContextMapHolder.context(contextMap); - return () -> detachContext(contextMap, prev == null ? newContextMap() : prev); - } else { - return slowPathSetContext(contextMap); - } - } - - private static Scope slowPathSetContext(ContextMap contextMap) { - ContextMap prev = CONTEXT_THREAD_LOCAL.get(); - CONTEXT_THREAD_LOCAL.set(contextMap); - return () -> detachContext(contextMap, prev); - } - - private static void detachContext(ContextMap expectedContext, ContextMap toRestore) { - final Thread currentThread = Thread.currentThread(); - if (currentThread instanceof ContextMapHolder) { - final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; - ContextMap current = asyncContextMapHolder.context(); - if (current != expectedContext) { - LOGGER.debug("Current context didn't match the expected context. current: {}, expected: {}", - current, expectedContext); - } - asyncContextMapHolder.context(toRestore); - } else { - slowPathDetachContext(expectedContext, toRestore); - } - } - - private static void slowPathDetachContext(ContextMap expectedContext, ContextMap toRestore) { - ContextMap current = CONTEXT_THREAD_LOCAL.get(); - if (current != expectedContext) { - LOGGER.debug("Current context didn't match the expected context. current: {}, expected: {}", - current, expectedContext); - } - CONTEXT_THREAD_LOCAL.set(toRestore); - } - - @Override - public CompletableSource.Subscriber wrapCancellable(final CompletableSource.Subscriber subscriber, - final ContextMap context) { - if (subscriber instanceof ContextPreservingCompletableSubscriber) { - final ContextPreservingCompletableSubscriber s = (ContextPreservingCompletableSubscriber) subscriber; - if (s.saved == context) { - return subscriber instanceof ContextPreservingCompletableSubscriberAndCancellable ? subscriber : - new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber && - ((ContextPreservingCancellableCompletableSubscriber) subscriber).saved == context) { - // no need to check for instanceof ContextPreservingCompletableSubscriberAndCancellable, because - // it extends from ContextPreservingSingleSubscriber. - return subscriber; - } - return new ContextPreservingCancellableCompletableSubscriber(subscriber, context); - } - - @Override - public CompletableSource.Subscriber wrapCompletableSubscriber(final CompletableSource.Subscriber subscriber, - final ContextMap context) { - if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber) { - final ContextPreservingCancellableCompletableSubscriber s = - (ContextPreservingCancellableCompletableSubscriber) subscriber; - if (s.saved == context) { - // replace current wrapper with wrapper that includes Subscriber and Cancellable - return new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingCompletableSubscriber && - ((ContextPreservingCompletableSubscriber) subscriber).saved == context) { - // no need to check for instanceof ContextPreservingCompletableSubscriberAndCancellable, because - // it extends from ContextPreservingCompletableSubscriber. - return subscriber; - } - return new ContextPreservingCompletableSubscriber(subscriber, context); - } - - @Override - public CompletableSource.Subscriber wrapCompletableSubscriberAndCancellable( - final CompletableSource.Subscriber subscriber, final ContextMap context) { - if (subscriber instanceof ContextPreservingCompletableSubscriber) { - final ContextPreservingCompletableSubscriber s = (ContextPreservingCompletableSubscriber) subscriber; - if (s.saved == context) { - return subscriber instanceof ContextPreservingCompletableSubscriberAndCancellable ? subscriber : - new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber) { - final ContextPreservingCancellableCompletableSubscriber s = - (ContextPreservingCancellableCompletableSubscriber) subscriber; - if (s.saved == context) { - return new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); - } - } - return new ContextPreservingCompletableSubscriberAndCancellable(subscriber, context); - } - - @Override - public SingleSource.Subscriber wrapCancellable(final SingleSource.Subscriber subscriber, - final ContextMap context) { - if (subscriber instanceof ContextPreservingSingleSubscriber) { - final ContextPreservingSingleSubscriber s = (ContextPreservingSingleSubscriber) subscriber; - if (s.saved == context) { - return subscriber instanceof ContextPreservingSingleSubscriberAndCancellable ? subscriber : - new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingCancellableSingleSubscriber && - ((ContextPreservingCancellableSingleSubscriber) subscriber).saved == context) { - // no need to check for instanceof ContextPreservingSingleSubscriberAndCancellable, because - // it extends from ContextPreservingSingleSubscriber. - return subscriber; - } - return new ContextPreservingCancellableSingleSubscriber<>(subscriber, context); - } - - @Override - public SingleSource.Subscriber wrapSingleSubscriber(final SingleSource.Subscriber subscriber, - final ContextMap context) { - if (subscriber instanceof ContextPreservingCancellableSingleSubscriber) { - final ContextPreservingCancellableSingleSubscriber s = - (ContextPreservingCancellableSingleSubscriber) subscriber; - if (s.saved == context) { - return new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingSingleSubscriber && - ((ContextPreservingSingleSubscriber) subscriber).saved == context) { - // no need to check for instanceof ContextPreservingSingleSubscriberAndCancellable, because - // it extends from ContextPreservingSingleSubscriber. - return subscriber; - } - return new ContextPreservingSingleSubscriber<>(subscriber, context); - } - - @Override - public SingleSource.Subscriber wrapSingleSubscriberAndCancellable( - final SingleSource.Subscriber subscriber, final ContextMap context) { - if (subscriber instanceof ContextPreservingSingleSubscriber) { - final ContextPreservingSingleSubscriber s = (ContextPreservingSingleSubscriber) subscriber; - if (s.saved == context) { - return subscriber instanceof ContextPreservingSingleSubscriberAndCancellable ? subscriber : - new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingCancellableSingleSubscriber) { - final ContextPreservingCancellableSingleSubscriber s = - (ContextPreservingCancellableSingleSubscriber) subscriber; - if (s.saved == context) { - return new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); - } - } - return new ContextPreservingSingleSubscriberAndCancellable<>(subscriber, context); - } - - @Override - public Subscriber wrapSubscription(final Subscriber subscriber, final ContextMap context) { - if (subscriber instanceof ContextPreservingSubscriber) { - final ContextPreservingSubscriber s = (ContextPreservingSubscriber) subscriber; - if (s.saved == context) { - return subscriber instanceof ContextPreservingSubscriberAndSubscription ? subscriber : - new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingSubscriptionSubscriber && - ((ContextPreservingSubscriptionSubscriber) subscriber).saved == context) { - // no need to check for instanceof ContextPreservingSubscriberAndSubscription, because - // it extends from ContextPreservingSubscriptionSubscriber. - return subscriber; - } - return new ContextPreservingSubscriptionSubscriber<>(subscriber, context); - } - - @Override - public Subscriber wrapPublisherSubscriber(final Subscriber subscriber, final ContextMap context) { - if (subscriber instanceof ContextPreservingSubscriptionSubscriber) { - final ContextPreservingSubscriptionSubscriber s = - (ContextPreservingSubscriptionSubscriber) subscriber; - if (s.saved == context) { - return new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingSubscriber && - ((ContextPreservingSubscriber) subscriber).saved == context) { - // no need to check for instanceof ContextPreservingSubscriberAndSubscription, because - // it extends from ContextPreservingSubscriptionSubscriber. - return subscriber; - } - return new ContextPreservingSubscriber<>(subscriber, context); - } - - @Override - public Subscriber wrapPublisherSubscriberAndSubscription(final Subscriber subscriber, - final ContextMap context) { - if (subscriber instanceof ContextPreservingSubscriber) { - final ContextPreservingSubscriber s = (ContextPreservingSubscriber) subscriber; - if (s.saved == context) { - return subscriber instanceof ContextPreservingSubscriberAndSubscription ? subscriber : - new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingSubscriptionSubscriber) { - final ContextPreservingSubscriptionSubscriber s = - (ContextPreservingSubscriptionSubscriber) subscriber; - if (s.saved == context) { - return new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); - } - } - return new ContextPreservingSubscriberAndSubscription<>(subscriber, context); - } - - @Override - public Executor wrapJdkExecutor(final Executor executor) { - return ContextPreservingExecutor.of(executor); - } - - @Override - public ExecutorService wrapJdkExecutorService(final ExecutorService executor) { - return ContextPreservingExecutorService.of(executor); - } - - @Override - public io.servicetalk.concurrent.api.Executor wrapExecutor(final io.servicetalk.concurrent.api.Executor executor) { - return ContextPreservingStExecutor.of(executor); - } - - @Override - public ScheduledExecutorService wrapJdkScheduledExecutorService(final ScheduledExecutorService executor) { - return ContextPreservingScheduledExecutorService.of(executor); - } - - @Override - public CompletableFuture wrapCompletableFuture(final CompletableFuture future, final ContextMap context) { - return ContextPreservingCompletableFuture.newContextPreservingFuture(future, context); - } - - @Override - public Runnable wrapRunnable(final Runnable runnable, final ContextMap context) { - return new ContextPreservingRunnable(runnable, context); - } - - @Override - public Callable wrapCallable(final Callable callable, final ContextMap context) { - return new ContextPreservingCallable<>(callable, context); - } - - @Override - public Consumer wrapConsumer(final Consumer consumer, final ContextMap context) { - return new ContextPreservingConsumer<>(consumer, context); - } - - @Override - public Function wrapFunction(final Function func, final ContextMap context) { - return new ContextPreservingFunction<>(func, context); - } - @Override - public BiConsumer wrapBiConsumer(final BiConsumer consumer, final ContextMap context) { - return new ContextPreservingBiConsumer<>(consumer, context); + public CapturedContext captureContext() { + ContextMap current = context(); + return () -> ContextMapThreadLocal.attachContext(current); } @Override - public BiFunction wrapBiFunction(final BiFunction func, final ContextMap context) { - return new ContextPreservingBiFunction<>(func, context); + public CapturedContext captureContextCopy() { + ContextMap currentCopy = context().copy(); + return () -> ContextMapThreadLocal.attachContext(currentCopy); } private static ContextMap newContextMap() { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/LiftSynchronousPublisherToSingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/LiftSynchronousPublisherToSingle.java index 6bec0272ff..73731b8ed5 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/LiftSynchronousPublisherToSingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/LiftSynchronousPublisherToSingle.java @@ -16,7 +16,6 @@ package io.servicetalk.concurrent.api; import io.servicetalk.concurrent.SingleSource; -import io.servicetalk.context.api.ContextMap; import static io.servicetalk.concurrent.internal.SubscriberUtils.deliverErrorFromSource; import static java.util.Objects.requireNonNull; @@ -44,7 +43,7 @@ public void subscribe(final SingleSource.Subscriber subscriber) { @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { - original.delegateSubscribe(customOperator.apply(subscriber), contextMap, contextProvider); + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + original.delegateSubscribe(customOperator.apply(subscriber), capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/MulticastPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/MulticastPublisher.java index 65476d16f5..353e2d4344 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/MulticastPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/MulticastPublisher.java @@ -20,7 +20,6 @@ import io.servicetalk.concurrent.internal.DelayedSubscription; import io.servicetalk.concurrent.internal.RejectedSubscribeException; import io.servicetalk.concurrent.internal.TerminalNotification; -import io.servicetalk.context.api.ContextMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,11 +92,11 @@ static MulticastPublisher newMulticastPublisher( } @Override - final void handleSubscribe(Subscriber subscriber, ContextMap contextMap, + final void handleSubscribe(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { final State cState = state; assert cState != null; - cState.addNewSubscriber(subscriber, contextMap, contextProvider); + cState.addNewSubscriber(subscriber, capturedContext, contextProvider); } void resetState(int maxQueueSize, int minSubscribers) { @@ -118,7 +117,7 @@ class State extends MulticastRootSubscriber> impleme @Nullable @Override final TerminalSubscriber addSubscriber(final MulticastFixedSubscriber subscriber, - @Nullable ContextMap contextMap, + @Nullable CapturedContext capturedContext, AsyncContextProvider contextProvider) { for (;;) { final Subscriber[] currSubs = subscribers; @@ -131,10 +130,10 @@ final TerminalSubscriber addSubscriber(final MulticastFixedSubscriber subs System.arraycopy(currSubs, 0, newSubs, 0, currSubs.length); newSubs[currSubs.length] = subscriber; if (newSubscribersUpdater.compareAndSet(this, currSubs, newSubs)) { - if (contextMap != null) { + if (capturedContext != null) { // This operator has special behavior where it chooses to use the AsyncContext and signal // offloader from the last subscribe operation. - original.delegateSubscribe(this, contextMap, contextProvider); + original.delegateSubscribe(this, capturedContext, contextProvider); } return null; } @@ -237,21 +236,21 @@ void processOnNextEvent(final Object wrapped) { throw new UnsupportedOperationException("onNext queuing not supported. wrapped=" + wrapped); } - final void addNewSubscriber(Subscriber subscriber, ContextMap contextMap, + final void addNewSubscriber(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { final int sCount = subscribeCountUpdater.incrementAndGet(this); if (exactlyMinSubscribers && sCount > minSubscribers) { - deliverOnSubscribeAndOnError(subscriber, contextMap, contextProvider, + deliverOnSubscribeAndOnError(subscriber, capturedContext, contextProvider, new RejectedSubscribeException("Only " + minSubscribers + " subscribers are allowed!")); return; } MulticastFixedSubscriber multiSubscriber = - new MulticastFixedSubscriber<>(this, subscriber, contextMap, contextProvider, sCount); + new MulticastFixedSubscriber<>(this, subscriber, capturedContext, contextProvider, sCount); if (tryAcquireLock(subscriptionLockUpdater, this)) { try { // This operator has special behavior where it chooses to use the AsyncContext and signal // offloader from the last subscribe operation. - processSubscribeEventInternal(multiSubscriber, sCount == minSubscribers ? contextMap : null, + processSubscribeEventInternal(multiSubscriber, sCount == minSubscribers ? capturedContext : null, contextProvider); } finally { if (!releaseLock(subscriptionLockUpdater, this)) { @@ -260,7 +259,7 @@ final void addNewSubscriber(Subscriber subscriber, ContextMap context } } else { subscriptionEvents.add(new SubscribeEvent<>(multiSubscriber, - sCount == minSubscribers ? contextMap : null, contextProvider)); + sCount == minSubscribers ? capturedContext : null, contextProvider)); processSubscriptionEvents(); } } @@ -355,13 +354,14 @@ private abstract static class MulticastRootSubscriber * Invocation while {@link #subscriptionLock} is held. * @param subscriber The {@link Subscriber} to remove. - * @param contextMap The context map to used when subscribing upstream, or {@code null} if should not subscribe. + * @param capturedContext The context map to used when subscribing upstream, or {@code null} if should not + * subscribe. * @param contextProvider The context provider to used when subscribing upstream. * @return {@code null} if {@code subscriber} was added to the list, or non-{@code null} if not added to the * because there was previously a terminal event. */ @Nullable - abstract TerminalSubscriber addSubscriber(T subscriber, @Nullable ContextMap contextMap, + abstract TerminalSubscriber addSubscriber(T subscriber, @Nullable CapturedContext capturedContext, AsyncContextProvider contextProvider); /** @@ -400,14 +400,14 @@ abstract TerminalSubscriber addSubscriber(T subscriber, @Nullable ContextMap abstract long processCancelEvent(T subscriber); /** - * Invoked after {@link #addSubscriber(MulticastLeafSubscriber, ContextMap, AsyncContextProvider)}. + * Invoked after {@link #addSubscriber(MulticastLeafSubscriber, CapturedContext, AsyncContextProvider)}. *

* Invocation while {@link #subscriptionLock} is held. * @param subscriber The subscriber which was passed to * @param terminalSubscriber {@code null} if the {@code subscriber} was added to the list or non-{@code null} * if a terminal event has occurred, and this method MUST eventually deliver the terminal signal to * {@code subscriber}. - * {@link #addSubscriber(MulticastLeafSubscriber, ContextMap, AsyncContextProvider)}. + * {@link #addSubscriber(MulticastLeafSubscriber, CapturedContext, AsyncContextProvider)}. * @return {@code false} to stop handling this processor and break out early (e.g. can happen if * {@code terminalSubscriber} is non-{@code null} no signals are queued and the terminal is delivered). * {@code true} to unblock the {@code subscriber}'s signal queue and keep processing events. @@ -504,7 +504,8 @@ final void processSubscriptionEvents() { } else if (event instanceof SubscribeEvent) { @SuppressWarnings("unchecked") final SubscribeEvent sEvent = (SubscribeEvent) event; - processSubscribeEventInternal(sEvent.subscriber, sEvent.contextMap, sEvent.contextProvider); + processSubscribeEventInternal(sEvent.subscriber, sEvent.capturedContext, + sEvent.contextProvider); } else if (event instanceof CancelEvent) { @SuppressWarnings("unchecked") final CancelEvent cEvent = (CancelEvent) event; @@ -544,9 +545,9 @@ private void processCancelEventInternal(T subscriber, boolean cancelUpstream) { } } - void processSubscribeEventInternal(T subscriber, @Nullable ContextMap contextMap, + void processSubscribeEventInternal(T subscriber, @Nullable CapturedContext capturedContext, AsyncContextProvider contextProvider) { - TerminalSubscriber terminalSubscriber = addSubscriber(subscriber, contextMap, contextProvider); + TerminalSubscriber terminalSubscriber = addSubscriber(subscriber, capturedContext, contextProvider); if (!processSubscribeEvent(subscriber, terminalSubscriber)) { return; } @@ -568,13 +569,13 @@ void processSubscribeEventInternal(T subscriber, @Nullable ContextMap contextMap static final class SubscribeEvent> { private final T subscriber; @Nullable - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; - SubscribeEvent(final T subscriber, @Nullable final ContextMap contextMap, + SubscribeEvent(final T subscriber, @Nullable final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { this.subscriber = subscriber; - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; } } @@ -620,12 +621,12 @@ static final class MulticastFixedSubscriber extends MulticastLeafSubscriber.State root, - final Subscriber subscriber, final ContextMap contextMap, + final Subscriber subscriber, final CapturedContext capturedContext, final AsyncContextProvider contextProvider, final int index) { this.root = root; this.index = index; this.subscriber = requireNonNull(subscriber); - ctxSubscriber = contextProvider.wrapPublisherSubscriber(subscriber, contextMap); + ctxSubscriber = contextProvider.wrapPublisherSubscriber(subscriber, capturedContext); } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java index f183672e53..62f603c3d9 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java @@ -37,6 +37,8 @@ final class NoopAsyncContextProvider implements AsyncContextProvider { static final AsyncContextProvider INSTANCE = new NoopAsyncContextProvider(); + private static final CapturedContext NOOP_SAVED_CONTEXT = () -> Scope.NOOP; + private NoopAsyncContextProvider() { // singleton } @@ -47,64 +49,64 @@ public ContextMap context() { } @Override - public ContextMap captureContext() { - return NoopContextMap.INSTANCE; + public CapturedContext captureContext() { + return NOOP_SAVED_CONTEXT; } @Override - public Scope attachContext(ContextMap contextMap) { - return Scope.NOOP; + public CapturedContext captureContextCopy() { + return NOOP_SAVED_CONTEXT; } @Override public CompletableSource.Subscriber wrapCancellable(final CompletableSource.Subscriber subscriber, - final ContextMap context) { + final CapturedContext context) { return subscriber; } @Override public CompletableSource.Subscriber wrapCompletableSubscriber(final CompletableSource.Subscriber subscriber, - final ContextMap context) { + final CapturedContext context) { return subscriber; } @Override public CompletableSource.Subscriber wrapCompletableSubscriberAndCancellable( - final CompletableSource.Subscriber subscriber, final ContextMap context) { + final CompletableSource.Subscriber subscriber, final CapturedContext context) { return subscriber; } @Override public SingleSource.Subscriber wrapCancellable(final SingleSource.Subscriber subscriber, - final ContextMap context) { + final CapturedContext context) { return subscriber; } @Override public SingleSource.Subscriber wrapSingleSubscriber(final SingleSource.Subscriber subscriber, - final ContextMap context) { + final CapturedContext context) { return subscriber; } @Override public SingleSource.Subscriber wrapSingleSubscriberAndCancellable( - final SingleSource.Subscriber subscriber, final ContextMap context) { + final SingleSource.Subscriber subscriber, final CapturedContext context) { return subscriber; } @Override - public Subscriber wrapSubscription(final Subscriber subscriber, final ContextMap context) { + public Subscriber wrapSubscription(final Subscriber subscriber, final CapturedContext context) { return subscriber; } @Override - public Subscriber wrapPublisherSubscriber(final Subscriber subscriber, final ContextMap context) { + public Subscriber wrapPublisherSubscriber(final Subscriber subscriber, final CapturedContext context) { return subscriber; } @Override public Subscriber wrapPublisherSubscriberAndSubscription(final Subscriber subscriber, - final ContextMap context) { + final CapturedContext context) { return subscriber; } @@ -130,42 +132,42 @@ public ScheduledExecutorService wrapJdkScheduledExecutorService(final ScheduledE @Override public CompletableFuture wrapCompletableFuture(final CompletableFuture future, - final ContextMap context) { + final CapturedContext context) { return future; } @Override - public Runnable wrapRunnable(final Runnable runnable, final ContextMap context) { + public Runnable wrapRunnable(final Runnable runnable, final CapturedContext context) { return runnable; } @Override - public Callable wrapCallable(final Callable callable, final ContextMap context) { + public Callable wrapCallable(final Callable callable, final CapturedContext context) { return callable; } @Override - public Consumer wrapConsumer(final Consumer consumer, final ContextMap context) { + public Consumer wrapConsumer(final Consumer consumer, final CapturedContext context) { return consumer; } @Override - public Function wrapFunction(final Function func, final ContextMap context) { + public Function wrapFunction(final Function func, final CapturedContext context) { return func; } @Override - public BiConsumer wrapBiConsumer(final BiConsumer consumer, final ContextMap context) { + public BiConsumer wrapBiConsumer(final BiConsumer consumer, final CapturedContext context) { return consumer; } @Override public BiFunction wrapBiFunction(final BiFunction func, - final ContextMap context) { + final CapturedContext context) { return func; } - private static final class NoopContextMap implements ContextMap { + static final class NoopContextMap implements ContextMap { static final ContextMap INSTANCE = new NoopContextMap(); private NoopContextMap() { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnErrorResumeCompletable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnErrorResumeCompletable.java index 20f1968538..b2bf3d8041 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnErrorResumeCompletable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnErrorResumeCompletable.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.internal.SequentialCancellable; -import io.servicetalk.context.api.ContextMap; import java.util.function.Function; import java.util.function.Predicate; @@ -40,26 +39,26 @@ final class OnErrorResumeCompletable extends AbstractNoHandleSubscribeCompletabl @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { original.delegateSubscribe( - new ResumeSubscriber(this, subscriber, contextMap, contextProvider), - contextMap, contextProvider); + new ResumeSubscriber(this, subscriber, capturedContext, contextProvider), + capturedContext, contextProvider); } private static final class ResumeSubscriber implements Subscriber { private final OnErrorResumeCompletable parent; private final Subscriber subscriber; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; @Nullable private SequentialCancellable sequentialCancellable; private boolean resubscribed; ResumeSubscriber(OnErrorResumeCompletable parent, Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { this.parent = parent; this.subscriber = subscriber; - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; } @@ -94,7 +93,7 @@ public void onError(Throwable throwable) { if (next == null) { subscriber.onError(throwable); } else { - final Subscriber wrappedSubscriber = contextProvider.wrapCompletableSubscriber(this, contextMap); + final Subscriber wrappedSubscriber = contextProvider.wrapCompletableSubscriber(this, capturedContext); next.subscribeInternal(wrappedSubscriber); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnErrorResumePublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnErrorResumePublisher.java index 23642bae6e..6d8ce4c090 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnErrorResumePublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnErrorResumePublisher.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import java.util.function.Function; import java.util.function.Predicate; import javax.annotation.Nullable; @@ -38,23 +36,23 @@ final class OnErrorResumePublisher extends AbstractNoHandleSubscribePublisher @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { - original.delegateSubscribe(new ResumeSubscriber(subscriber, contextMap, contextProvider), - contextMap, contextProvider); + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + original.delegateSubscribe(new ResumeSubscriber(subscriber, capturedContext, contextProvider), + capturedContext, contextProvider); } private final class ResumeSubscriber implements Subscriber { private final Subscriber subscriber; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; @Nullable private SequentialSubscription sequentialSubscription; private boolean resubscribed; - ResumeSubscriber(Subscriber subscriber, ContextMap contextMap, + ResumeSubscriber(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { this.subscriber = subscriber; - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; } @@ -90,7 +88,7 @@ public void onError(Throwable throwable) { subscriber.onError(throwable); } else { final Subscriber offloadedSubscriber = - contextProvider.wrapPublisherSubscriber(this, contextMap); + contextProvider.wrapPublisherSubscriber(this, capturedContext); next.subscribeInternal(offloadedSubscriber); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnErrorResumeSingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnErrorResumeSingle.java index 8c92ac938e..3cd59c17d2 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnErrorResumeSingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnErrorResumeSingle.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.internal.SequentialCancellable; -import io.servicetalk.context.api.ContextMap; import java.util.function.Function; import java.util.function.Predicate; @@ -40,23 +39,23 @@ final class OnErrorResumeSingle extends AbstractNoHandleSubscribeSingle { @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { - original.delegateSubscribe(new ResumeSubscriber(subscriber, contextMap, contextProvider), - contextMap, contextProvider); + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + original.delegateSubscribe(new ResumeSubscriber(subscriber, capturedContext, contextProvider), + capturedContext, contextProvider); } private final class ResumeSubscriber implements Subscriber { private final Subscriber subscriber; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; @Nullable private SequentialCancellable sequentialCancellable; private boolean resubscribed; - ResumeSubscriber(Subscriber subscriber, ContextMap contextMap, + ResumeSubscriber(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { this.subscriber = subscriber; - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; } @@ -89,7 +88,8 @@ public void onError(Throwable throwable) { if (next == null) { subscriber.onError(throwable); } else { - final Subscriber wrappedSubscriber = contextProvider.wrapSingleSubscriber(this, contextMap); + final Subscriber wrappedSubscriber = + contextProvider.wrapSingleSubscriber(this, capturedContext); next.subscribeInternal(wrappedSubscriber); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnSubscribeIgnoringSubscriberForOffloading.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnSubscribeIgnoringSubscriberForOffloading.java index b4d7af0732..b4c1b2f7a3 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnSubscribeIgnoringSubscriberForOffloading.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/OnSubscribeIgnoringSubscriberForOffloading.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.PublisherSource; import io.servicetalk.concurrent.PublisherSource.Subscriber; -import io.servicetalk.context.api.ContextMap; import javax.annotation.Nullable; @@ -52,10 +51,10 @@ public void onComplete() { } static Subscriber wrapWithDummyOnSubscribe(Subscriber original, - ContextMap contextMap, + CapturedContext capturedContext, AsyncContextProvider contextProvider) { Subscriber toReturn = contextProvider.wrapPublisherSubscriber( - new OnSubscribeIgnoringSubscriberForOffloading<>(original), contextMap); + new OnSubscribeIgnoringSubscriberForOffloading<>(original), capturedContext); // We have created a wrapped Subscriber but we have sent onSubscribe to the original Subscriber // already, so we send an onSubscribe to the wrapped Subscriber which ignores this signal but makes // the wrapped does not see spec violation (onError without onSubscribe) for the offloaded diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublishAndSubscribeOnCompletables.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublishAndSubscribeOnCompletables.java index 0956f9e85b..85e922dea9 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublishAndSubscribeOnCompletables.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublishAndSubscribeOnCompletables.java @@ -18,7 +18,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.CompletableSource; import io.servicetalk.concurrent.CompletableSource.Subscriber; -import io.servicetalk.context.api.ContextMap; import java.util.function.BooleanSupplier; @@ -40,9 +39,9 @@ private PublishAndSubscribeOnCompletables() { } static void deliverOnSubscribeAndOnError(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider, + CapturedContext capturedContext, AsyncContextProvider contextProvider, Throwable cause) { - deliverErrorFromSource(contextProvider.wrapCompletableSubscriber(subscriber, contextMap), cause); + deliverErrorFromSource(contextProvider.wrapCompletableSubscriber(subscriber, capturedContext), cause); } static Completable publishOn(final Completable original, @@ -74,7 +73,7 @@ private static final class PublishOn extends TaskBasedAsyncCompletableOperator { @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { final Subscriber upstreamSubscriber; try { BooleanSupplier shouldOffload = shouldOffload(); @@ -88,7 +87,7 @@ void handleSubscribe(final Subscriber subscriber, return; } - super.handleSubscribe(upstreamSubscriber, contextMap, contextProvider); + super.handleSubscribe(upstreamSubscriber, capturedContext, contextProvider); } } @@ -110,7 +109,7 @@ private static final class SubscribeOn extends TaskBasedAsyncCompletableOperator @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { final Subscriber upstreamSubscriber; try { BooleanSupplier shouldOffload = shouldOffload(); @@ -123,7 +122,7 @@ void handleSubscribe(final Subscriber subscriber, if (shouldOffload.getAsBoolean()) { // offload the remainder of subscribe() executor().execute(() -> super.handleSubscribe( - upstreamSubscriber, contextMap, contextProvider)); + upstreamSubscriber, capturedContext, contextProvider)); return; } } catch (Throwable throwable) { @@ -133,7 +132,7 @@ void handleSubscribe(final Subscriber subscriber, } // continue non-offloaded subscribe() - super.handleSubscribe(upstreamSubscriber, contextMap, contextProvider); + super.handleSubscribe(upstreamSubscriber, capturedContext, contextProvider); } } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublishAndSubscribeOnPublishers.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublishAndSubscribeOnPublishers.java index 06da561e8c..455a99c8ee 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublishAndSubscribeOnPublishers.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublishAndSubscribeOnPublishers.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.PublisherSource; import io.servicetalk.concurrent.PublisherSource.Subscriber; -import io.servicetalk.context.api.ContextMap; import java.util.function.BooleanSupplier; @@ -35,9 +34,9 @@ private PublishAndSubscribeOnPublishers() { } static void deliverOnSubscribeAndOnError(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider, + CapturedContext capturedContext, AsyncContextProvider contextProvider, Throwable cause) { - deliverErrorFromSource(contextProvider.wrapPublisherSubscriber(subscriber, contextMap), cause); + deliverErrorFromSource(contextProvider.wrapPublisherSubscriber(subscriber, capturedContext), cause); } static Publisher publishOn(final Publisher original, @@ -69,7 +68,7 @@ private static final class PublishOn extends TaskBasedAsyncPublisherOperator< @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { final Subscriber upstreamSubscriber; try { BooleanSupplier shouldOffload = shouldOffload(); @@ -85,7 +84,7 @@ void handleSubscribe(final Subscriber subscriber, return; } - super.handleSubscribe(upstreamSubscriber, contextMap, contextProvider); + super.handleSubscribe(upstreamSubscriber, capturedContext, contextProvider); } } @@ -108,7 +107,7 @@ private static final class SubscribeOn extends TaskBasedAsyncPublisherOperato @Override public void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { final Subscriber upstreamSubscriber; try { BooleanSupplier shouldOffload = shouldOffload(); @@ -121,7 +120,7 @@ public void handleSubscribe(final Subscriber subscriber, if (shouldOffload.getAsBoolean()) { // offload the remainder of subscribe() executor().execute(() -> super.handleSubscribe( - upstreamSubscriber, contextMap, contextProvider)); + upstreamSubscriber, capturedContext, contextProvider)); return; } } catch (Throwable throwable) { @@ -132,7 +131,7 @@ public void handleSubscribe(final Subscriber subscriber, } // continue non-offloaded subscribe() - super.handleSubscribe(upstreamSubscriber, contextMap, contextProvider); + super.handleSubscribe(upstreamSubscriber, capturedContext, contextProvider); } } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublishAndSubscribeOnSingles.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublishAndSubscribeOnSingles.java index 7cb9c09854..c3511c24b1 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublishAndSubscribeOnSingles.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublishAndSubscribeOnSingles.java @@ -18,7 +18,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.CompletableSource; import io.servicetalk.concurrent.SingleSource; -import io.servicetalk.context.api.ContextMap; import java.util.function.BooleanSupplier; @@ -36,9 +35,9 @@ private PublishAndSubscribeOnSingles() { } static void deliverOnSubscribeAndOnError(SingleSource.Subscriber subscriber, - ContextMap contextMap, + CapturedContext capturedContext, AsyncContextProvider contextProvider, Throwable cause) { - deliverErrorFromSource(contextProvider.wrapSingleSubscriber(subscriber, contextMap), cause); + deliverErrorFromSource(contextProvider.wrapSingleSubscriber(subscriber, capturedContext), cause); } static Single publishOn(final Single original, @@ -69,7 +68,7 @@ private static final class PublishOn extends TaskBasedAsyncSingleOperator @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { Subscriber upstreamSubscriber; try { BooleanSupplier shouldOffload = shouldOffload(); @@ -83,7 +82,7 @@ void handleSubscribe(final Subscriber subscriber, return; } - super.handleSubscribe(upstreamSubscriber, contextMap, contextProvider); + super.handleSubscribe(upstreamSubscriber, capturedContext, contextProvider); } } @@ -105,7 +104,7 @@ private static final class SubscribeOn extends TaskBasedAsyncSingleOperator subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { Subscriber upstreamSubscriber; try { BooleanSupplier shouldOffload = shouldOffload(); @@ -118,7 +117,7 @@ public void handleSubscribe(final Subscriber subscriber, if (shouldOffload.getAsBoolean()) { // offload the remainder of subscribe() executor().execute(() -> super.handleSubscribe( - upstreamSubscriber, contextMap, contextProvider)); + upstreamSubscriber, capturedContext, contextProvider)); return; } } catch (Throwable throwable) { @@ -129,7 +128,7 @@ public void handleSubscribe(final Subscriber subscriber, } // continue non-offloaded subscribe() - super.handleSubscribe(upstreamSubscriber, contextMap, contextProvider); + super.handleSubscribe(upstreamSubscriber, capturedContext, contextProvider); } } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Publisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Publisher.java index 13716de23f..75c1559e23 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Publisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Publisher.java @@ -4353,9 +4353,9 @@ public final BlockingIterable toIterable(int queueCapacityHint) { * @param provider The {@link AsyncContextProvider} which is the source of the map * @return {@link ContextMap} for this subscribe operation. */ - ContextMap contextForSubscribe(AsyncContextProvider provider) { + CapturedContext contextForSubscribe(AsyncContextProvider provider) { // the default behavior is to copy the map. Some operators may want to use shared map - return provider.captureContext().copy(); + return provider.captureContextCopy(); } /** @@ -4367,15 +4367,15 @@ ContextMap contextForSubscribe(AsyncContextProvider provider) { protected void subscribeInternal(Subscriber subscriber) { requireNonNull(subscriber); AsyncContextProvider provider = AsyncContext.provider(); - final ContextMap contextMap = contextForSubscribe(provider); - Subscriber wrapped = provider.wrapSubscription(subscriber, contextMap); - if (provider.context() == contextMap) { + CapturedContext capturedContext = contextForSubscribe(provider); + Subscriber wrapped = provider.wrapSubscription(subscriber, capturedContext); + if (provider.context() == capturedContext) { // No need to wrap as we are sharing the AsyncContext - handleSubscribe(wrapped, contextMap, provider); + handleSubscribe(wrapped, capturedContext, provider); } else { // Ensure that AsyncContext used for handleSubscribe() is the contextMap for the subscribe() - try (Scope ignored = provider.attachContext(contextMap)) { - handleSubscribe(wrapped, contextMap, provider); + try (Scope ignored = capturedContext.restoreContext()) { + handleSubscribe(wrapped, capturedContext, provider); } } } @@ -4860,12 +4860,12 @@ public static Publisher mergeAllDelayError(int maxConcurrency, Publisher< * Delegate subscribe calls in an operator chain. This method is used by operators to subscribe to the upstream * source. * @param subscriber the subscriber. - * @param contextMap the {@link ContextMap} to use for this {@link Subscriber}. + * @param capturedContext the {@link ContextMap} to use for this {@link Subscriber}. * @param contextProvider the {@link AsyncContextProvider} used to wrap any objects to preserve {@link ContextMap}. */ final void delegateSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { - handleSubscribe(subscriber, contextMap, contextProvider); + CapturedContext capturedContext, AsyncContextProvider contextProvider) { + handleSubscribe(subscriber, capturedContext, contextProvider); } /** @@ -4874,13 +4874,13 @@ final void delegateSubscribe(Subscriber subscriber, * then calls {@link #handleSubscribe(PublisherSource.Subscriber)}. * Operators that do not wish to wrap the passed {@link Subscriber} can override this method and omit the wrapping. * @param subscriber the subscriber. - * @param contextMap the {@link ContextMap} to use for this {@link Subscriber}. + * @param capturedContext the {@link ContextMap} to use for this {@link Subscriber}. * @param contextProvider the {@link AsyncContextProvider} used to wrap any objects to preserve {@link ContextMap}. */ - void handleSubscribe(Subscriber subscriber, ContextMap contextMap, + void handleSubscribe(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { try { - Subscriber wrapped = contextProvider.wrapPublisherSubscriber(subscriber, contextMap); + Subscriber wrapped = contextProvider.wrapPublisherSubscriber(subscriber, capturedContext); handleSubscribe(wrapped); } catch (Throwable t) { LOGGER.warn("Unexpected exception from subscribe(), assuming no interaction with the Subscriber.", t); diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherGroupBy.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherGroupBy.java index 923c1aa538..924567604e 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherGroupBy.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherGroupBy.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import java.util.function.Function; import javax.annotation.Nullable; @@ -43,16 +41,16 @@ final class PublisherGroupBy extends AbstractPublisherGroupBy { @Override void handleSubscribe(Subscriber> subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { - original.delegateSubscribe(new GroupBySubscriber(subscriber, queueLimit, initialCapacityForGroups, contextMap, - contextProvider), contextMap, contextProvider); + CapturedContext capturedContext, AsyncContextProvider contextProvider) { + original.delegateSubscribe(new GroupBySubscriber(subscriber, queueLimit, initialCapacityForGroups, capturedContext, + contextProvider), capturedContext, contextProvider); } private final class GroupBySubscriber extends AbstractGroupBySubscriber { GroupBySubscriber(final Subscriber> target, final int maxQueueSize, - final int initialCapacityForGroups, final ContextMap contextMap, + final int initialCapacityForGroups, final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { - super(target, maxQueueSize, initialCapacityForGroups, contextMap, contextProvider); + super(target, maxQueueSize, initialCapacityForGroups, capturedContext, contextProvider); } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherGroupToMany.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherGroupToMany.java index d3f6045623..4f44f7033a 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherGroupToMany.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherGroupToMany.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import java.util.Iterator; import java.util.function.Function; import javax.annotation.Nullable; @@ -45,16 +43,16 @@ final class PublisherGroupToMany extends AbstractPublisherGroupBy> subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { original.delegateSubscribe(new GroupBySubscriber(subscriber, queueLimit, initialCapacityForGroups, - contextMap, contextProvider), contextMap, contextProvider); + capturedContext, contextProvider), capturedContext, contextProvider); } private final class GroupBySubscriber extends AbstractGroupBySubscriber { GroupBySubscriber(final Subscriber> target, final int maxQueueSize, - final int initialCapacityForGroups, final ContextMap contextMap, + final int initialCapacityForGroups, final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { - super(target, maxQueueSize, initialCapacityForGroups, contextMap, contextProvider); + super(target, maxQueueSize, initialCapacityForGroups, capturedContext, contextProvider); } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java index 0f1ddc9e09..1b2e296962 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java @@ -29,16 +29,24 @@ final class PublisherSetContextOnSubscribe extends AbstractNoHandleSubscribeP } @Override - ContextMap contextForSubscribe(AsyncContextProvider provider) { - return context; + CapturedContext contextForSubscribe(AsyncContextProvider provider) { + CapturedContext parentContext = super.contextForSubscribe(provider); + return () -> { + Scope outer = parentContext.restoreContext(); + Scope inner = ContextMapThreadLocal.attachContext(context); + return () -> { + inner.close(); + outer.close(); + }; + }; } @Override void handleSubscribe(final Subscriber singleSubscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { // This operator currently only targets the subscribe method. Given this limitation if we try to change the // ContextMap now it is possible that operators downstream in the subscribe call stack may have modified // the ContextMap and we don't want to discard those changes by using a different ContextMap. - original.handleSubscribe(singleSubscriber, contextMap, contextProvider); + original.handleSubscribe(singleSubscriber, capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherShareContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherShareContextOnSubscribe.java index e7739df887..0abc8a3a6b 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherShareContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherShareContextOnSubscribe.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - final class PublisherShareContextOnSubscribe extends AbstractNoHandleSubscribePublisher { private final Publisher original; @@ -25,16 +23,16 @@ final class PublisherShareContextOnSubscribe extends AbstractNoHandleSubscrib } @Override - ContextMap contextForSubscribe(AsyncContextProvider provider) { + CapturedContext contextForSubscribe(AsyncContextProvider provider) { return provider.captureContext(); } @Override void handleSubscribe(final Subscriber singleSubscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { // This operator currently only targets the subscribe method. Given this limitation if we try to change the // ContextMap now it is possible that operators downstream in the subscribe call stack may have modified // the ContextMap and we don't want to discard those changes by using a different ContextMap. - original.handleSubscribe(singleSubscriber, contextMap, contextProvider); + original.handleSubscribe(singleSubscriber, capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RedoPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RedoPublisher.java index cdcb4f320b..8fc834ba0c 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RedoPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RedoPublisher.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.internal.ConcurrentSubscription; import io.servicetalk.concurrent.internal.TerminalNotification; -import io.servicetalk.context.api.ContextMap; import java.util.function.BiPredicate; import java.util.function.IntPredicate; @@ -46,14 +45,14 @@ final class RedoPublisher extends AbstractNoHandleSubscribePublisher { } @Override - void handleSubscribe(Subscriber subscriber, ContextMap contextMap, + void handleSubscribe(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { // Current expected behavior is to capture the context on the first subscribe, save it, and re-use it on each // resubscribe. This allows for async context to be shared across each request retry, and follows the same // shared state model as the request object on the client. If copy-on-each-resubscribe is desired this could // be provided by an independent operator, or manually cleared/overwritten. original.delegateSubscribe(new RedoSubscriber<>(terminateOnNextException, new SequentialSubscription(), 0, - subscriber, contextMap, contextProvider, this), contextMap, contextProvider); + subscriber, capturedContext, contextProvider, this), capturedContext, contextProvider); } abstract static class AbstractRedoSubscriber implements Subscriber { @@ -153,15 +152,16 @@ private void handleOnNextException(Throwable cause) { private static final class RedoSubscriber extends AbstractRedoSubscriber { private final RedoPublisher redoPublisher; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; RedoSubscriber(boolean terminateOnNextException, SequentialSubscription subscription, int redoCount, - Subscriber subscriber, ContextMap contextMap, AsyncContextProvider contextProvider, + Subscriber subscriber, CapturedContext capturedContext, + AsyncContextProvider contextProvider, RedoPublisher redoPublisher) { super(terminateOnNextException, subscription, redoCount, subscriber); this.redoPublisher = redoPublisher; - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; } @@ -191,7 +191,7 @@ private void tryRedo(TerminalNotification notification) { if (shouldRedo) { // Either we copy the map up front before subscribe, or we just re-use the same map and let the async // source at the top of the chain reset if necessary. We currently choose the second option. - redoPublisher.original.delegateSubscribe(this, contextMap, contextProvider); + redoPublisher.original.delegateSubscribe(this, capturedContext, contextProvider); } else { notification.terminate(subscriber); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RedoWhenPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RedoWhenPublisher.java index e86bee9d59..d45f56f04d 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RedoWhenPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RedoWhenPublisher.java @@ -20,7 +20,6 @@ import io.servicetalk.concurrent.api.RedoPublisher.AbstractRedoSubscriber; import io.servicetalk.concurrent.internal.SequentialCancellable; import io.servicetalk.concurrent.internal.TerminalNotification; -import io.servicetalk.context.api.ContextMap; import java.util.function.BiFunction; import java.util.function.IntFunction; @@ -63,19 +62,19 @@ final class RedoWhenPublisher extends AbstractNoHandleSubscribePublisher { @Override void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { // Current expected behavior is to capture the context on the first subscribe, save it, and re-use it on each // resubscribe. This allows for async context to be shared across each request retry, and follows the same // shared state model as the request object on the client. If copy-on-each-resubscribe is desired this could // be provided by an independent operator, or manually cleared/overwritten. original.delegateSubscribe(new RedoSubscriber<>(terminateOnNextException, new SequentialSubscription(), 0, - subscriber, contextMap, contextProvider, this), contextMap, contextProvider); + subscriber, capturedContext, contextProvider, this), capturedContext, contextProvider); } private static final class RedoSubscriber extends AbstractRedoSubscriber { private final SequentialCancellable cancellable; private final RedoWhenPublisher redoPublisher; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; private final CompletableSource.Subscriber completableSubscriber = new CompletableSource.Subscriber() { @Override @@ -87,7 +86,7 @@ public void onSubscribe(Cancellable completableCancellable) { public void onComplete() { // Either we copy the map up front before subscribe, or we just re-use the same map and let the async // source at the top of the chain reset if necessary. We currently choose the second option. - redoPublisher.original.delegateSubscribe(RedoSubscriber.this, contextMap, contextProvider); + redoPublisher.original.delegateSubscribe(RedoSubscriber.this, capturedContext, contextProvider); } @Override @@ -104,11 +103,11 @@ public void onError(Throwable t) { }; RedoSubscriber(boolean terminateOnNextException, SequentialSubscription subscription, int redoCount, - Subscriber subscriber, ContextMap contextMap, AsyncContextProvider contextProvider, - RedoWhenPublisher redoPublisher) { + Subscriber subscriber, CapturedContext capturedContext, + AsyncContextProvider contextProvider, RedoWhenPublisher redoPublisher) { super(terminateOnNextException, subscription, redoCount, subscriber); this.redoPublisher = redoPublisher; - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; cancellable = new SequentialCancellable(); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ReduceSingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ReduceSingle.java index d3e03c0ec0..5467063ebd 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ReduceSingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ReduceSingle.java @@ -18,7 +18,6 @@ import io.servicetalk.concurrent.PublisherSource; import io.servicetalk.concurrent.PublisherSource.Subscription; import io.servicetalk.concurrent.internal.DelayedCancellable; -import io.servicetalk.context.api.ContextMap; import java.util.function.BiFunction; import java.util.function.Supplier; @@ -56,17 +55,17 @@ final class ReduceSingle extends AbstractNoHandleSubscribeSingle { @Override void handleSubscribe(final Subscriber singleSubscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { final R r; try { r = resultFactory.get(); } catch (Throwable t) { - deliverOnSubscribeAndOnError(singleSubscriber, contextMap, contextProvider, t); + deliverOnSubscribeAndOnError(singleSubscriber, capturedContext, contextProvider, t); return; } PublisherSource.Subscriber offloadedSubscription = - contextProvider.wrapSubscription(new ReduceSubscriber<>(r, reducer, singleSubscriber), contextMap); - source.delegateSubscribe(offloadedSubscription, contextMap, contextProvider); + contextProvider.wrapSubscription(new ReduceSubscriber<>(r, reducer, singleSubscriber), capturedContext); + source.delegateSubscribe(offloadedSubscription, capturedContext, contextProvider); } private static final class ReduceSubscriber extends DelayedCancellable diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RepeatWhenSingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RepeatWhenSingle.java index 72825b87a0..c9c721bf8c 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RepeatWhenSingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RepeatWhenSingle.java @@ -20,7 +20,6 @@ import io.servicetalk.concurrent.SingleSource; import io.servicetalk.concurrent.internal.FlowControlUtils; import io.servicetalk.concurrent.internal.SequentialCancellable; -import io.servicetalk.context.api.ContextMap; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import javax.annotation.Nullable; @@ -42,9 +41,9 @@ final class RepeatWhenSingle extends AbstractNoHandleSubscribePublisher { @Override void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { try { - subscriber.onSubscribe(new RepeatSubscription<>(this, subscriber, contextMap, contextProvider)); + subscriber.onSubscribe(new RepeatSubscription<>(this, subscriber, capturedContext, contextProvider)); } catch (Throwable cause) { handleExceptionFromOnSubscribe(subscriber, cause); } @@ -60,17 +59,17 @@ private static final class RepeatSubscription implements Subscription { private final RepeatWhenSingle outer; private final SequentialCancellable sequentialCancellable = new SequentialCancellable(); private final Subscriber subscriber; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; private final RepeatSubscriber repeatSubscriber = new RepeatSubscriber(); private volatile long outstandingDemand; private int repeatCount; private RepeatSubscription(final RepeatWhenSingle outer, final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { this.outer = outer; this.subscriber = subscriber; - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; } @@ -82,7 +81,7 @@ public void request(final long n) { if (prev == 0) { // Either we copy the map up front before subscribe, or we just re-use the same map and let the // async source at the top of the chain reset if necessary. We currently choose the second option. - outer.original.delegateSubscribe(repeatSubscriber, contextMap, contextProvider); + outer.original.delegateSubscribe(repeatSubscriber, capturedContext, contextProvider); } } else { requestNInvalid(n); @@ -148,7 +147,7 @@ public void onComplete() { // Either we copy the map up front before subscribe, or we just re-use the same map and // let the async source at the top of the chain reset if necessary. We currently choose // the second option. - outer.original.delegateSubscribe(RepeatSubscriber.this, contextMap, contextProvider); + outer.original.delegateSubscribe(RepeatSubscriber.this, capturedContext, contextProvider); } break; } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RetrySingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RetrySingle.java index c70c88c7b0..cd71fe6ec8 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RetrySingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RetrySingle.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.internal.SequentialCancellable; -import io.servicetalk.context.api.ContextMap; import javax.annotation.Nullable; @@ -39,13 +38,13 @@ final class RetrySingle extends AbstractNoHandleSubscribeSingle { @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { // Current expected behavior is to capture the context on the first subscribe, save it, and re-use it on each // resubscribe. This allows for async context to be shared across each request retry, and follows the same // shared state model as the request object on the client. If copy-on-each-resubscribe is desired this could // be provided by an independent operator, or manually cleared/overwritten. - original.delegateSubscribe(new RetrySubscriber<>(new SequentialCancellable(), this, subscriber, 0, contextMap, - contextProvider), contextMap, contextProvider); + original.delegateSubscribe(new RetrySubscriber<>(new SequentialCancellable(), this, subscriber, 0, capturedContext, + contextProvider), capturedContext, contextProvider); } abstract static class AbstractRetrySubscriber implements Subscriber { @@ -76,15 +75,15 @@ Cancellable decorate(Cancellable cancellable) { private static final class RetrySubscriber extends AbstractRetrySubscriber { private final RetrySingle retrySingle; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; RetrySubscriber(SequentialCancellable sequentialCancellable, RetrySingle retrySingle, - Subscriber target, int retryCount, ContextMap contextMap, + Subscriber target, int retryCount, CapturedContext capturedContext, AsyncContextProvider contextProvider) { super(sequentialCancellable, target, retryCount); this.retrySingle = retrySingle; - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; } @@ -105,7 +104,7 @@ public void onError(Throwable t) { if (shouldRetry) { // Either we copy the map up front before subscribe, or we just re-use the same map and let the async // source at the top of the chain reset if necessary. We currently choose the second option. - retrySingle.original.delegateSubscribe(this, contextMap, contextProvider); + retrySingle.original.delegateSubscribe(this, capturedContext, contextProvider); } else { target.onError(t); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RetryWhenSingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RetryWhenSingle.java index 69a88a25ef..ffd58cbf2c 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RetryWhenSingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RetryWhenSingle.java @@ -18,7 +18,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.CompletableSource; import io.servicetalk.concurrent.internal.SequentialCancellable; -import io.servicetalk.context.api.ContextMap; import javax.annotation.Nullable; @@ -41,19 +40,19 @@ final class RetryWhenSingle extends AbstractNoHandleSubscribeSingle { @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { // Current expected behavior is to capture the context on the first subscribe, save it, and re-use it on each // resubscribe. This allows for async context to be shared across each request retry, and follows the same // shared state model as the request object on the client. If copy-on-each-resubscribe is desired this could // be provided by an independent operator, or manually cleared/overwritten. - original.delegateSubscribe(new RetrySubscriber<>(new SequentialCancellable(), 0, subscriber, contextMap, - contextProvider, this), contextMap, contextProvider); + original.delegateSubscribe(new RetrySubscriber<>(new SequentialCancellable(), 0, subscriber, capturedContext, + contextProvider, this), capturedContext, contextProvider); } private static final class RetrySubscriber extends RetrySingle.AbstractRetrySubscriber { private final SequentialCancellable retrySignalCancellable; private final RetryWhenSingle retrySingle; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; private final CompletableSource.Subscriber completableSubscriber = new CompletableSource.Subscriber() { @Override @@ -65,7 +64,7 @@ public void onSubscribe(Cancellable completableCancellable) { public void onComplete() { // Either we copy the map up front before subscribe, or we just re-use the same map and let the async // source at the top of the chain reset if necessary. We currently choose the second option. - retrySingle.original.delegateSubscribe(RetrySubscriber.this, contextMap, contextProvider); + retrySingle.original.delegateSubscribe(RetrySubscriber.this, capturedContext, contextProvider); } @Override @@ -75,12 +74,12 @@ public void onError(Throwable t) { }; RetrySubscriber(SequentialCancellable cancellable, int redoCount, Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider, + CapturedContext capturedContext, AsyncContextProvider contextProvider, RetryWhenSingle retrySingle) { super(cancellable, subscriber, redoCount); this.retrySingle = retrySingle; retrySignalCancellable = new SequentialCancellable(); - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ScanWithLifetimePublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ScanWithLifetimePublisher.java index 64ecc8f9ca..5321c91742 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ScanWithLifetimePublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ScanWithLifetimePublisher.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.api.ScanWithPublisher.ScanMapperAdapter; import io.servicetalk.concurrent.api.ScanWithPublisher.ScanWithSubscriber; -import io.servicetalk.context.api.ContextMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,15 +47,15 @@ final class ScanWithLifetimePublisher extends AbstractNoHandleSubscribePub } @Override - ContextMap contextForSubscribe(AsyncContextProvider provider) { + CapturedContext contextForSubscribe(AsyncContextProvider provider) { return provider.captureContext(); } @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { original.delegateSubscribe(new ScanWithLifetimeSubscriber<>(subscriber, mapperSupplier.get(), - contextMap, contextProvider), contextMap, contextProvider); + capturedContext, contextProvider), capturedContext, contextProvider); } /** @@ -79,8 +78,8 @@ private static final class ScanWithLifetimeSubscriber extends ScanWithSubs ScanWithLifetimeSubscriber(final Subscriber subscriber, final ScanLifetimeMapper mapper, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { - super(subscriber, mapper, contextProvider, contextMap); + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + super(subscriber, mapper, contextProvider, capturedContext); this.mapper = requireNonNull(mapper); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ScanWithPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ScanWithPublisher.java index bb205adec8..4b5ae0bd0c 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ScanWithPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ScanWithPublisher.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.api.ScanMapper.MappedTerminal; import io.servicetalk.concurrent.internal.FlowControlUtils; -import io.servicetalk.context.api.ContextMap; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.function.BiFunction; @@ -51,15 +50,15 @@ final class ScanWithPublisher extends AbstractNoHandleSubscribePublisher subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { original.delegateSubscribe(new ScanWithSubscriber<>(subscriber, mapperSupplier.get(), - contextProvider, contextMap), contextMap, contextProvider); + contextProvider, capturedContext), capturedContext, contextProvider); } static class ScanWithSubscriber implements Subscriber { @@ -77,7 +76,7 @@ static class ScanWithSubscriber implements Subscriber { private static final long INVALID_DEMAND = -1; private final Subscriber subscriber; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; private final ScanMapper mapper; private volatile long demand; @@ -90,10 +89,10 @@ static class ScanWithSubscriber implements Subscriber { ScanWithSubscriber(final Subscriber subscriber, final ScanMapper mapper, - final AsyncContextProvider contextProvider, final ContextMap contextMap) { + final AsyncContextProvider contextProvider, final CapturedContext capturedContext) { this.subscriber = subscriber; this.contextProvider = contextProvider; - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.mapper = requireNonNull(mapper); } @@ -136,7 +135,7 @@ private void handleInvalidDemand(final long n) { } private Subscriber newOffloadedSubscriber() { - return wrapWithDummyOnSubscribe(subscriber, contextMap, contextProvider); + return wrapWithDummyOnSubscribe(subscriber, capturedContext, contextProvider); } }; } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Single.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Single.java index 193520fe46..6158c45de6 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Single.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Single.java @@ -2671,9 +2671,9 @@ public static Single zipDelayError(Function subscriber, AsyncContextProvider provider) { + final CapturedContext subscribeAndReturnContext(Subscriber subscriber, AsyncContextProvider provider) { requireNonNull(subscriber); - final ContextMap contextMap = contextForSubscribe(provider); - Subscriber wrapped = provider.wrapCancellable(subscriber, contextMap); - if (provider.context() == contextMap) { + final CapturedContext capturedContext = contextForSubscribe(provider); + Subscriber wrapped = provider.wrapCancellable(subscriber, capturedContext); + if (provider.context() == capturedContext) { // No need to wrap as we are sharing the AsyncContext - handleSubscribe(wrapped, contextMap, provider); + handleSubscribe(wrapped, capturedContext, provider); } else { // Ensure that AsyncContext used for handleSubscribe() is the contextMap for the subscribe() - try (Scope ignored = provider.attachContext(contextMap)) { - handleSubscribe(wrapped, contextMap, provider); + try (Scope ignored = capturedContext.restoreContext()) { + handleSubscribe(wrapped, capturedContext, provider); } } - return contextMap; + return capturedContext; } /** * Delegate subscribe calls in an operator chain. This method is used by operators to subscribe to the upstream * source. * @param subscriber the subscriber. - * @param contextMap the {@link ContextMap} to use for this {@link Subscriber}. + * @param capturedContext the {@link ContextMap} to use for this {@link Subscriber}. * @param contextProvider the {@link AsyncContextProvider} used to wrap any objects to preserve {@link ContextMap}. */ final void delegateSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { - handleSubscribe(subscriber, contextMap, contextProvider); + CapturedContext capturedContext, AsyncContextProvider contextProvider) { + handleSubscribe(subscriber, capturedContext, contextProvider); } /** @@ -2718,13 +2718,13 @@ final void delegateSubscribe(Subscriber subscriber, * This method wraps the passed {@link Subscriber}. Operators that do not wish to wrap the passed {@link Subscriber} * can override this method and omit the wrapping. * @param subscriber the subscriber. - * @param contextMap the {@link ContextMap} to use for this {@link Subscriber}. + * @param capturedContext the {@link ContextMap} to use for this {@link Subscriber}. * @param contextProvider the {@link AsyncContextProvider} used to wrap any objects to preserve {@link ContextMap}. */ void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { try { - Subscriber wrapped = contextProvider.wrapSingleSubscriber(subscriber, contextMap); + Subscriber wrapped = contextProvider.wrapSingleSubscriber(subscriber, capturedContext); handleSubscribe(wrapped); } catch (Throwable t) { LOGGER.warn("Unexpected exception from subscribe(), assuming no interaction with the Subscriber.", t); diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleAmbWith.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleAmbWith.java index 25dd73cce6..5b2abd4803 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleAmbWith.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleAmbWith.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.api.AmbSingles.AmbSubscriber; import io.servicetalk.concurrent.api.AmbSingles.State; -import io.servicetalk.context.api.ContextMap; import static io.servicetalk.concurrent.internal.SubscriberUtils.handleExceptionFromOnSubscribe; import static java.util.Objects.requireNonNull; @@ -33,7 +32,7 @@ final class SingleAmbWith extends AbstractNoHandleSubscribeSingle { @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { State state = new State<>(subscriber); try { subscriber.onSubscribe(state); @@ -46,10 +45,10 @@ void handleSubscribe(final Subscriber subscriber, AmbSubscriber originalSubscriber = new AmbSubscriber<>(state); AmbSubscriber ambWithSubscriber = new AmbSubscriber<>(state); state.delayedCancellable(CompositeCancellable.create(originalSubscriber, ambWithSubscriber)); - original.delegateSubscribe(originalSubscriber, contextMap, contextProvider); + original.delegateSubscribe(originalSubscriber, capturedContext, contextProvider); ambWith.subscribeInternal( // If the other Single delivers the result, we should restore the context. - contextProvider.wrapSingleSubscriber(ambWithSubscriber, contextMap)); + contextProvider.wrapSingleSubscriber(ambWithSubscriber, capturedContext)); } catch (Throwable t) { state.tryError(t); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleConcatWithCompletable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleConcatWithCompletable.java index edc95e2153..134ca01260 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleConcatWithCompletable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleConcatWithCompletable.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - import javax.annotation.Nullable; import static java.util.Objects.requireNonNull; @@ -37,8 +35,8 @@ final class SingleConcatWithCompletable extends AbstractCompletableAndSingleC @Override void delegateSubscribeToOriginal(final Subscriber offloadSubscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { - original.delegateSubscribe(new ConcatWithSubscriber<>(offloadSubscriber, next), contextMap, + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + original.delegateSubscribe(new ConcatWithSubscriber<>(offloadSubscriber, next), capturedContext, contextProvider); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleConcatWithPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleConcatWithPublisher.java index 1d7450ad49..36211f919b 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleConcatWithPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleConcatWithPublisher.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.SingleSource; -import io.servicetalk.context.api.ContextMap; import java.util.Objects; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; @@ -44,9 +43,9 @@ final class SingleConcatWithPublisher extends AbstractNoHandleSubscribePublis @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { original.delegateSubscribe(deferSubscribe ? new ConcatDeferNextSubscriber<>(subscriber, next, propagateCancel) : - new ConcatSubscriber<>(subscriber, next, propagateCancel), contextMap, contextProvider); + new ConcatSubscriber<>(subscriber, next, propagateCancel), capturedContext, contextProvider); } private abstract static class AbstractConcatSubscriber extends DelayedCancellableThenSubscription diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleFlatMapCompletable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleFlatMapCompletable.java index 56fc0572a0..73892f9d42 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleFlatMapCompletable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleFlatMapCompletable.java @@ -18,7 +18,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.SingleSource; import io.servicetalk.concurrent.internal.SequentialCancellable; -import io.servicetalk.context.api.ContextMap; import java.util.function.Function; import javax.annotation.Nullable; @@ -39,24 +38,24 @@ final class SingleFlatMapCompletable extends AbstractNoHandleSubscribeComplet @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { - original.delegateSubscribe(new SubscriberImpl<>(subscriber, nextFactory, contextMap, contextProvider), - contextMap, contextProvider); + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + original.delegateSubscribe(new SubscriberImpl<>(subscriber, nextFactory, capturedContext, contextProvider), + capturedContext, contextProvider); } private static final class SubscriberImpl implements SingleSource.Subscriber, Subscriber { private final Subscriber subscriber; private final Function nextFactory; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; @Nullable private SequentialCancellable sequentialCancellable; SubscriberImpl(Subscriber subscriber, Function nextFactory, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { this.subscriber = subscriber; this.nextFactory = nextFactory; - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; } @@ -87,7 +86,7 @@ public void onSuccess(@Nullable T result) { // The static AsyncContext should be the same as the original contextMap at this point because we are // being notified in the Subscriber path, but we make sure that it is restored after the asynchronous // boundary and use an isolated copy to subscribe to the new source. - next.subscribeInternal(contextProvider.wrapCompletableSubscriber(this, contextMap)); + next.subscribeInternal(contextProvider.wrapCompletableSubscriber(this, capturedContext)); } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleFlatMapPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleFlatMapPublisher.java index 3e98c09d9f..50b9dc04e4 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleFlatMapPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleFlatMapPublisher.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.SingleSource; -import io.servicetalk.context.api.ContextMap; import java.util.function.Function; import javax.annotation.Nullable; @@ -38,24 +37,24 @@ final class SingleFlatMapPublisher extends AbstractNoHandleSubscribePublis @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { original.delegateSubscribe(new SubscriberImpl<>(subscriber, nextFactory, - contextMap, contextProvider), contextMap, contextProvider); + capturedContext, contextProvider), capturedContext, contextProvider); } private static final class SubscriberImpl extends DelayedCancellableThenSubscription implements SingleSource.Subscriber, Subscriber { private final Subscriber subscriber; private final Function> nextFactory; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; SubscriberImpl(Subscriber subscriber, Function> nextFactory, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { this.subscriber = subscriber; this.nextFactory = requireNonNull(nextFactory); - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; } @@ -83,7 +82,7 @@ public void onSuccess(@Nullable T result) { // being notified in the Subscriber path, but we make sure that it is restored after the asynchronous // boundary and explicitly use it to subscribe. next.subscribeInternal((Subscriber) - contextProvider.wrapPublisherSubscriber(this, contextMap)); + contextProvider.wrapPublisherSubscriber(this, capturedContext)); } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java index 186c265fef..02a7e771bd 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java @@ -29,16 +29,24 @@ final class SingleSetContextOnSubscribe extends AbstractNoHandleSubscribeSing } @Override - ContextMap contextForSubscribe(AsyncContextProvider provider) { - return context; + CapturedContext contextForSubscribe(AsyncContextProvider provider) { + CapturedContext parentContext = super.contextForSubscribe(provider); + return () -> { + Scope outer = parentContext.restoreContext(); + Scope inner = ContextMapThreadLocal.attachContext(context); + return () -> { + inner.close(); + outer.close(); + }; + }; } @Override void handleSubscribe(final Subscriber singleSubscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { // This operator currently only targets the subscribe method. Given this limitation if we try to change the // ContextMap now it is possible that operators downstream in the subscribe call stack may have modified // the ContextMap and we don't want to discard those changes by using a different ContextMap. - original.handleSubscribe(singleSubscriber, contextMap, contextProvider); + original.handleSubscribe(singleSubscriber, capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleShareContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleShareContextOnSubscribe.java index 2cdc1d2586..5c1c273a40 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleShareContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleShareContextOnSubscribe.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - final class SingleShareContextOnSubscribe extends AbstractNoHandleSubscribeSingle { private final Single original; @@ -25,16 +23,16 @@ final class SingleShareContextOnSubscribe extends AbstractNoHandleSubscribeSi } @Override - ContextMap contextForSubscribe(AsyncContextProvider provider) { + CapturedContext contextForSubscribe(AsyncContextProvider provider) { return provider.captureContext(); } @Override void handleSubscribe(final Subscriber singleSubscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { // This operator currently only targets the subscribe method. Given this limitation if we try to change the // ContextMap now it is possible that operators downstream in the subscribe call stack may have modified // the ContextMap and we don't want to discard those changes by using a different ContextMap. - original.handleSubscribe(singleSubscriber, contextMap, contextProvider); + original.handleSubscribe(singleSubscriber, capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToCompletable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToCompletable.java index 6a422d812b..6e2ca18d03 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToCompletable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToCompletable.java @@ -17,7 +17,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.SingleSource; -import io.servicetalk.context.api.ContextMap; import javax.annotation.Nullable; @@ -32,7 +31,7 @@ final class SingleToCompletable extends AbstractNoHandleSubscribeCompletable @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { // We are not modifying the Cancellable between sources, so we do not need to take care of offloading between // the sources (in this operator). If the Cancellable is configured to be offloaded, it will be done when the // resulting Completable is subscribed. Since, it is the same source, just viewed as a Completable, there is no @@ -52,6 +51,6 @@ public void onSuccess(@Nullable T result) { public void onError(Throwable t) { subscriber.onError(t); } - }, contextMap, contextProvider); + }, capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToCompletableFuture.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToCompletableFuture.java index d09cb8ba99..612da90dad 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToCompletableFuture.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToCompletableFuture.java @@ -18,6 +18,7 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.SingleSource.Subscriber; import io.servicetalk.concurrent.internal.SequentialCancellable; +import io.servicetalk.context.api.ContextMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToPublisher.java index 39bee48e30..458ca87292 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToPublisher.java @@ -18,7 +18,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.SingleSource; import io.servicetalk.concurrent.internal.SequentialCancellable; -import io.servicetalk.context.api.ContextMap; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import javax.annotation.Nullable; @@ -41,9 +40,9 @@ final class SingleToPublisher extends AbstractNoHandleSubscribePublisher { @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { - original.delegateSubscribe(new ConversionSubscriber<>(subscriber, contextMap, contextProvider), - contextMap, contextProvider); + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + original.delegateSubscribe(new ConversionSubscriber<>(subscriber, capturedContext, contextProvider), + capturedContext, contextProvider); } private static final class ConversionSubscriber extends SequentialCancellable @@ -56,7 +55,7 @@ private static final class ConversionSubscriber extends SequentialCancellable private static final AtomicIntegerFieldUpdater stateUpdater = newUpdater(ConversionSubscriber.class, "state"); private final Subscriber subscriber; - private final ContextMap contextMap; + private final CapturedContext capturedContext; private final AsyncContextProvider contextProvider; @Nullable @@ -64,9 +63,9 @@ private static final class ConversionSubscriber extends SequentialCancellable private volatile int state; ConversionSubscriber(Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { this.subscriber = subscriber; - this.contextMap = contextMap; + this.capturedContext = capturedContext; this.contextProvider = contextProvider; } @@ -112,7 +111,7 @@ public void request(long n) { // Single Subscriber methods which is correctly offloaded. This is the case where we invoke the // Subscriber directly, hence we explicitly wrap with Async context. terminateSuccessfully(result, wrapWithDummyOnSubscribe(subscriber, - contextMap, contextProvider)); + capturedContext, contextProvider)); return; } else if (cState == STATE_IDLE && stateUpdater.compareAndSet(this, STATE_IDLE, STATE_REQUESTED)) { @@ -122,7 +121,7 @@ public void request(long n) { } } } else if (stateUpdater.getAndSet(this, STATE_TERMINATED) != STATE_TERMINATED) { - Subscriber wrapped = wrapWithDummyOnSubscribe(subscriber, contextMap, contextProvider); + Subscriber wrapped = wrapWithDummyOnSubscribe(subscriber, capturedContext, contextProvider); try { // offloadSubscriber before cancellation so that signalOffloader does not exit on seeing a cancel. cancel(); diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TaskBasedAsyncCompletableOperator.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TaskBasedAsyncCompletableOperator.java index 84f50852b9..8530c4e96f 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TaskBasedAsyncCompletableOperator.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TaskBasedAsyncCompletableOperator.java @@ -16,7 +16,6 @@ package io.servicetalk.concurrent.api; import io.servicetalk.concurrent.Cancellable; -import io.servicetalk.context.api.ContextMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,8 +64,8 @@ final io.servicetalk.concurrent.Executor executor() { @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { - original.delegateSubscribe(subscriber, contextMap, contextProvider); + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + original.delegateSubscribe(subscriber, capturedContext, contextProvider); } abstract static class AbstractOffloadedSingleValueSubscriber { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TaskBasedAsyncPublisherOperator.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TaskBasedAsyncPublisherOperator.java index 71fc2d57be..8233c0c99d 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TaskBasedAsyncPublisherOperator.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TaskBasedAsyncPublisherOperator.java @@ -18,7 +18,6 @@ import io.servicetalk.concurrent.internal.FlowControlUtils; import io.servicetalk.concurrent.internal.QueueFullException; import io.servicetalk.concurrent.internal.TerminalNotification; -import io.servicetalk.context.api.ContextMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -81,8 +80,8 @@ final io.servicetalk.concurrent.Executor executor() { @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { - original.delegateSubscribe(subscriber, contextMap, contextProvider); + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + original.delegateSubscribe(subscriber, capturedContext, contextProvider); } /** diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TaskBasedAsyncSingleOperator.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TaskBasedAsyncSingleOperator.java index fa5fa14c42..5ac363ed9c 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TaskBasedAsyncSingleOperator.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TaskBasedAsyncSingleOperator.java @@ -19,7 +19,6 @@ import io.servicetalk.concurrent.api.TaskBasedAsyncCompletableOperator.AbstractOffloadedSingleValueSubscriber; import io.servicetalk.concurrent.api.TaskBasedAsyncCompletableOperator.OffloadedCancellable; import io.servicetalk.concurrent.internal.TerminalNotification; -import io.servicetalk.context.api.ContextMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,8 +75,8 @@ final io.servicetalk.concurrent.Executor executor() { @Override void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { - original.delegateSubscribe(subscriber, contextMap, contextProvider); + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + original.delegateSubscribe(subscriber, capturedContext, contextProvider); } static final class SingleSubscriberOffloadedTerminals extends AbstractOffloadedSingleValueSubscriber diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutCompletable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutCompletable.java index 6ca0f013cd..df45191a2e 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutCompletable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutCompletable.java @@ -16,7 +16,6 @@ package io.servicetalk.concurrent.api; import io.servicetalk.concurrent.Cancellable; -import io.servicetalk.context.api.ContextMap; import java.time.Duration; import java.util.concurrent.TimeUnit; @@ -56,10 +55,10 @@ final class TimeoutCompletable extends AbstractNoHandleSubscribeCompletable { @Override protected void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { original.delegateSubscribe( - TimeoutSubscriber.newInstance(this, subscriber, contextMap, contextProvider), - contextMap, contextProvider); + TimeoutSubscriber.newInstance(this, subscriber, capturedContext, contextProvider), + capturedContext, contextProvider); } private static final class TimeoutSubscriber implements Subscriber, Cancellable { @@ -91,7 +90,7 @@ private TimeoutSubscriber(TimeoutCompletable parent, Subscriber target, } static TimeoutSubscriber newInstance(TimeoutCompletable parent, Subscriber target, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { TimeoutSubscriber s = new TimeoutSubscriber(parent, target, contextProvider); Cancellable localTimerCancellable; try { @@ -108,7 +107,7 @@ static TimeoutSubscriber newInstance(TimeoutCompletable parent, Subscriber targe localTimerCancellable = IGNORE_CANCEL; // We must set this to ignore so there are no further interactions with Subscriber in the future. s.cancellable = LOCAL_IGNORE_CANCEL; - deliverOnSubscribeAndOnError(target, contextMap, contextProvider, cause); + deliverOnSubscribeAndOnError(target, capturedContext, contextProvider, cause); } s.timerCancellable = localTimerCancellable; return s; diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutDemandPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutDemandPublisher.java index 349ea6c5e0..0c8bb71a52 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutDemandPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutDemandPublisher.java @@ -19,7 +19,6 @@ import io.servicetalk.concurrent.Executor; import io.servicetalk.concurrent.api.TimeoutPublisher.AbstractTimeoutSubscriber; import io.servicetalk.concurrent.internal.FlowControlUtils; -import io.servicetalk.context.api.ContextMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -52,10 +51,10 @@ final class TimeoutDemandPublisher extends AbstractNoHandleSubscribePublisher @Override void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { original.delegateSubscribe( - TimeoutDemandSubscriber.newInstance(this, subscriber, contextMap, contextProvider), - contextMap, contextProvider); + TimeoutDemandSubscriber.newInstance(this, subscriber, capturedContext, contextProvider), + capturedContext, contextProvider); } private static final class TimeoutDemandSubscriber extends AbstractTimeoutSubscriber { @@ -80,10 +79,10 @@ private TimeoutDemandSubscriber(TimeoutDemandPublisher parent, static TimeoutDemandSubscriber newInstance(TimeoutDemandPublisher parent, Subscriber target, - ContextMap contextMap, + CapturedContext capturedContext, AsyncContextProvider contextProvider) { TimeoutDemandSubscriber s = new TimeoutDemandSubscriber<>(parent, target, contextProvider); - s.initTimer(parent.durationNs, parent.timeoutExecutor, contextMap); + s.initTimer(parent.durationNs, parent.timeoutExecutor, capturedContext); return s; } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutPublisher.java index da268c8ae1..d3c213c7b4 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutPublisher.java @@ -19,7 +19,6 @@ import io.servicetalk.concurrent.Executor; import io.servicetalk.concurrent.internal.ConcurrentSubscription; import io.servicetalk.concurrent.internal.ConcurrentTerminalSubscriber; -import io.servicetalk.context.api.ContextMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -76,10 +75,10 @@ final class TimeoutPublisher extends AbstractNoHandleSubscribePublisher { @Override void handleSubscribe(Subscriber subscriber, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { original.delegateSubscribe( - TimeoutSubscriber.newInstance(this, subscriber, contextMap, contextProvider), - contextMap, contextProvider); + TimeoutSubscriber.newInstance(this, subscriber, capturedContext, contextProvider), + capturedContext, contextProvider); } abstract static class AbstractTimeoutSubscriber implements Subscriber, Subscription { @@ -110,7 +109,7 @@ abstract static class AbstractTimeoutSubscriber implements Subscriber, Sub } final void initTimer(long durationNs, io.servicetalk.concurrent.Executor timeoutExecutor, - ContextMap contextMap) { + CapturedContext capturedContext) { try { // CAS is just in case the timer fired, the timerFires method schedule a new timer before this thread is // able to set the initial timer value. In this case we don't want to overwrite the active timer. @@ -124,7 +123,7 @@ final void initTimer(long durationNs, io.servicetalk.concurrent.Executor timeout timerCancellableUpdater.compareAndSet(this, null, requireNonNull( timeoutExecutor.schedule(this::timerFires, durationNs, NANOSECONDS))); } catch (Throwable cause) { - handleConstructorException(this, contextMap, contextProvider, cause); + handleConstructorException(this, capturedContext, contextProvider, cause); } } @@ -200,12 +199,12 @@ final void processTimeout(Throwable cause) { * @param cause The exception. */ private static void handleConstructorException( - AbstractTimeoutSubscriber s, ContextMap contextMap, AsyncContextProvider contextProvider, + AbstractTimeoutSubscriber s, CapturedContext capturedContext, AsyncContextProvider contextProvider, Throwable cause) { // We must set local state so there are no further interactions with Subscriber in the future. s.timerCancellable = LOCAL_IGNORE_CANCEL; s.subscription = EMPTY_SUBSCRIPTION; - deliverOnSubscribeAndOnError(s.target, contextMap, contextProvider, cause); + deliverOnSubscribeAndOnError(s.target, capturedContext, contextProvider, cause); } } @@ -245,10 +244,10 @@ private TimeoutSubscriber(TimeoutPublisher parent, static TimeoutSubscriber newInstance(TimeoutPublisher parent, Subscriber target, - ContextMap contextMap, + CapturedContext capturedContext, AsyncContextProvider contextProvider) { TimeoutSubscriber s = new TimeoutSubscriber<>(parent, target, contextProvider); - s.initTimer(parent.durationNs, parent.timeoutExecutor, contextMap); + s.initTimer(parent.durationNs, parent.timeoutExecutor, capturedContext); return s; } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutSingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutSingle.java index 0c104bc415..efc092e425 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutSingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutSingle.java @@ -16,7 +16,6 @@ package io.servicetalk.concurrent.api; import io.servicetalk.concurrent.Cancellable; -import io.servicetalk.context.api.ContextMap; import java.time.Duration; import java.util.concurrent.TimeUnit; @@ -56,10 +55,10 @@ final class TimeoutSingle extends AbstractNoHandleSubscribeSingle { @Override protected void handleSubscribe(final Subscriber subscriber, - final ContextMap contextMap, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { original.delegateSubscribe( - TimeoutSubscriber.newInstance(this, subscriber, contextMap, contextProvider), - contextMap, contextProvider); + TimeoutSubscriber.newInstance(this, subscriber, capturedContext, contextProvider), + capturedContext, contextProvider); } private static final class TimeoutSubscriber implements Subscriber, Cancellable { @@ -93,7 +92,7 @@ private TimeoutSubscriber(TimeoutSingle parent, Subscriber target, } static TimeoutSubscriber newInstance(TimeoutSingle parent, Subscriber target, - ContextMap contextMap, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, AsyncContextProvider contextProvider) { TimeoutSubscriber s = new TimeoutSubscriber<>(parent, target, contextProvider); Cancellable localTimerCancellable; try { @@ -110,7 +109,7 @@ static TimeoutSubscriber newInstance(TimeoutSingle parent, Subscriber< localTimerCancellable = IGNORE_CANCEL; // We must set this to ignore so there are no further interactions with Subscriber in the future. s.cancellable = LOCAL_IGNORE_CANCEL; - deliverOnSubscribeAndOnError(target, contextMap, contextProvider, cause); + deliverOnSubscribeAndOnError(target, capturedContext, contextProvider, cause); } s.timerCancellable = localTimerCancellable; return s; diff --git a/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java b/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java index dcb6997400..5f19f9ce10 100644 --- a/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java +++ b/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java @@ -505,7 +505,7 @@ void testWrapFunctions() throws Exception { Function f = INSTANCE.wrapFunction(v -> { collector.complete(AsyncContext.context()); return v; - }, AsyncContext.context()); + }, AsyncContext.provider().captureContext()); executor.execute(() -> f.apply(null)); }) .verifyContext(verifier); @@ -513,7 +513,7 @@ void testWrapFunctions() throws Exception { new ContextCapturer() .runAndWait(collector -> { Consumer c = INSTANCE.wrapConsumer(v -> collector.complete(AsyncContext.context()), - AsyncContext.context()); + AsyncContext.provider().captureContext()); executor.execute(() -> c.accept(null)); }) .verifyContext(verifier); @@ -523,7 +523,7 @@ void testWrapFunctions() throws Exception { BiFunction bf = INSTANCE.wrapBiFunction((v1, v2) -> { collector.complete(AsyncContext.context()); return v1; - }, AsyncContext.context()); + }, AsyncContext.provider().captureContext()); executor.execute(() -> bf.apply(null, null)); }) .verifyContext(verifier); @@ -532,7 +532,7 @@ void testWrapFunctions() throws Exception { .runAndWait(collector -> { BiConsumer bc = INSTANCE.wrapBiConsumer((v1, v2) -> { collector.complete(AsyncContext.context()); - }, AsyncContext.context()); + }, AsyncContext.provider().captureContext()); executor.execute(() -> bc.accept(null, null)); }) .verifyContext(verifier); diff --git a/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/SubscribeThrowsTest.java b/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/SubscribeThrowsTest.java index c09b2c0e1c..28749cb905 100644 --- a/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/SubscribeThrowsTest.java +++ b/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/SubscribeThrowsTest.java @@ -78,7 +78,7 @@ protected void handleSubscribe(final CompletableSource.Subscriber subscriber) { } }; AsyncContextProvider provider = AsyncContext.provider(); - c.delegateSubscribe(subscriber, provider.context(), provider); + c.delegateSubscribe(subscriber, provider.captureContext(), provider); verify(subscriber).onError(DELIBERATE_EXCEPTION); } } From 115667ed1a613adc9d6cc74d3f09fbfe19d2e3c4 Mon Sep 17 00:00:00 2001 From: Bryce Anderson Date: Wed, 5 Feb 2025 10:21:10 -0700 Subject: [PATCH 02/14] Add a copyright header --- .../concurrent/api/ContextCaptureProvider.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java index bcf2b4a862..b7e6107d77 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java @@ -1,5 +1,23 @@ +/* + * Copyright © 2025 Apple Inc. and the ServiceTalk project authors + * + * Licensed 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 io.servicetalk.concurrent.api; +/** + * Functionality related to capturing thread-local like context for later restoration across async boundaries. + */ interface ContextCaptureProvider { /** From d73d0c0634cfd753b5c5f83aa05e746dd856a2bf Mon Sep 17 00:00:00 2001 From: Bryce Anderson Date: Wed, 5 Feb 2025 15:19:25 -0700 Subject: [PATCH 03/14] Remove allocations of the basic CapturedContext implementation --- .../concurrent/api/CapturedContext.java | 8 ++ .../api/CompletableSetContextOnSubscribe.java | 22 +++-- .../concurrent/api/ContextMapThreadLocal.java | 90 ++++++++++++++----- .../api/ContextPreservingBiFunction.java | 12 ++- .../concurrent/api/CopyOnWriteContextMap.java | 20 ++++- .../CustomCaptureAsyncContextProvider.java | 10 +-- .../api/DefaultAsyncContextProvider.java | 12 +-- .../api/NoopAsyncContextProvider.java | 12 ++- .../api/PublisherSetContextOnSubscribe.java | 22 +++-- .../api/SingleSetContextOnSubscribe.java | 22 +++-- 10 files changed, 167 insertions(+), 63 deletions(-) diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java index a1f2d43487..40ccc4ffee 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java @@ -15,12 +15,20 @@ */ package io.servicetalk.concurrent.api; +import io.servicetalk.context.api.ContextMap; + /** * An interface representing the restoration of the thread-local like context that can be restored later * during an async operation. */ interface CapturedContext { + /** + * The {@link ContextMap} that was captured as part of the context. + * @return + */ + ContextMap captured(); + /** * Restore the thread-local like context. * @return a {@link Scope} that will revert the restoration and return the thread-local like state to the state diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java index a812dca3b2..ea6f31d861 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java @@ -31,13 +31,21 @@ final class CompletableSetContextOnSubscribe extends AbstractNoHandleSubscribeCo @Override CapturedContext contextForSubscribe(AsyncContextProvider provider) { CapturedContext parentContext = super.contextForSubscribe(provider); - return () -> { - Scope outer = parentContext.restoreContext(); - Scope inner = ContextMapThreadLocal.attachContext(context); - return () -> { - inner.close(); - outer.close(); - }; + return new CapturedContext() { + @Override + public ContextMap captured() { + return context; + } + + @Override + public Scope restoreContext() { + Scope outer = parentContext.restoreContext(); + Scope inner = ContextMapThreadLocal.attachContext(context); + return () -> { + inner.close(); + outer.close(); + }; + } }; } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextMapThreadLocal.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextMapThreadLocal.java index 23bc9a6d4e..0243728285 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextMapThreadLocal.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextMapThreadLocal.java @@ -1,3 +1,18 @@ +/* + * Copyright © 2025 Apple Inc. and the ServiceTalk project authors + * + * Licensed 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 io.servicetalk.concurrent.api; import io.servicetalk.context.api.ContextMap; @@ -13,10 +28,9 @@ final class ContextMapThreadLocal { private static final Logger LOGGER = LoggerFactory.getLogger(ContextMapThreadLocal.class); - protected static final ThreadLocal CONTEXT_THREAD_LOCAL = + private static final ThreadLocal CONTEXT_THREAD_LOCAL = withInitial(ContextMapThreadLocal::newContextMap); - @Nonnull static ContextMap context() { final Thread t = Thread.currentThread(); if (t instanceof ContextMapHolder) { @@ -32,46 +46,78 @@ static ContextMap context() { } } + static CapturedContext captureContext() { + return toCaptureContext(context()); + } + + static CapturedContext captureContextCopy() { + return toCaptureContext(context().copy()); + } + static Scope attachContext(ContextMap contextMap) { + ContextMap prev = exchangeContext(contextMap); + return !LOGGER.isDebugEnabled() && prev instanceof Scope ? (Scope) prev : () -> detachContext(contextMap, prev); + } + + // Used for `CopyOnWriteContextMap.close()` + static void setContext(ContextMap contextMap) { final Thread currentThread = Thread.currentThread(); if (currentThread instanceof ContextMapHolder) { final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; - ContextMap prev = asyncContextMapHolder.context(); asyncContextMapHolder.context(contextMap); - return prev == null ? Scope.NOOP : () -> detachContext(contextMap, prev); } else { - return slowPathSetContext(contextMap); + CONTEXT_THREAD_LOCAL.set(contextMap); } } - private static Scope slowPathSetContext(ContextMap contextMap) { - ContextMap prev = CONTEXT_THREAD_LOCAL.get(); - CONTEXT_THREAD_LOCAL.set(contextMap); - return () -> detachContext(contextMap, prev); - } - - private static void detachContext(ContextMap expectedContext, ContextMap toRestore) { + // Used for CaptureContext operations + private static ContextMap exchangeContext(ContextMap contextMap) { final Thread currentThread = Thread.currentThread(); + ContextMap result; if (currentThread instanceof ContextMapHolder) { final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; - ContextMap current = asyncContextMapHolder.context(); - if (current != expectedContext) { - LOGGER.warn("Current context didn't match the expected context. current: {}, expected: {}", - current, expectedContext); + result = asyncContextMapHolder.context(); + if (result == null) { + result = newContextMap(); } - asyncContextMapHolder.context(toRestore); + asyncContextMapHolder.context(contextMap); } else { - slowPathDetachContext(expectedContext, toRestore); + result = CONTEXT_THREAD_LOCAL.get(); + CONTEXT_THREAD_LOCAL.set(contextMap); + } + return result; + } + + private static CapturedContext toCaptureContext(ContextMap contextMap) { + return contextMap instanceof CapturedContext ? + (CapturedContext) contextMap : new CapturedContextImpl(contextMap); + } + + private static final class CapturedContextImpl implements CapturedContext { + + private final ContextMap contextMap; + + CapturedContextImpl(ContextMap contextMap) { + this.contextMap = contextMap; + } + + @Override + public ContextMap captured() { + return contextMap; + } + + @Override + public Scope restoreContext() { + return attachContext(contextMap); } } - private static void slowPathDetachContext(ContextMap expectedContext, ContextMap toRestore) { - ContextMap current = CONTEXT_THREAD_LOCAL.get(); + private static void detachContext(ContextMap expectedContext, ContextMap toRestore) { + ContextMap current = exchangeContext(toRestore); if (current != expectedContext) { - LOGGER.warn("Current context didn't match the expected context. current: {}, expected: {}", + LOGGER.debug("Current context didn't match the expected context. current: {}, expected: {}", current, expectedContext); } - CONTEXT_THREAD_LOCAL.set(toRestore); } private static ContextMap newContextMap() { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java index b24555f831..2882735156 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java @@ -15,22 +15,28 @@ */ package io.servicetalk.concurrent.api; +import io.servicetalk.context.api.ContextMap; + import java.util.function.BiFunction; import static java.util.Objects.requireNonNull; final class ContextPreservingBiFunction implements BiFunction { - private final CapturedContext saved; + + // TODO: remove once we can get the java agents onto the new API. + private final ContextMap saved; + private final CapturedContext capturedContext; private final BiFunction delegate; ContextPreservingBiFunction(BiFunction delegate, CapturedContext capturedContext) { - this.saved = requireNonNull(capturedContext); + this.capturedContext = requireNonNull(capturedContext); this.delegate = requireNonNull(delegate); + this.saved = capturedContext.captured(); } @Override public V apply(T t, U u) { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { return delegate.apply(t, u); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java index 8ae44657dc..f4e5861cba 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java @@ -39,7 +39,7 @@ * {@link ContextMap.Key}-value entries in a single {@link ContextMap}. Common {@link ContextMap.Key}-value entries are * (tracing, MDC, auth, 3-custom user entries). */ -final class CopyOnWriteContextMap implements ContextMap { +final class CopyOnWriteContextMap implements ContextMap, Scope, CapturedContext { private static final AtomicReferenceFieldUpdater mapUpdater = AtomicReferenceFieldUpdater.newUpdater(CopyOnWriteContextMap.class, CopyContextMap.class, "map"); @@ -2973,4 +2973,22 @@ CopyContextMap build() { return new SevenOrMoreContextMap(Arrays.copyOf(pairs, index)); } } + + // CapturedContext methods + + @Override + public ContextMap captured() { + return this; + } + + @Override + public Scope restoreContext() { + return ContextMapThreadLocal.attachContext(this); + } + + // Scope method + @Override + public void close() { + ContextMapThreadLocal.setContext(this); + } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java index cbee6bd520..0725c3a4d1 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java @@ -4,19 +4,19 @@ final class CustomCaptureAsyncContextProvider extends AbstractAsyncContextProvider { - private final ContextCaptureProvider contextCaptureProvider; + private final ContextCaptureProvider delegate; - CustomCaptureAsyncContextProvider(ContextCaptureProvider contextCaptureProvider) { - this.contextCaptureProvider = requireNonNull(contextCaptureProvider); + CustomCaptureAsyncContextProvider(ContextCaptureProvider delegate) { + this.delegate = requireNonNull(delegate); } @Override public CapturedContext captureContext() { - return contextCaptureProvider.captureContext(); + return delegate.captureContext(); } @Override public CapturedContext captureContextCopy() { - return contextCaptureProvider.captureContextCopy(); + return delegate.captureContextCopy(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java index 1117a6970b..58248aaa8a 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java @@ -15,8 +15,6 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - final class DefaultAsyncContextProvider extends AbstractAsyncContextProvider { static final DefaultAsyncContextProvider INSTANCE = new DefaultAsyncContextProvider(); @@ -27,17 +25,11 @@ private DefaultAsyncContextProvider() { @Override public CapturedContext captureContext() { - ContextMap current = context(); - return () -> ContextMapThreadLocal.attachContext(current); + return ContextMapThreadLocal.captureContext(); } @Override public CapturedContext captureContextCopy() { - ContextMap currentCopy = context().copy(); - return () -> ContextMapThreadLocal.attachContext(currentCopy); - } - - private static ContextMap newContextMap() { - return new CopyOnWriteContextMap(); + return ContextMapThreadLocal.captureContextCopy(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java index 62f603c3d9..6b94018461 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java @@ -37,7 +37,17 @@ final class NoopAsyncContextProvider implements AsyncContextProvider { static final AsyncContextProvider INSTANCE = new NoopAsyncContextProvider(); - private static final CapturedContext NOOP_SAVED_CONTEXT = () -> Scope.NOOP; + private static final CapturedContext NOOP_SAVED_CONTEXT = new CapturedContext() { + @Override + public ContextMap captured() { + return NoopContextMap.INSTANCE; + } + + @Override + public Scope restoreContext() { + return Scope.NOOP; + } + }; private NoopAsyncContextProvider() { // singleton diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java index 1b2e296962..cb940233bf 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java @@ -31,13 +31,21 @@ final class PublisherSetContextOnSubscribe extends AbstractNoHandleSubscribeP @Override CapturedContext contextForSubscribe(AsyncContextProvider provider) { CapturedContext parentContext = super.contextForSubscribe(provider); - return () -> { - Scope outer = parentContext.restoreContext(); - Scope inner = ContextMapThreadLocal.attachContext(context); - return () -> { - inner.close(); - outer.close(); - }; + return new CapturedContext() { + @Override + public ContextMap captured() { + return context; + } + + @Override + public Scope restoreContext() { + Scope outer = parentContext.restoreContext(); + Scope inner = ContextMapThreadLocal.attachContext(context); + return () -> { + inner.close(); + outer.close(); + }; + } }; } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java index 02a7e771bd..7728e7986d 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java @@ -31,13 +31,21 @@ final class SingleSetContextOnSubscribe extends AbstractNoHandleSubscribeSing @Override CapturedContext contextForSubscribe(AsyncContextProvider provider) { CapturedContext parentContext = super.contextForSubscribe(provider); - return () -> { - Scope outer = parentContext.restoreContext(); - Scope inner = ContextMapThreadLocal.attachContext(context); - return () -> { - inner.close(); - outer.close(); - }; + return new CapturedContext() { + @Override + public ContextMap captured() { + return context; + } + + @Override + public Scope restoreContext() { + Scope outer = parentContext.restoreContext(); + Scope inner = ContextMapThreadLocal.attachContext(context); + return () -> { + inner.close(); + outer.close(); + }; + } }; } From 76ca6cd7a0c62aa1862ffdda0477557ed85b0115 Mon Sep 17 00:00:00 2001 From: Bryce Anderson Date: Wed, 5 Feb 2025 18:21:13 -0700 Subject: [PATCH 04/14] merge --- .../api/AbstractAsyncContextProvider.java | 265 ------------- .../concurrent/api/AsyncContextProvider.java | 6 +- .../api/CompletableSetContextOnSubscribe.java | 2 +- .../concurrent/api/ConcurrentContextMap.java | 181 --------- .../api/ContextCaptureProvider.java | 7 + .../concurrent/api/ContextMapThreadLocal.java | 252 ++++++------- .../concurrent/api/CopyOnWriteContextMap.java | 36 +- .../CustomCaptureAsyncContextProvider.java | 2 +- .../api/DefaultAsyncContextProvider.java | 350 +++++++++++++++++- .../api/NoopAsyncContextProvider.java | 11 + .../api/PublisherSetContextOnSubscribe.java | 2 +- .../api/SingleSetContextOnSubscribe.java | 2 +- 12 files changed, 515 insertions(+), 601 deletions(-) delete mode 100644 servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsyncContextProvider.java delete mode 100644 servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ConcurrentContextMap.java diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsyncContextProvider.java deleted file mode 100644 index cdc1152a5a..0000000000 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractAsyncContextProvider.java +++ /dev/null @@ -1,265 +0,0 @@ -/* - * Copyright © 2018-2019, 2021 Apple Inc. and the ServiceTalk project authors - * - * Licensed 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 io.servicetalk.concurrent.api; - -import io.servicetalk.concurrent.CompletableSource; -import io.servicetalk.concurrent.PublisherSource.Subscriber; -import io.servicetalk.concurrent.SingleSource; -import io.servicetalk.context.api.ContextMap; - -import javax.annotation.Nonnull; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ScheduledExecutorService; -import java.util.function.BiConsumer; -import java.util.function.BiFunction; -import java.util.function.Consumer; -import java.util.function.Function; - -abstract class AbstractAsyncContextProvider implements AsyncContextProvider { - - AbstractAsyncContextProvider() { - // singleton - } - - @Nonnull - @Override - public final ContextMap context() { - return ContextMapThreadLocal.context(); - } - - @Override - public final CompletableSource.Subscriber wrapCancellable(final CompletableSource.Subscriber subscriber, - final CapturedContext context) { - if (subscriber instanceof ContextPreservingCompletableSubscriber) { - final ContextPreservingCompletableSubscriber s = (ContextPreservingCompletableSubscriber) subscriber; - if (s.saved == context) { - return subscriber instanceof ContextPreservingCompletableSubscriberAndCancellable ? subscriber : - new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber && - ((ContextPreservingCancellableCompletableSubscriber) subscriber).saved == context) { - // no need to check for instanceof ContextPreservingCompletableSubscriberAndCancellable, because - // it extends from ContextPreservingSingleSubscriber. - return subscriber; - } - return new ContextPreservingCancellableCompletableSubscriber(subscriber, context); - } - - @Override - public final CompletableSource.Subscriber wrapCompletableSubscriber(final CompletableSource.Subscriber subscriber, - final CapturedContext context) { - if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber) { - final ContextPreservingCancellableCompletableSubscriber s = - (ContextPreservingCancellableCompletableSubscriber) subscriber; - if (s.saved == context) { - // replace current wrapper with wrapper that includes Subscriber and Cancellable - return new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingCompletableSubscriber && - ((ContextPreservingCompletableSubscriber) subscriber).saved == context) { - // no need to check for instanceof ContextPreservingCompletableSubscriberAndCancellable, because - // it extends from ContextPreservingCompletableSubscriber. - return subscriber; - } - return new ContextPreservingCompletableSubscriber(subscriber, context); - } - - @Override - public final CompletableSource.Subscriber wrapCompletableSubscriberAndCancellable( - final CompletableSource.Subscriber subscriber, final CapturedContext context) { - if (subscriber instanceof ContextPreservingCompletableSubscriber) { - final ContextPreservingCompletableSubscriber s = (ContextPreservingCompletableSubscriber) subscriber; - if (s.saved == context) { - return subscriber instanceof ContextPreservingCompletableSubscriberAndCancellable ? subscriber : - new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber) { - final ContextPreservingCancellableCompletableSubscriber s = - (ContextPreservingCancellableCompletableSubscriber) subscriber; - if (s.saved == context) { - return new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); - } - } - return new ContextPreservingCompletableSubscriberAndCancellable(subscriber, context); - } - - @Override - public final SingleSource.Subscriber wrapCancellable(final SingleSource.Subscriber subscriber, - final CapturedContext context) { - if (subscriber instanceof ContextPreservingSingleSubscriber) { - final ContextPreservingSingleSubscriber s = (ContextPreservingSingleSubscriber) subscriber; - if (s.saved == context) { - return subscriber instanceof ContextPreservingSingleSubscriberAndCancellable ? subscriber : - new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingCancellableSingleSubscriber && - ((ContextPreservingCancellableSingleSubscriber) subscriber).saved == context) { - // no need to check for instanceof ContextPreservingSingleSubscriberAndCancellable, because - // it extends from ContextPreservingSingleSubscriber. - return subscriber; - } - return new ContextPreservingCancellableSingleSubscriber<>(subscriber, context); - } - - @Override - public final SingleSource.Subscriber wrapSingleSubscriber(final SingleSource.Subscriber subscriber, - final CapturedContext context) { - if (subscriber instanceof ContextPreservingCancellableSingleSubscriber) { - final ContextPreservingCancellableSingleSubscriber s = - (ContextPreservingCancellableSingleSubscriber) subscriber; - if (s.saved == context) { - return new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingSingleSubscriber && - ((ContextPreservingSingleSubscriber) subscriber).saved == context) { - // no need to check for instanceof ContextPreservingSingleSubscriberAndCancellable, because - // it extends from ContextPreservingSingleSubscriber. - return subscriber; - } - return new ContextPreservingSingleSubscriber<>(subscriber, context); - } - - @Override - public final SingleSource.Subscriber wrapSingleSubscriberAndCancellable( - final SingleSource.Subscriber subscriber, final CapturedContext context) { - if (subscriber instanceof ContextPreservingSingleSubscriber) { - final ContextPreservingSingleSubscriber s = (ContextPreservingSingleSubscriber) subscriber; - if (s.saved == context) { - return subscriber instanceof ContextPreservingSingleSubscriberAndCancellable ? subscriber : - new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingCancellableSingleSubscriber) { - final ContextPreservingCancellableSingleSubscriber s = - (ContextPreservingCancellableSingleSubscriber) subscriber; - if (s.saved == context) { - return new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); - } - } - return new ContextPreservingSingleSubscriberAndCancellable<>(subscriber, context); - } - - @Override - public final Subscriber wrapSubscription(final Subscriber subscriber, final CapturedContext context) { - if (subscriber instanceof ContextPreservingSubscriber) { - final ContextPreservingSubscriber s = (ContextPreservingSubscriber) subscriber; - if (s.saved == context) { - return subscriber instanceof ContextPreservingSubscriberAndSubscription ? subscriber : - new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingSubscriptionSubscriber && - ((ContextPreservingSubscriptionSubscriber) subscriber).saved == context) { - // no need to check for instanceof ContextPreservingSubscriberAndSubscription, because - // it extends from ContextPreservingSubscriptionSubscriber. - return subscriber; - } - return new ContextPreservingSubscriptionSubscriber<>(subscriber, context); - } - - @Override - public final Subscriber wrapPublisherSubscriber(final Subscriber subscriber, final CapturedContext context) { - if (subscriber instanceof ContextPreservingSubscriptionSubscriber) { - final ContextPreservingSubscriptionSubscriber s = - (ContextPreservingSubscriptionSubscriber) subscriber; - if (s.saved == context) { - return new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingSubscriber && - ((ContextPreservingSubscriber) subscriber).saved == context) { - // no need to check for instanceof ContextPreservingSubscriberAndSubscription, because - // it extends from ContextPreservingSubscriptionSubscriber. - return subscriber; - } - return new ContextPreservingSubscriber<>(subscriber, context); - } - - @Override - public final Subscriber wrapPublisherSubscriberAndSubscription(final Subscriber subscriber, - final CapturedContext context) { - if (subscriber instanceof ContextPreservingSubscriber) { - final ContextPreservingSubscriber s = (ContextPreservingSubscriber) subscriber; - if (s.saved == context) { - return subscriber instanceof ContextPreservingSubscriberAndSubscription ? subscriber : - new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); - } - } else if (subscriber instanceof ContextPreservingSubscriptionSubscriber) { - final ContextPreservingSubscriptionSubscriber s = - (ContextPreservingSubscriptionSubscriber) subscriber; - if (s.saved == context) { - return new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); - } - } - return new ContextPreservingSubscriberAndSubscription<>(subscriber, context); - } - - @Override - public final Executor wrapJdkExecutor(final Executor executor) { - return ContextPreservingExecutor.of(executor); - } - - @Override - public final ExecutorService wrapJdkExecutorService(final ExecutorService executor) { - return ContextPreservingExecutorService.of(executor); - } - - @Override - public final io.servicetalk.concurrent.api.Executor wrapExecutor(final io.servicetalk.concurrent.api.Executor executor) { - return ContextPreservingStExecutor.of(executor); - } - - @Override - public final ScheduledExecutorService wrapJdkScheduledExecutorService(final ScheduledExecutorService executor) { - return ContextPreservingScheduledExecutorService.of(executor); - } - - @Override - public final CompletableFuture wrapCompletableFuture(final CompletableFuture future, - final CapturedContext context) { - return ContextPreservingCompletableFuture.newContextPreservingFuture(future, context); - } - - @Override - public final Runnable wrapRunnable(final Runnable runnable, final CapturedContext context) { - return new ContextPreservingRunnable(runnable, context); - } - - @Override - public final Callable wrapCallable(final Callable callable, final CapturedContext context) { - return new ContextPreservingCallable<>(callable, context); - } - - @Override - public final Consumer wrapConsumer(final Consumer consumer, final CapturedContext context) { - return new ContextPreservingConsumer<>(consumer, context); - } - - @Override - public final Function wrapFunction(final Function func, final CapturedContext context) { - return new ContextPreservingFunction<>(func, context); - } - - @Override - public final BiConsumer wrapBiConsumer(final BiConsumer consumer, final CapturedContext context) { - return new ContextPreservingBiConsumer<>(consumer, context); - } - - @Override - public final BiFunction wrapBiFunction(final BiFunction func, final CapturedContext context) { - return new ContextPreservingBiFunction<>(func, context); - } -} diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java index af455c6917..7f6ee4594e 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java @@ -21,6 +21,7 @@ import io.servicetalk.concurrent.PublisherSource.Subscription; import io.servicetalk.concurrent.SingleSource; import io.servicetalk.context.api.ContextMap; +import io.servicetalk.context.api.ContextMapHolder; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -36,7 +37,7 @@ /** * Implementation that backs the {@link AsyncContext}. */ -interface AsyncContextProvider extends ContextCaptureProvider { +interface AsyncContextProvider extends ContextCaptureProvider, ContextMapHolder { /** * Get the current context. * @@ -45,8 +46,11 @@ interface AsyncContextProvider extends ContextCaptureProvider { * * @return The current context. */ + @Override ContextMap context(); + Scope attachContext(ContextMap contextMap); + /** * Wrap the {@link Cancellable} to ensure it is able to track {@link AsyncContext} correctly. * diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java index ea6f31d861..3cea407b58 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java @@ -40,7 +40,7 @@ public ContextMap captured() { @Override public Scope restoreContext() { Scope outer = parentContext.restoreContext(); - Scope inner = ContextMapThreadLocal.attachContext(context); + Scope inner = AsyncContext.provider().attachContext(context); return () -> { inner.close(); outer.close(); diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ConcurrentContextMap.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ConcurrentContextMap.java deleted file mode 100644 index 334100966e..0000000000 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ConcurrentContextMap.java +++ /dev/null @@ -1,181 +0,0 @@ -/* - * Copyright © 2018-2019, 2021 Apple Inc. and the ServiceTalk project authors - * - * Licensed 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 io.servicetalk.concurrent.api; - -import io.servicetalk.concurrent.internal.ContextMapUtils; -import io.servicetalk.context.api.ContextMap; - -import java.util.Map; -import java.util.Map.Entry; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.function.BiPredicate; -import java.util.function.Function; -import javax.annotation.Nullable; - -final class ConcurrentContextMap implements ContextMap { - - private final ConcurrentMap, Object> theMap; - - ConcurrentContextMap() { - theMap = new ConcurrentHashMap<>(4); // start with a smaller table - } - - private ConcurrentContextMap(ConcurrentContextMap rhs) { - theMap = new ConcurrentHashMap<>(rhs.theMap); - } - - @Override - public int size() { - return theMap.size(); - } - - @Override - public boolean isEmpty() { - return theMap.isEmpty(); - } - - @Override - public boolean containsKey(final Key key) { - return theMap.containsKey(key); - } - - @Override - public boolean containsValue(@Nullable final Object value) { - assert value != null; - return theMap.containsValue(value); - } - - @Override - public boolean contains(final Key key, @Nullable final T value) { - final T current = get(key); - return current != null && current.equals(value); - } - - @Nullable - @Override - @SuppressWarnings("unchecked") - public T get(final Key key) { - return (T) theMap.get(key); - } - - @Nullable - @Override - @SuppressWarnings("unchecked") - public T getOrDefault(final Key key, final T defaultValue) { - return (T) theMap.getOrDefault(key, defaultValue); - } - - @Nullable - @Override - @SuppressWarnings("unchecked") - public T put(final Key key, @Nullable final T value) { - assert value != null; - return (T) theMap.put(key, value); - } - - @Nullable - @Override - @SuppressWarnings("unchecked") - public T putIfAbsent(final Key key, @Nullable final T value) { - assert value != null; - return (T) theMap.putIfAbsent(key, value); - } - - @Nullable - @Override - @SuppressWarnings("unchecked") - public T computeIfAbsent(final Key key, final Function, T> computeFunction) { - return (T) theMap.computeIfAbsent(key, k -> computeFunction.apply((Key) k)); - } - - @Override - public void putAll(final ContextMap map) { - if (map instanceof ConcurrentContextMap) { - final ConcurrentContextMap ccm = (ConcurrentContextMap) map; - theMap.putAll(ccm.theMap); - } else { - ContextMap.super.putAll(map); - } - } - - @Override - public void putAll(final Map, Object> map) { - map.forEach(ContextMapUtils::ensureType); - theMap.putAll(map); - } - - @Override - @SuppressWarnings("unchecked") - public T remove(final Key key) { - return (T) theMap.remove(key); - } - - @Override - public boolean removeAll(final Iterable> keys) { - boolean removed = false; - for (Key k : keys) { - // Null values aren't allowed so if a non-null value is seen then the map has been modified. - removed |= theMap.remove(k) != null; - } - return removed; - } - - @Override - public void clear() { - theMap.clear(); - } - - @Nullable - @Override - public Key forEach(final BiPredicate, Object> consumer) { - for (Entry, Object> entry : theMap.entrySet()) { - if (!consumer.test(entry.getKey(), entry.getValue())) { - return entry.getKey(); - } - } - return null; - } - - @Override - public ContextMap copy() { - return new ConcurrentContextMap(this); - } - - @Override - public boolean equals(final Object o) { - if (this == o) { - return true; - } - if (!(o instanceof ContextMap)) { - return false; - } - if (o instanceof ConcurrentContextMap) { - return theMap.equals(((ConcurrentContextMap) o).theMap); - } - return ContextMapUtils.equals(this, (ContextMap) o); - } - - @Override - public int hashCode() { - return theMap.hashCode(); - } - - @Override - public String toString() { - return ContextMapUtils.toString(this); - } -} diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java index b7e6107d77..dca9760b96 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java @@ -29,5 +29,12 @@ interface ContextCaptureProvider { */ CapturedContext captureContext(); + /** + * Save a copy of the existing context in preparation for an asynchronous thread jump. + * + * Note that this can do more than just package up the ServiceTalk {@link AsyncContext} and could be enhanced or + * wrapped to bundle up additional contexts such as the OpenTelemetry or grpc contexts. + * @return the copied saved context state that may be restored later. + */ CapturedContext captureContextCopy(); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextMapThreadLocal.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextMapThreadLocal.java index 0243728285..7b9ad60db6 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextMapThreadLocal.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextMapThreadLocal.java @@ -1,126 +1,126 @@ -/* - * Copyright © 2025 Apple Inc. and the ServiceTalk project authors - * - * Licensed 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 io.servicetalk.concurrent.api; - -import io.servicetalk.context.api.ContextMap; -import io.servicetalk.context.api.ContextMapHolder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nonnull; - -import static java.lang.ThreadLocal.withInitial; - -final class ContextMapThreadLocal { - - private static final Logger LOGGER = LoggerFactory.getLogger(ContextMapThreadLocal.class); - - private static final ThreadLocal CONTEXT_THREAD_LOCAL = - withInitial(ContextMapThreadLocal::newContextMap); - - static ContextMap context() { - final Thread t = Thread.currentThread(); - if (t instanceof ContextMapHolder) { - final ContextMapHolder contextMapHolder = (ContextMapHolder) t; - ContextMap map = contextMapHolder.context(); - if (map == null) { - map = newContextMap(); - contextMapHolder.context(map); - } - return map; - } else { - return CONTEXT_THREAD_LOCAL.get(); - } - } - - static CapturedContext captureContext() { - return toCaptureContext(context()); - } - - static CapturedContext captureContextCopy() { - return toCaptureContext(context().copy()); - } - - static Scope attachContext(ContextMap contextMap) { - ContextMap prev = exchangeContext(contextMap); - return !LOGGER.isDebugEnabled() && prev instanceof Scope ? (Scope) prev : () -> detachContext(contextMap, prev); - } - - // Used for `CopyOnWriteContextMap.close()` - static void setContext(ContextMap contextMap) { - final Thread currentThread = Thread.currentThread(); - if (currentThread instanceof ContextMapHolder) { - final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; - asyncContextMapHolder.context(contextMap); - } else { - CONTEXT_THREAD_LOCAL.set(contextMap); - } - } - - // Used for CaptureContext operations - private static ContextMap exchangeContext(ContextMap contextMap) { - final Thread currentThread = Thread.currentThread(); - ContextMap result; - if (currentThread instanceof ContextMapHolder) { - final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; - result = asyncContextMapHolder.context(); - if (result == null) { - result = newContextMap(); - } - asyncContextMapHolder.context(contextMap); - } else { - result = CONTEXT_THREAD_LOCAL.get(); - CONTEXT_THREAD_LOCAL.set(contextMap); - } - return result; - } - - private static CapturedContext toCaptureContext(ContextMap contextMap) { - return contextMap instanceof CapturedContext ? - (CapturedContext) contextMap : new CapturedContextImpl(contextMap); - } - - private static final class CapturedContextImpl implements CapturedContext { - - private final ContextMap contextMap; - - CapturedContextImpl(ContextMap contextMap) { - this.contextMap = contextMap; - } - - @Override - public ContextMap captured() { - return contextMap; - } - - @Override - public Scope restoreContext() { - return attachContext(contextMap); - } - } - - private static void detachContext(ContextMap expectedContext, ContextMap toRestore) { - ContextMap current = exchangeContext(toRestore); - if (current != expectedContext) { - LOGGER.debug("Current context didn't match the expected context. current: {}, expected: {}", - current, expectedContext); - } - } - - private static ContextMap newContextMap() { - return new CopyOnWriteContextMap(); - } -} +///* +// * Copyright © 2025 Apple Inc. and the ServiceTalk project authors +// * +// * Licensed 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 io.servicetalk.concurrent.api; +// +//import io.servicetalk.context.api.ContextMap; +//import io.servicetalk.context.api.ContextMapHolder; +//import org.slf4j.Logger; +//import org.slf4j.LoggerFactory; +// +//import javax.annotation.Nonnull; +// +//import static java.lang.ThreadLocal.withInitial; +// +//final class ContextMapThreadLocal { +// +// private static final Logger LOGGER = LoggerFactory.getLogger(ContextMapThreadLocal.class); +// +// private static final ThreadLocal CONTEXT_THREAD_LOCAL = +// withInitial(ContextMapThreadLocal::newContextMap); +// +// static ContextMap context() { +// final Thread t = Thread.currentThread(); +// if (t instanceof ContextMapHolder) { +// final ContextMapHolder contextMapHolder = (ContextMapHolder) t; +// ContextMap map = contextMapHolder.context(); +// if (map == null) { +// map = newContextMap(); +// contextMapHolder.context(map); +// } +// return map; +// } else { +// return CONTEXT_THREAD_LOCAL.get(); +// } +// } +// +// static CapturedContext captureContext() { +// return toCaptureContext(context()); +// } +// +// static CapturedContext captureContextCopy() { +// return toCaptureContext(context().copy()); +// } +// +// static Scope attachContext(ContextMap contextMap) { +// ContextMap prev = exchangeContext(contextMap); +// return !LOGGER.isDebugEnabled() && prev instanceof Scope ? (Scope) prev : () -> detachContext(contextMap, prev); +// } +// +// // Used for `CopyOnWriteContextMap.close()` +// static void setContext(ContextMap contextMap) { +// final Thread currentThread = Thread.currentThread(); +// if (currentThread instanceof ContextMapHolder) { +// final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; +// asyncContextMapHolder.context(contextMap); +// } else { +// CONTEXT_THREAD_LOCAL.set(contextMap); +// } +// } +// +// // Used for CaptureContext operations +// private static ContextMap exchangeContext(ContextMap contextMap) { +// final Thread currentThread = Thread.currentThread(); +// ContextMap result; +// if (currentThread instanceof ContextMapHolder) { +// final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; +// result = asyncContextMapHolder.context(); +// if (result == null) { +// result = newContextMap(); +// } +// asyncContextMapHolder.context(contextMap); +// } else { +// result = CONTEXT_THREAD_LOCAL.get(); +// CONTEXT_THREAD_LOCAL.set(contextMap); +// } +// return result; +// } +// +// private static CapturedContext toCaptureContext(ContextMap contextMap) { +// return contextMap instanceof CapturedContext ? +// (CapturedContext) contextMap : new CapturedContextImpl(contextMap); +// } +// +// private static final class CapturedContextImpl implements CapturedContext { +// +// private final ContextMap contextMap; +// +// CapturedContextImpl(ContextMap contextMap) { +// this.contextMap = contextMap; +// } +// +// @Override +// public ContextMap captured() { +// return contextMap; +// } +// +// @Override +// public Scope restoreContext() { +// return attachContext(contextMap); +// } +// } +// +// private static void detachContext(ContextMap expectedContext, ContextMap toRestore) { +// ContextMap current = exchangeContext(toRestore); +// if (current != expectedContext) { +// LOGGER.debug("Current context didn't match the expected context. current: {}, expected: {}", +// current, expectedContext); +// } +// } +// +// private static ContextMap newContextMap() { +// return new CopyOnWriteContextMap(); +// } +//} diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java index f4e5861cba..eb2ac1e5f6 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java @@ -187,6 +187,24 @@ public String toString() { return ContextMapUtils.toString(this); } + // CapturedContext methods + + @Override + public ContextMap captured() { + return this; + } + + @Override + public Scope restoreContext() { + return AsyncContext.provider().attachContext(this); + } + + // Scope method + @Override + public void close() { + AsyncContext.provider().context(this); + } + private interface CopyContextMap { int size(); @@ -2973,22 +2991,4 @@ CopyContextMap build() { return new SevenOrMoreContextMap(Arrays.copyOf(pairs, index)); } } - - // CapturedContext methods - - @Override - public ContextMap captured() { - return this; - } - - @Override - public Scope restoreContext() { - return ContextMapThreadLocal.attachContext(this); - } - - // Scope method - @Override - public void close() { - ContextMapThreadLocal.setContext(this); - } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java index 0725c3a4d1..c6e124ce0a 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java @@ -2,7 +2,7 @@ import static java.util.Objects.requireNonNull; -final class CustomCaptureAsyncContextProvider extends AbstractAsyncContextProvider { +final class CustomCaptureAsyncContextProvider extends DefaultAsyncContextProvider { private final ContextCaptureProvider delegate; diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java index 58248aaa8a..00b8ba24d1 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java @@ -15,21 +15,359 @@ */ package io.servicetalk.concurrent.api; -final class DefaultAsyncContextProvider extends AbstractAsyncContextProvider { +import io.servicetalk.concurrent.CompletableSource; +import io.servicetalk.concurrent.PublisherSource; +import io.servicetalk.concurrent.SingleSource; +import io.servicetalk.context.api.ContextMap; +import io.servicetalk.context.api.ContextMapHolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; - static final DefaultAsyncContextProvider INSTANCE = new DefaultAsyncContextProvider(); +import javax.annotation.Nonnull; +import javax.annotation.Nullable; - private DefaultAsyncContextProvider() { - // singleton +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; +import java.util.function.BiConsumer; +import java.util.function.BiFunction; +import java.util.function.Consumer; +import java.util.function.Function; + +import static java.lang.ThreadLocal.withInitial; + +class DefaultAsyncContextProvider implements AsyncContextProvider { + + private static final ThreadLocal CONTEXT_THREAD_LOCAL = + withInitial(DefaultAsyncContextProvider::newContextMap); + + private static final Logger LOGGER = LoggerFactory.getLogger(DefaultAsyncContextProvider.class); + private static final boolean NOT_IS_DEBUG_ENABLED = !LOGGER.isDebugEnabled(); + + static final AsyncContextProvider INSTANCE = new DefaultAsyncContextProvider(); + + protected DefaultAsyncContextProvider() { + } + + @Nonnull + @Override + public final ContextMap context() { + final Thread t = Thread.currentThread(); + if (t instanceof ContextMapHolder) { + final ContextMapHolder contextMapHolder = (ContextMapHolder) t; + ContextMap map = contextMapHolder.context(); + if (map == null) { + map = newContextMap(); + contextMapHolder.context(map); + } + return map; + } else { + return CONTEXT_THREAD_LOCAL.get(); + } + } + + @Override + public final ContextMapHolder context(@Nullable ContextMap contextMap) { + final Thread currentThread = Thread.currentThread(); + if (currentThread instanceof ContextMapHolder) { + final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; + asyncContextMapHolder.context(contextMap); + } else if (contextMap == null) { + CONTEXT_THREAD_LOCAL.remove(); + } else { + CONTEXT_THREAD_LOCAL.set(contextMap); + } + return this; + } + + @Override + public final Scope attachContext(ContextMap contextMap) { + ContextMap prev = exchangeContext(contextMap); + return NOT_IS_DEBUG_ENABLED && prev instanceof Scope ? (Scope) prev : () -> detachContext(contextMap, prev); } @Override public CapturedContext captureContext() { - return ContextMapThreadLocal.captureContext(); + return toCaptureContext(context()); } @Override public CapturedContext captureContextCopy() { - return ContextMapThreadLocal.captureContextCopy(); + return toCaptureContext(context().copy()); + } + + @Override + public final CompletableSource.Subscriber wrapCancellable(final CompletableSource.Subscriber subscriber, + final CapturedContext context) { + if (subscriber instanceof ContextPreservingCompletableSubscriber) { + final ContextPreservingCompletableSubscriber s = (ContextPreservingCompletableSubscriber) subscriber; + if (s.saved == context) { + return subscriber instanceof ContextPreservingCompletableSubscriberAndCancellable ? subscriber : + new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber && + ((ContextPreservingCancellableCompletableSubscriber) subscriber).saved == context) { + // no need to check for instanceof ContextPreservingCompletableSubscriberAndCancellable, because + // it extends from ContextPreservingSingleSubscriber. + return subscriber; + } + return new ContextPreservingCancellableCompletableSubscriber(subscriber, context); + } + + @Override + public final CompletableSource.Subscriber wrapCompletableSubscriber(final CompletableSource.Subscriber subscriber, + final CapturedContext context) { + if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber) { + final ContextPreservingCancellableCompletableSubscriber s = + (ContextPreservingCancellableCompletableSubscriber) subscriber; + if (s.saved == context) { + // replace current wrapper with wrapper that includes Subscriber and Cancellable + return new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingCompletableSubscriber && + ((ContextPreservingCompletableSubscriber) subscriber).saved == context) { + // no need to check for instanceof ContextPreservingCompletableSubscriberAndCancellable, because + // it extends from ContextPreservingCompletableSubscriber. + return subscriber; + } + return new ContextPreservingCompletableSubscriber(subscriber, context); + } + + @Override + public final CompletableSource.Subscriber wrapCompletableSubscriberAndCancellable( + final CompletableSource.Subscriber subscriber, final CapturedContext context) { + if (subscriber instanceof ContextPreservingCompletableSubscriber) { + final ContextPreservingCompletableSubscriber s = (ContextPreservingCompletableSubscriber) subscriber; + if (s.saved == context) { + return subscriber instanceof ContextPreservingCompletableSubscriberAndCancellable ? subscriber : + new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber) { + final ContextPreservingCancellableCompletableSubscriber s = + (ContextPreservingCancellableCompletableSubscriber) subscriber; + if (s.saved == context) { + return new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); + } + } + return new ContextPreservingCompletableSubscriberAndCancellable(subscriber, context); + } + + @Override + public final SingleSource.Subscriber wrapCancellable(final SingleSource.Subscriber subscriber, + final CapturedContext context) { + if (subscriber instanceof ContextPreservingSingleSubscriber) { + final ContextPreservingSingleSubscriber s = (ContextPreservingSingleSubscriber) subscriber; + if (s.saved == context) { + return subscriber instanceof ContextPreservingSingleSubscriberAndCancellable ? subscriber : + new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingCancellableSingleSubscriber && + ((ContextPreservingCancellableSingleSubscriber) subscriber).saved == context) { + // no need to check for instanceof ContextPreservingSingleSubscriberAndCancellable, because + // it extends from ContextPreservingSingleSubscriber. + return subscriber; + } + return new ContextPreservingCancellableSingleSubscriber<>(subscriber, context); + } + + @Override + public final SingleSource.Subscriber wrapSingleSubscriber(final SingleSource.Subscriber subscriber, + final CapturedContext context) { + if (subscriber instanceof ContextPreservingCancellableSingleSubscriber) { + final ContextPreservingCancellableSingleSubscriber s = + (ContextPreservingCancellableSingleSubscriber) subscriber; + if (s.saved == context) { + return new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingSingleSubscriber && + ((ContextPreservingSingleSubscriber) subscriber).saved == context) { + // no need to check for instanceof ContextPreservingSingleSubscriberAndCancellable, because + // it extends from ContextPreservingSingleSubscriber. + return subscriber; + } + return new ContextPreservingSingleSubscriber<>(subscriber, context); + } + + @Override + public final SingleSource.Subscriber wrapSingleSubscriberAndCancellable( + final SingleSource.Subscriber subscriber, final CapturedContext context) { + if (subscriber instanceof ContextPreservingSingleSubscriber) { + final ContextPreservingSingleSubscriber s = (ContextPreservingSingleSubscriber) subscriber; + if (s.saved == context) { + return subscriber instanceof ContextPreservingSingleSubscriberAndCancellable ? subscriber : + new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingCancellableSingleSubscriber) { + final ContextPreservingCancellableSingleSubscriber s = + (ContextPreservingCancellableSingleSubscriber) subscriber; + if (s.saved == context) { + return new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); + } + } + return new ContextPreservingSingleSubscriberAndCancellable<>(subscriber, context); + } + + @Override + public final PublisherSource.Subscriber wrapSubscription(final PublisherSource.Subscriber subscriber, final CapturedContext context) { + if (subscriber instanceof ContextPreservingSubscriber) { + final ContextPreservingSubscriber s = (ContextPreservingSubscriber) subscriber; + if (s.saved == context) { + return subscriber instanceof ContextPreservingSubscriberAndSubscription ? subscriber : + new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingSubscriptionSubscriber && + ((ContextPreservingSubscriptionSubscriber) subscriber).saved == context) { + // no need to check for instanceof ContextPreservingSubscriberAndSubscription, because + // it extends from ContextPreservingSubscriptionSubscriber. + return subscriber; + } + return new ContextPreservingSubscriptionSubscriber<>(subscriber, context); + } + + @Override + public final PublisherSource.Subscriber wrapPublisherSubscriber(final PublisherSource.Subscriber subscriber, final CapturedContext context) { + if (subscriber instanceof ContextPreservingSubscriptionSubscriber) { + final ContextPreservingSubscriptionSubscriber s = + (ContextPreservingSubscriptionSubscriber) subscriber; + if (s.saved == context) { + return new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingSubscriber && + ((ContextPreservingSubscriber) subscriber).saved == context) { + // no need to check for instanceof ContextPreservingSubscriberAndSubscription, because + // it extends from ContextPreservingSubscriptionSubscriber. + return subscriber; + } + return new ContextPreservingSubscriber<>(subscriber, context); + } + + @Override + public final PublisherSource.Subscriber wrapPublisherSubscriberAndSubscription(final PublisherSource.Subscriber subscriber, + final CapturedContext context) { + if (subscriber instanceof ContextPreservingSubscriber) { + final ContextPreservingSubscriber s = (ContextPreservingSubscriber) subscriber; + if (s.saved == context) { + return subscriber instanceof ContextPreservingSubscriberAndSubscription ? subscriber : + new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); + } + } else if (subscriber instanceof ContextPreservingSubscriptionSubscriber) { + final ContextPreservingSubscriptionSubscriber s = + (ContextPreservingSubscriptionSubscriber) subscriber; + if (s.saved == context) { + return new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); + } + } + return new ContextPreservingSubscriberAndSubscription<>(subscriber, context); + } + + @Override + public final java.util.concurrent.Executor wrapJdkExecutor(final java.util.concurrent.Executor executor) { + return ContextPreservingExecutor.of(executor); + } + + @Override + public final ExecutorService wrapJdkExecutorService(final ExecutorService executor) { + return ContextPreservingExecutorService.of(executor); + } + + @Override + public final io.servicetalk.concurrent.api.Executor wrapExecutor(final io.servicetalk.concurrent.api.Executor executor) { + return ContextPreservingStExecutor.of(executor); + } + + @Override + public final ScheduledExecutorService wrapJdkScheduledExecutorService(final ScheduledExecutorService executor) { + return ContextPreservingScheduledExecutorService.of(executor); + } + + @Override + public final CompletableFuture wrapCompletableFuture(final CompletableFuture future, + final CapturedContext context) { + return ContextPreservingCompletableFuture.newContextPreservingFuture(future, context); + } + + @Override + public final Runnable wrapRunnable(final Runnable runnable, final CapturedContext context) { + return new ContextPreservingRunnable(runnable, context); + } + + @Override + public final Callable wrapCallable(final Callable callable, final CapturedContext context) { + return new ContextPreservingCallable<>(callable, context); + } + + @Override + public final Consumer wrapConsumer(final Consumer consumer, final CapturedContext context) { + return new ContextPreservingConsumer<>(consumer, context); + } + + @Override + public final Function wrapFunction(final Function func, final CapturedContext context) { + return new ContextPreservingFunction<>(func, context); + } + + @Override + public final BiConsumer wrapBiConsumer(final BiConsumer consumer, final CapturedContext context) { + return new ContextPreservingBiConsumer<>(consumer, context); + } + + @Override + public final BiFunction wrapBiFunction(final BiFunction func, final CapturedContext context) { + return new ContextPreservingBiFunction<>(func, context); + } + + private static final class CapturedContextImpl implements CapturedContext { + + private final ContextMap contextMap; + + CapturedContextImpl(ContextMap contextMap) { + this.contextMap = contextMap; + } + + @Override + public ContextMap captured() { + return contextMap; + } + + @Override + public Scope restoreContext() { + ContextMap prev = exchangeContext(contextMap); + return NOT_IS_DEBUG_ENABLED && prev instanceof Scope ? (Scope) prev : () -> detachContext(contextMap, prev); + } + } + + private static CapturedContext toCaptureContext(ContextMap contextMap) { + return contextMap instanceof CapturedContext ? + (CapturedContext) contextMap : new CapturedContextImpl(contextMap); + } + + private static ContextMap exchangeContext(ContextMap contextMap) { + final Thread currentThread = Thread.currentThread(); + ContextMap result; + if (currentThread instanceof ContextMapHolder) { + final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; + result = asyncContextMapHolder.context(); + if (result == null) { + result = newContextMap(); + } + asyncContextMapHolder.context(contextMap); + } else { + result = CONTEXT_THREAD_LOCAL.get(); + CONTEXT_THREAD_LOCAL.set(contextMap); + } + return result; + } + + private static void detachContext(ContextMap expectedContext, ContextMap toRestore) { + ContextMap current = exchangeContext(toRestore); + if (current != expectedContext) { + LOGGER.debug("Current context didn't match the expected context. current: {}, expected: {}", + current, expectedContext); + } + } + + private static ContextMap newContextMap() { + return new CopyOnWriteContextMap(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java index 6b94018461..6c6f7cfd70 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java @@ -20,6 +20,7 @@ import io.servicetalk.concurrent.SingleSource; import io.servicetalk.concurrent.internal.ContextMapUtils; import io.servicetalk.context.api.ContextMap; +import io.servicetalk.context.api.ContextMapHolder; import java.util.Map; import java.util.concurrent.Callable; @@ -68,6 +69,16 @@ public CapturedContext captureContextCopy() { return NOOP_SAVED_CONTEXT; } + @Override + public ContextMapHolder context(@Nullable ContextMap contextMap) { + return this; + } + + @Override + public Scope attachContext(ContextMap contextMap) { + return Scope.NOOP; + } + @Override public CompletableSource.Subscriber wrapCancellable(final CompletableSource.Subscriber subscriber, final CapturedContext context) { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java index cb940233bf..53e9b242d5 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java @@ -40,7 +40,7 @@ public ContextMap captured() { @Override public Scope restoreContext() { Scope outer = parentContext.restoreContext(); - Scope inner = ContextMapThreadLocal.attachContext(context); + Scope inner = AsyncContext.provider().attachContext(context); return () -> { inner.close(); outer.close(); diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java index 7728e7986d..4d92956d0a 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java @@ -40,7 +40,7 @@ public ContextMap captured() { @Override public Scope restoreContext() { Scope outer = parentContext.restoreContext(); - Scope inner = ContextMapThreadLocal.attachContext(context); + Scope inner = AsyncContext.provider().attachContext(context); return () -> { inner.close(); outer.close(); From edf142afc86415dbcee94025947c7d40074a262a Mon Sep 17 00:00:00 2001 From: Bryce Anderson Date: Wed, 5 Feb 2025 18:24:50 -0700 Subject: [PATCH 05/14] Residual feedback from #3185 --- .../concurrent/api/DefaultAsyncContextProvider.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java index 00b8ba24d1..384f1a7c13 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java @@ -43,7 +43,7 @@ class DefaultAsyncContextProvider implements AsyncContextProvider { withInitial(DefaultAsyncContextProvider::newContextMap); private static final Logger LOGGER = LoggerFactory.getLogger(DefaultAsyncContextProvider.class); - private static final boolean NOT_IS_DEBUG_ENABLED = !LOGGER.isDebugEnabled(); + private static final boolean NO_DEBUG_LOGGING = !LOGGER.isDebugEnabled(); static final AsyncContextProvider INSTANCE = new DefaultAsyncContextProvider(); @@ -84,7 +84,7 @@ public final ContextMapHolder context(@Nullable ContextMap contextMap) { @Override public final Scope attachContext(ContextMap contextMap) { ContextMap prev = exchangeContext(contextMap); - return NOT_IS_DEBUG_ENABLED && prev instanceof Scope ? (Scope) prev : () -> detachContext(contextMap, prev); + return NO_DEBUG_LOGGING && prev instanceof Scope ? (Scope) prev : () -> detachContext(contextMap, prev); } @Override @@ -333,7 +333,7 @@ public ContextMap captured() { @Override public Scope restoreContext() { ContextMap prev = exchangeContext(contextMap); - return NOT_IS_DEBUG_ENABLED && prev instanceof Scope ? (Scope) prev : () -> detachContext(contextMap, prev); + return NO_DEBUG_LOGGING && prev instanceof Scope ? (Scope) prev : () -> detachContext(contextMap, prev); } } @@ -362,8 +362,10 @@ private static ContextMap exchangeContext(ContextMap contextMap) { private static void detachContext(ContextMap expectedContext, ContextMap toRestore) { ContextMap current = exchangeContext(toRestore); if (current != expectedContext) { - LOGGER.debug("Current context didn't match the expected context. current: {}, expected: {}", - current, expectedContext); + if (!NO_DEBUG_LOGGING) { + LOGGER.debug("Current context didn't match the expected context. current: {}, expected: {}", + current, expectedContext, new Throwable("stack trace")); + } } } From 4ebbdb6c0032b90084b33467e68ac34c12d8c7a2 Mon Sep 17 00:00:00 2001 From: Bryce Anderson Date: Wed, 5 Feb 2025 20:00:03 -0700 Subject: [PATCH 06/14] Refine the api a bit --- .../concurrent/api/AsyncContext.java | 25 +++++--- .../concurrent/api/AsyncContextProvider.java | 18 +++++- .../api/CapturedContextProvider.java | 61 +++++++++++++++++++ .../concurrent/api/Completable.java | 2 +- .../CompletableShareContextOnSubscribe.java | 2 +- .../api/ContextCaptureProvider.java | 40 ------------ .../CustomCaptureAsyncContextProvider.java | 15 ++--- .../api/DefaultAsyncContextProvider.java | 17 ++---- .../api/NoopAsyncContextProvider.java | 13 ++-- .../servicetalk/concurrent/api/Publisher.java | 2 +- .../io/servicetalk/concurrent/api/Single.java | 2 +- 11 files changed, 114 insertions(+), 83 deletions(-) create mode 100644 servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProvider.java delete mode 100644 servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContext.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContext.java index efe9ca0ab2..bad3327ba7 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContext.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContext.java @@ -33,7 +33,6 @@ import java.util.function.BiPredicate; import java.util.function.Consumer; import java.util.function.Function; -import java.util.function.UnaryOperator; import javax.annotation.Nullable; import static io.servicetalk.concurrent.api.AsyncContextExecutorPlugin.EXECUTOR_PLUGIN; @@ -70,16 +69,22 @@ public final class AsyncContext { private static AsyncContextProvider provider; static { - AsyncContextProvider result = DefaultAsyncContextProvider.INSTANCE; - List> wrappers = asyncProviderWrappers(); - if (!wrappers.isEmpty()) { - ContextCaptureProvider contextCaptureProvider = DefaultAsyncContextProvider.INSTANCE; - for (UnaryOperator wrapper : wrappers) { - contextCaptureProvider = wrapper.apply(result); + CapturedContextProvider capturedContextProvider = null; + for (CapturedContextProvider provider : asyncProviderWrappers()) { + if (capturedContextProvider == null) { + capturedContextProvider = provider; + } else { + final CapturedContextProvider finalCapturedContextProvider = capturedContextProvider; + // TODO: at some point this is perhaps better as a list iteration. + capturedContextProvider = (context) -> + provider.captureContext(finalCapturedContextProvider.captureContext(context)); } - result = new CustomCaptureAsyncContextProvider(contextCaptureProvider); } - DEFAULT_ENABLED_PROVIDER = result; + if (capturedContextProvider == null) { + DEFAULT_ENABLED_PROVIDER = DefaultAsyncContextProvider.INSTANCE; + } else { + DEFAULT_ENABLED_PROVIDER = new CustomCaptureAsyncContextProvider(capturedContextProvider); + } provider = DEFAULT_ENABLED_PROVIDER; } @@ -581,7 +586,7 @@ private static void disable0() { LOGGER.info("Disabled. Features that depend on AsyncContext will stop working."); } - private static List> asyncProviderWrappers() { + private static List asyncProviderWrappers() { return Collections.emptyList(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java index 7f6ee4594e..53a22db2e6 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java @@ -23,6 +23,8 @@ import io.servicetalk.context.api.ContextMap; import io.servicetalk.context.api.ContextMapHolder; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -37,7 +39,7 @@ /** * Implementation that backs the {@link AsyncContext}. */ -interface AsyncContextProvider extends ContextCaptureProvider, ContextMapHolder { +interface AsyncContextProvider { /** * Get the current context. * @@ -46,11 +48,23 @@ interface AsyncContextProvider extends ContextCaptureProvider, ContextMapHolder * * @return The current context. */ - @Override ContextMap context(); + /** + * Set the {@link ContextMap}. + * + * @param context the new value for {@link ContextMap}. + * @return {@code this}. + */ + void context(ContextMap context); + Scope attachContext(ContextMap contextMap); + CapturedContext captureContext(ContextMap contextMap); + + // TODO: refactor to remove this + CapturedContext captureContext(); + /** * Wrap the {@link Cancellable} to ensure it is able to track {@link AsyncContext} correctly. * diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProvider.java new file mode 100644 index 0000000000..cafa4d5e94 --- /dev/null +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProvider.java @@ -0,0 +1,61 @@ +/* + * Copyright © 2025 Apple Inc. and the ServiceTalk project authors + * + * Licensed 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 io.servicetalk.concurrent.api; + +/** + * Functionality related to capturing thread-local like context for later restoration across async boundaries. + */ +interface CapturedContextProvider { + + /** + * Save existing context in preparation for an asynchronous thread jump. + * + * If you want to save any external state you can create a wrapper {@link CapturedContext} to add additional + * state capturing to the context pathway. This state can then be restored by wrapping the {@link CapturedContext} + * with the additional functionality to restore and finally revert the context state. + *

+ * An example provider may be implemented as follows: + *

{@code
+     *     private class CapturedContextImpl {
+     *         private final CapturedContext delegate;
+     *         private final String state;
+     *
+     *         Scope restoreContext() {
+     *             String old = getMyString();
+     *             MY_THREAD_LOCAL.set(state);
+     *             Scope outer = delegate.restoreContext();
+     *             return () -> {
+     *                 outer.close();
+     *                 setMyString(old);
+     *             };
+     *         }
+     *     }
+     *
+     *     private String getMyString() {
+     *         // capture context state from the local environment
+     *     }
+     *
+     *     private void setMyString(String string) {
+     *         // set the context state in the local environment
+     *     }
+     *
+     *     CapturedContext captureContext(CapturedContext underlying) {
+     *          return new CapturedContextImpl(delegate, getMyString());
+     *     }
+     * }
+ */ + CapturedContext captureContext(CapturedContext underlying); +} diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Completable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Completable.java index b8d31a9534..45dd7e5a9e 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Completable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Completable.java @@ -1730,7 +1730,7 @@ public final Future toFuture() { */ CapturedContext contextForSubscribe(AsyncContextProvider provider) { // the default behavior is to copy the map. Some operators may want to use shared map - return provider.captureContextCopy(); + return provider.captureContext(provider.context().copy()); } /** diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableShareContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableShareContextOnSubscribe.java index 84dd669fbb..bc3b743a18 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableShareContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableShareContextOnSubscribe.java @@ -24,7 +24,7 @@ final class CompletableShareContextOnSubscribe extends AbstractNoHandleSubscribe @Override CapturedContext contextForSubscribe(AsyncContextProvider provider) { - return provider.captureContext(); + return provider.captureContext(provider.context()); } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java deleted file mode 100644 index dca9760b96..0000000000 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextCaptureProvider.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Copyright © 2025 Apple Inc. and the ServiceTalk project authors - * - * Licensed 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 io.servicetalk.concurrent.api; - -/** - * Functionality related to capturing thread-local like context for later restoration across async boundaries. - */ -interface ContextCaptureProvider { - - /** - * Save existing context in preparation for an asynchronous thread jump. - * - * Note that this can do more than just package up the ServiceTalk {@link AsyncContext} and could be enhanced or - * wrapped to bundle up additional contexts such as the OpenTelemetry or grpc contexts. - * @return the saved context state that may be restored later. - */ - CapturedContext captureContext(); - - /** - * Save a copy of the existing context in preparation for an asynchronous thread jump. - * - * Note that this can do more than just package up the ServiceTalk {@link AsyncContext} and could be enhanced or - * wrapped to bundle up additional contexts such as the OpenTelemetry or grpc contexts. - * @return the copied saved context state that may be restored later. - */ - CapturedContext captureContextCopy(); -} diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java index c6e124ce0a..bd2ae89228 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java @@ -1,22 +1,19 @@ package io.servicetalk.concurrent.api; +import io.servicetalk.context.api.ContextMap; + import static java.util.Objects.requireNonNull; final class CustomCaptureAsyncContextProvider extends DefaultAsyncContextProvider { - private final ContextCaptureProvider delegate; + private final CapturedContextProvider delegate; - CustomCaptureAsyncContextProvider(ContextCaptureProvider delegate) { + CustomCaptureAsyncContextProvider(CapturedContextProvider delegate) { this.delegate = requireNonNull(delegate); } @Override - public CapturedContext captureContext() { - return delegate.captureContext(); - } - - @Override - public CapturedContext captureContextCopy() { - return delegate.captureContextCopy(); + public CapturedContext captureContext(ContextMap contextMap) { + return delegate.captureContext(super.captureContext(contextMap)); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java index 384f1a7c13..52caf53c4b 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java @@ -68,7 +68,7 @@ public final ContextMap context() { } @Override - public final ContextMapHolder context(@Nullable ContextMap contextMap) { + public final void context(@Nullable ContextMap contextMap) { final Thread currentThread = Thread.currentThread(); if (currentThread instanceof ContextMapHolder) { final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; @@ -78,7 +78,6 @@ public final ContextMapHolder context(@Nullable ContextMap contextMap) { } else { CONTEXT_THREAD_LOCAL.set(contextMap); } - return this; } @Override @@ -88,13 +87,14 @@ public final Scope attachContext(ContextMap contextMap) { } @Override - public CapturedContext captureContext() { - return toCaptureContext(context()); + public CapturedContext captureContext(ContextMap contextMap) { + return contextMap instanceof CapturedContext ? + (CapturedContext) contextMap : new CapturedContextImpl(contextMap); } @Override - public CapturedContext captureContextCopy() { - return toCaptureContext(context().copy()); + public final CapturedContext captureContext() { + return captureContext(context()); } @Override @@ -337,11 +337,6 @@ public Scope restoreContext() { } } - private static CapturedContext toCaptureContext(ContextMap contextMap) { - return contextMap instanceof CapturedContext ? - (CapturedContext) contextMap : new CapturedContextImpl(contextMap); - } - private static ContextMap exchangeContext(ContextMap contextMap) { final Thread currentThread = Thread.currentThread(); ContextMap result; diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java index 6c6f7cfd70..1973bdd545 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java @@ -20,7 +20,6 @@ import io.servicetalk.concurrent.SingleSource; import io.servicetalk.concurrent.internal.ContextMapUtils; import io.servicetalk.context.api.ContextMap; -import io.servicetalk.context.api.ContextMapHolder; import java.util.Map; import java.util.concurrent.Callable; @@ -38,7 +37,7 @@ final class NoopAsyncContextProvider implements AsyncContextProvider { static final AsyncContextProvider INSTANCE = new NoopAsyncContextProvider(); - private static final CapturedContext NOOP_SAVED_CONTEXT = new CapturedContext() { + private static final CapturedContext NOOP_CAPTURED_CONTEXT = new CapturedContext() { @Override public ContextMap captured() { return NoopContextMap.INSTANCE; @@ -61,17 +60,17 @@ public ContextMap context() { @Override public CapturedContext captureContext() { - return NOOP_SAVED_CONTEXT; + return NOOP_CAPTURED_CONTEXT; } @Override - public CapturedContext captureContextCopy() { - return NOOP_SAVED_CONTEXT; + public CapturedContext captureContext(ContextMap contextMap) { + return NOOP_CAPTURED_CONTEXT; } @Override - public ContextMapHolder context(@Nullable ContextMap contextMap) { - return this; + public void context(@Nullable ContextMap contextMap) { + // noop } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Publisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Publisher.java index 75c1559e23..9854d203ca 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Publisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Publisher.java @@ -4355,7 +4355,7 @@ public final BlockingIterable toIterable(int queueCapacityHint) { */ CapturedContext contextForSubscribe(AsyncContextProvider provider) { // the default behavior is to copy the map. Some operators may want to use shared map - return provider.captureContextCopy(); + return provider.captureContext(provider.context().copy()); } /** diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Single.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Single.java index 6158c45de6..ba63e7bb32 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Single.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Single.java @@ -2673,7 +2673,7 @@ public static Single zipDelayError(Function Date: Thu, 6 Feb 2025 11:28:29 -0700 Subject: [PATCH 07/14] bring back the ContextMap saved; fields for compatibility --- .../concurrent/api/AsyncContextProvider.java | 3 - .../api/ContextPreservingBiConsumer.java | 5 + .../api/ContextPreservingBiFunction.java | 2 +- .../api/ContextPreservingCallable.java | 13 +- .../api/ContextPreservingCancellable.java | 16 ++- ...rvingCancellableCompletableSubscriber.java | 12 +- ...PreservingCancellableSingleSubscriber.java | 12 +- .../ContextPreservingCompletableFuture.java | 112 ++++++++++-------- ...ontextPreservingCompletableSubscriber.java | 16 ++- ...ngCompletableSubscriberAndCancellable.java | 6 +- .../api/ContextPreservingConsumer.java | 13 +- .../api/ContextPreservingFunction.java | 11 +- .../api/ContextPreservingRunnable.java | 13 +- .../ContextPreservingSingleSubscriber.java | 16 ++- ...servingSingleSubscriberAndCancellable.java | 6 +- .../api/ContextPreservingSubscriber.java | 18 +-- ...xtPreservingSubscriberAndSubscription.java | 6 +- .../api/ContextPreservingSubscription.java | 16 ++- ...ntextPreservingSubscriptionSubscriber.java | 12 +- .../CustomCaptureAsyncContextProvider.java | 15 +++ .../api/DefaultAsyncContextProvider.java | 36 +++--- 21 files changed, 218 insertions(+), 141 deletions(-) diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java index 53a22db2e6..ea22cfee68 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java @@ -21,10 +21,7 @@ import io.servicetalk.concurrent.PublisherSource.Subscription; import io.servicetalk.concurrent.SingleSource; import io.servicetalk.context.api.ContextMap; -import io.servicetalk.context.api.ContextMapHolder; -import javax.annotation.Nonnull; -import javax.annotation.Nullable; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiConsumer.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiConsumer.java index f6c0be9a9f..5ac97ff41c 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiConsumer.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiConsumer.java @@ -15,18 +15,23 @@ */ package io.servicetalk.concurrent.api; +import io.servicetalk.context.api.ContextMap; + import java.util.function.BiConsumer; import static java.util.Objects.requireNonNull; final class ContextPreservingBiConsumer implements BiConsumer { + // TODO: remove after 0.42.55 + private final ContextMap saved; private final CapturedContext capturedContext; private final BiConsumer delegate; ContextPreservingBiConsumer(BiConsumer delegate, CapturedContext capturedContext) { this.capturedContext = requireNonNull(capturedContext); this.delegate = requireNonNull(delegate); + this.saved = capturedContext.captured(); } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java index 2882735156..3de5388f67 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java @@ -23,7 +23,7 @@ final class ContextPreservingBiFunction implements BiFunction { - // TODO: remove once we can get the java agents onto the new API. + // TODO: remove after 0.42.55 private final ContextMap saved; private final CapturedContext capturedContext; private final BiFunction delegate; diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCallable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCallable.java index 79fc4994ce..fe8d31cd5d 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCallable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCallable.java @@ -15,26 +15,31 @@ */ package io.servicetalk.concurrent.api; +import io.servicetalk.context.api.ContextMap; + import java.util.concurrent.Callable; import static java.util.Objects.requireNonNull; final class ContextPreservingCallable implements Callable { - private final CapturedContext saved; + // TODO: remove after 0.42.55 + private final ContextMap saved; + private final CapturedContext capturedContext; private final Callable delegate; ContextPreservingCallable(Callable delegate) { this(delegate, AsyncContext.provider().captureContext()); } - ContextPreservingCallable(Callable delegate, CapturedContext current) { - this.saved = requireNonNull(current); + ContextPreservingCallable(Callable delegate, CapturedContext capturedContext) { + this.capturedContext = requireNonNull(capturedContext); this.delegate = requireNonNull(delegate); + this.saved = this.capturedContext.captured(); } @Override public V call() throws Exception { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { return delegate.call(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellable.java index 77722f96c4..20a5d41cda 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellable.java @@ -16,29 +16,33 @@ package io.servicetalk.concurrent.api; import io.servicetalk.concurrent.Cancellable; +import io.servicetalk.context.api.ContextMap; import static java.util.Objects.requireNonNull; final class ContextPreservingCancellable implements Cancellable { - private final CapturedContext saved; + // TODO: remove after 0.42.55 + private final ContextMap saved; + private final CapturedContext capturedContext; private final Cancellable delegate; private ContextPreservingCancellable(Cancellable delegate, CapturedContext current) { - this.saved = requireNonNull(current); + this.capturedContext = requireNonNull(current); this.delegate = requireNonNull(delegate); + this.saved = capturedContext.captured(); } - static Cancellable wrap(Cancellable delegate, CapturedContext current) { + static Cancellable wrap(Cancellable delegate, CapturedContext capturedContext) { // The double wrapping can be observed when folks manually create a Single/Completable and directly call the // onSubscribe method. return delegate instanceof ContextPreservingCancellable && - ((ContextPreservingCancellable) delegate).saved == current ? delegate : - new ContextPreservingCancellable(delegate, current); + ((ContextPreservingCancellable) delegate).capturedContext == capturedContext ? delegate : + new ContextPreservingCancellable(delegate, capturedContext); } @Override public void cancel() { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { delegate.cancel(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableCompletableSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableCompletableSubscriber.java index 77789f37d1..566d64796b 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableCompletableSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableCompletableSubscriber.java @@ -16,22 +16,26 @@ package io.servicetalk.concurrent.api; import io.servicetalk.concurrent.Cancellable; +import io.servicetalk.context.api.ContextMap; import static io.servicetalk.concurrent.CompletableSource.Subscriber; import static java.util.Objects.requireNonNull; final class ContextPreservingCancellableCompletableSubscriber implements Subscriber { - final CapturedContext saved; + // TODO: remove after 0.42.55 + private final ContextMap saved; + final CapturedContext capturedContext; final Subscriber subscriber; - ContextPreservingCancellableCompletableSubscriber(Subscriber subscriber, CapturedContext current) { + ContextPreservingCancellableCompletableSubscriber(Subscriber subscriber, CapturedContext capturedContext) { this.subscriber = requireNonNull(subscriber); - this.saved = requireNonNull(current); + this.capturedContext = requireNonNull(capturedContext); + this.saved = capturedContext.captured(); } @Override public void onSubscribe(final Cancellable cancellable) { - subscriber.onSubscribe(ContextPreservingCancellable.wrap(cancellable, saved)); + subscriber.onSubscribe(ContextPreservingCancellable.wrap(cancellable, capturedContext)); } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableSingleSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableSingleSubscriber.java index 200fb4e518..6331454fa1 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableSingleSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellableSingleSubscriber.java @@ -18,23 +18,27 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.SingleSource; import io.servicetalk.concurrent.SingleSource.Subscriber; +import io.servicetalk.context.api.ContextMap; import javax.annotation.Nullable; import static java.util.Objects.requireNonNull; final class ContextPreservingCancellableSingleSubscriber implements Subscriber { - final CapturedContext saved; + // TODO: remove after 0.42.55 + private final ContextMap saved; + final CapturedContext capturedContext; final SingleSource.Subscriber subscriber; - ContextPreservingCancellableSingleSubscriber(Subscriber subscriber, CapturedContext current) { + ContextPreservingCancellableSingleSubscriber(Subscriber subscriber, CapturedContext capturedContext) { this.subscriber = requireNonNull(subscriber); - this.saved = requireNonNull(current); + this.capturedContext = requireNonNull(capturedContext); + this.saved = capturedContext.captured(); } @Override public void onSubscribe(final Cancellable cancellable) { - subscriber.onSubscribe(ContextPreservingCancellable.wrap(cancellable, saved)); + subscriber.onSubscribe(ContextPreservingCancellable.wrap(cancellable, capturedContext)); } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableFuture.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableFuture.java index b9cf40fd1d..d8e9decdc7 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableFuture.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableFuture.java @@ -15,6 +15,8 @@ */ package io.servicetalk.concurrent.api; +import io.servicetalk.context.api.ContextMap; + import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; @@ -31,12 +33,15 @@ import static java.util.Objects.requireNonNull; final class ContextPreservingCompletableFuture extends CompletableFuture { + // TODO: remove after 0.42.55 + private final ContextMap saved; private final CompletableFuture delegate; - private final CapturedContext saved; + private final CapturedContext capturedContext; - private ContextPreservingCompletableFuture(CompletableFuture delegate, CapturedContext current) { + private ContextPreservingCompletableFuture(CompletableFuture delegate, CapturedContext capturedContext) { this.delegate = requireNonNull(delegate); - this.saved = requireNonNull(current); + this.capturedContext = requireNonNull(capturedContext); + this.saved = capturedContext.captured(); } static ContextPreservingCompletableFuture newContextPreservingFuture(CompletableFuture original, @@ -50,71 +55,74 @@ static ContextPreservingCompletableFuture newContextPreservingFuture(Comp // CompletionStage begin @Override public CompletableFuture thenApply(final Function fn) { - return newContextPreservingFuture(delegate.thenApply(AsyncContext.provider().wrapFunction(fn, saved)), saved); + return newContextPreservingFuture(delegate.thenApply( + AsyncContext.provider().wrapFunction(fn, capturedContext)), capturedContext); } @Override public CompletableFuture thenApplyAsync(final Function fn) { - return newContextPreservingFuture(delegate.thenApplyAsync(AsyncContext.provider().wrapFunction(fn, saved)), - saved); + return newContextPreservingFuture(delegate.thenApplyAsync( + AsyncContext.provider().wrapFunction(fn, capturedContext)), + capturedContext); } @Override public CompletableFuture thenApplyAsync(final Function fn, final java.util.concurrent.Executor executor) { - return newContextPreservingFuture(delegate.thenApplyAsync(AsyncContext.provider().wrapFunction(fn, saved), - executor), saved); + return newContextPreservingFuture(delegate.thenApplyAsync( + AsyncContext.provider().wrapFunction(fn, capturedContext), executor), capturedContext); } @Override public CompletableFuture thenAccept(final Consumer action) { - return newContextPreservingFuture(delegate.thenAccept(AsyncContext.provider().wrapConsumer(action, saved)), - saved); + return newContextPreservingFuture(delegate.thenAccept( + AsyncContext.provider().wrapConsumer(action, capturedContext)), capturedContext); } @Override public CompletableFuture thenAcceptAsync(final Consumer action) { - return newContextPreservingFuture(delegate.thenAcceptAsync(AsyncContext.provider().wrapConsumer(action, saved)), - saved); + return newContextPreservingFuture(delegate.thenAcceptAsync( + AsyncContext.provider().wrapConsumer(action, capturedContext)), capturedContext); } @Override public CompletableFuture thenAcceptAsync(final Consumer action, final java.util.concurrent.Executor executor) { - return newContextPreservingFuture(delegate.thenAcceptAsync(AsyncContext.provider().wrapConsumer(action, saved), - executor), - saved); + return newContextPreservingFuture(delegate.thenAcceptAsync( + AsyncContext.provider().wrapConsumer(action, capturedContext), executor), capturedContext); } @Override public CompletableFuture thenRun(final Runnable action) { - return newContextPreservingFuture(delegate.thenRun(AsyncContext.provider().wrapRunnable(action, saved)), saved); + return newContextPreservingFuture(delegate.thenRun( + AsyncContext.provider().wrapRunnable(action, capturedContext)), capturedContext); } @Override public CompletableFuture thenRunAsync(final Runnable action) { - return newContextPreservingFuture(delegate.thenRunAsync(AsyncContext.provider().wrapRunnable(action, saved)), - saved); + return newContextPreservingFuture(delegate.thenRunAsync( + AsyncContext.provider().wrapRunnable(action, capturedContext)), + capturedContext); } @Override public CompletableFuture thenRunAsync(final Runnable action, final java.util.concurrent.Executor executor) { - return newContextPreservingFuture(delegate.thenRunAsync(AsyncContext.provider().wrapRunnable(action, saved), - executor), saved); + return newContextPreservingFuture(delegate.thenRunAsync(AsyncContext.provider().wrapRunnable(action, capturedContext), + executor), capturedContext); } @Override public CompletableFuture thenCombine(final CompletionStage other, final BiFunction fn) { return newContextPreservingFuture(delegate.thenCombine(other, - AsyncContext.provider().wrapBiFunction(fn, saved)), saved); + AsyncContext.provider().wrapBiFunction(fn, capturedContext)), capturedContext); } @Override public CompletableFuture thenCombineAsync(final CompletionStage other, final BiFunction fn) { return newContextPreservingFuture(delegate.thenCombineAsync(other, - AsyncContext.provider().wrapBiFunction(fn, saved)), saved); + AsyncContext.provider().wrapBiFunction(fn, capturedContext)), capturedContext); } @Override @@ -122,21 +130,21 @@ public CompletableFuture thenCombineAsync(final CompletionStage fn, final java.util.concurrent.Executor executor) { return newContextPreservingFuture(delegate.thenCombineAsync(other, - AsyncContext.provider().wrapBiFunction(fn, saved), executor), saved); + AsyncContext.provider().wrapBiFunction(fn, capturedContext), executor), capturedContext); } @Override public CompletableFuture thenAcceptBoth(final CompletionStage other, final BiConsumer action) { return newContextPreservingFuture(delegate.thenAcceptBoth(other, - AsyncContext.provider().wrapBiConsumer(action, saved)), saved); + AsyncContext.provider().wrapBiConsumer(action, capturedContext)), capturedContext); } @Override public CompletableFuture thenAcceptBothAsync(final CompletionStage other, final BiConsumer action) { return newContextPreservingFuture(delegate.thenAcceptBothAsync(other, - AsyncContext.provider().wrapBiConsumer(action, saved)), saved); + AsyncContext.provider().wrapBiConsumer(action, capturedContext)), capturedContext); } @Override @@ -144,40 +152,40 @@ public CompletableFuture thenAcceptBothAsync(final CompletionStage action, final java.util.concurrent.Executor executor) { return newContextPreservingFuture(delegate.thenAcceptBothAsync(other, - AsyncContext.provider().wrapBiConsumer(action, saved), executor), saved); + AsyncContext.provider().wrapBiConsumer(action, capturedContext), executor), capturedContext); } @Override public CompletableFuture runAfterBoth(final CompletionStage other, final Runnable action) { return newContextPreservingFuture(delegate.runAfterBoth(other, - AsyncContext.provider().wrapRunnable(action, saved)), saved); + AsyncContext.provider().wrapRunnable(action, capturedContext)), capturedContext); } @Override public CompletableFuture runAfterBothAsync(final CompletionStage other, final Runnable action) { return newContextPreservingFuture(delegate.runAfterBothAsync(other, - AsyncContext.provider().wrapRunnable(action, saved)), saved); + AsyncContext.provider().wrapRunnable(action, capturedContext)), capturedContext); } @Override public CompletableFuture runAfterBothAsync(final CompletionStage other, final Runnable action, final java.util.concurrent.Executor executor) { return newContextPreservingFuture(delegate.runAfterBothAsync(other, - AsyncContext.provider().wrapRunnable(action, saved), executor), saved); + AsyncContext.provider().wrapRunnable(action, capturedContext), executor), capturedContext); } @Override public CompletableFuture applyToEither(final CompletionStage other, final Function fn) { return newContextPreservingFuture(delegate.applyToEither(other, - AsyncContext.provider().wrapFunction(fn, saved)), saved); + AsyncContext.provider().wrapFunction(fn, capturedContext)), capturedContext); } @Override public CompletableFuture applyToEitherAsync(final CompletionStage other, final Function fn) { return newContextPreservingFuture(delegate.applyToEitherAsync(other, - AsyncContext.provider().wrapFunction(fn, saved)), saved); + AsyncContext.provider().wrapFunction(fn, capturedContext)), capturedContext); } @Override @@ -185,21 +193,21 @@ public CompletableFuture applyToEitherAsync(final CompletionStage fn, final java.util.concurrent.Executor executor) { return newContextPreservingFuture(delegate.applyToEitherAsync(other, - AsyncContext.provider().wrapFunction(fn, saved), executor), saved); + AsyncContext.provider().wrapFunction(fn, capturedContext), executor), capturedContext); } @Override public CompletableFuture acceptEither(final CompletionStage other, final Consumer action) { return newContextPreservingFuture(delegate.acceptEither(other, - AsyncContext.provider().wrapConsumer(action, saved)), saved); + AsyncContext.provider().wrapConsumer(action, capturedContext)), capturedContext); } @Override public CompletableFuture acceptEitherAsync(final CompletionStage other, final Consumer action) { return newContextPreservingFuture(delegate.acceptEitherAsync(other, - AsyncContext.provider().wrapConsumer(action, saved)), saved); + AsyncContext.provider().wrapConsumer(action, capturedContext)), capturedContext); } @Override @@ -207,87 +215,87 @@ public CompletableFuture acceptEitherAsync(final CompletionStage action, final java.util.concurrent.Executor executor) { return newContextPreservingFuture(delegate.acceptEitherAsync(other, - AsyncContext.provider().wrapConsumer(action, saved), executor), saved); + AsyncContext.provider().wrapConsumer(action, capturedContext), executor), capturedContext); } @Override public CompletableFuture runAfterEither(final CompletionStage other, final Runnable action) { return newContextPreservingFuture(delegate.runAfterEither(other, - AsyncContext.provider().wrapRunnable(action, saved)), saved); + AsyncContext.provider().wrapRunnable(action, capturedContext)), capturedContext); } @Override public CompletableFuture runAfterEitherAsync(final CompletionStage other, final Runnable action) { return newContextPreservingFuture(delegate.runAfterEitherAsync(other, - AsyncContext.provider().wrapRunnable(action, saved)), saved); + AsyncContext.provider().wrapRunnable(action, capturedContext)), capturedContext); } @Override public CompletableFuture runAfterEitherAsync(final CompletionStage other, final Runnable action, final java.util.concurrent.Executor executor) { return newContextPreservingFuture(delegate.runAfterEitherAsync(other, - AsyncContext.provider().wrapRunnable(action, saved), executor), saved); + AsyncContext.provider().wrapRunnable(action, capturedContext), executor), capturedContext); } @Override public CompletableFuture thenCompose(final Function> fn) { - return newContextPreservingFuture(delegate.thenCompose(AsyncContext.provider().wrapFunction(fn, saved)), saved); + return newContextPreservingFuture(delegate.thenCompose(AsyncContext.provider().wrapFunction(fn, capturedContext)), capturedContext); } @Override public CompletableFuture thenComposeAsync(final Function> fn) { - return newContextPreservingFuture(delegate.thenComposeAsync(AsyncContext.provider().wrapFunction(fn, saved)), - saved); + return newContextPreservingFuture(delegate.thenComposeAsync(AsyncContext.provider().wrapFunction(fn, capturedContext)), + capturedContext); } @Override public CompletableFuture thenComposeAsync(final Function> fn, final java.util.concurrent.Executor executor) { return newContextPreservingFuture(delegate.thenComposeAsync( - AsyncContext.provider().wrapFunction(fn, saved), executor), saved); + AsyncContext.provider().wrapFunction(fn, capturedContext), executor), capturedContext); } @Override public CompletableFuture exceptionally(final Function fn) { - return newContextPreservingFuture(delegate.exceptionally(AsyncContext.provider().wrapFunction(fn, saved)), - saved); + return newContextPreservingFuture(delegate.exceptionally(AsyncContext.provider().wrapFunction(fn, capturedContext)), + capturedContext); } @Override public CompletableFuture whenComplete(final BiConsumer action) { - return newContextPreservingFuture(delegate.whenComplete(AsyncContext.provider().wrapBiConsumer(action, saved)), - saved); + return newContextPreservingFuture(delegate.whenComplete(AsyncContext.provider().wrapBiConsumer(action, capturedContext)), + capturedContext); } @Override public CompletableFuture whenCompleteAsync(final BiConsumer action) { return newContextPreservingFuture(delegate.whenCompleteAsync( - AsyncContext.provider().wrapBiConsumer(action, saved)), saved); + AsyncContext.provider().wrapBiConsumer(action, capturedContext)), capturedContext); } @Override public CompletableFuture whenCompleteAsync(final BiConsumer action, final java.util.concurrent.Executor executor) { return newContextPreservingFuture(delegate.whenCompleteAsync( - AsyncContext.provider().wrapBiConsumer(action, saved), executor), saved); + AsyncContext.provider().wrapBiConsumer(action, capturedContext), executor), capturedContext); } @Override public CompletableFuture handle(final BiFunction fn) { - return newContextPreservingFuture(delegate.handle(AsyncContext.provider().wrapBiFunction(fn, saved)), saved); + return newContextPreservingFuture(delegate.handle(AsyncContext.provider().wrapBiFunction(fn, capturedContext)), capturedContext); } @Override public CompletableFuture handleAsync(final BiFunction fn) { - return newContextPreservingFuture(delegate.handleAsync(AsyncContext.provider().wrapBiFunction(fn, saved)), - saved); + return newContextPreservingFuture(delegate.handleAsync(AsyncContext.provider().wrapBiFunction(fn, capturedContext)), + capturedContext); } @Override public CompletableFuture handleAsync(final BiFunction fn, final Executor executor) { return newContextPreservingFuture(delegate.handleAsync( - AsyncContext.provider().wrapBiFunction(fn, saved), executor), saved); + AsyncContext.provider().wrapBiFunction(fn, capturedContext), executor), capturedContext); } // CompletionStage end diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriber.java index fa24a3b4b3..217ddc2351 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriber.java @@ -17,16 +17,20 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.CompletableSource.Subscriber; +import io.servicetalk.context.api.ContextMap; import static java.util.Objects.requireNonNull; class ContextPreservingCompletableSubscriber implements Subscriber { - final CapturedContext saved; + // TODO: remove after 0.42.55 + private final ContextMap saved; + final CapturedContext capturedContext; final Subscriber subscriber; - ContextPreservingCompletableSubscriber(Subscriber subscriber, CapturedContext current) { + ContextPreservingCompletableSubscriber(Subscriber subscriber, CapturedContext capturedContext) { this.subscriber = requireNonNull(subscriber); - this.saved = requireNonNull(current); + this.capturedContext = requireNonNull(capturedContext); + this.saved = capturedContext.captured(); } void invokeOnSubscribe(Cancellable cancellable) { @@ -35,21 +39,21 @@ void invokeOnSubscribe(Cancellable cancellable) { @Override public final void onSubscribe(final Cancellable cancellable) { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { invokeOnSubscribe(cancellable); } } @Override public final void onComplete() { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { subscriber.onComplete(); } } @Override public final void onError(Throwable t) { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { subscriber.onError(t); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriberAndCancellable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriberAndCancellable.java index 3476c4edde..bf81936892 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriberAndCancellable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriberAndCancellable.java @@ -19,12 +19,12 @@ import io.servicetalk.concurrent.CompletableSource.Subscriber; final class ContextPreservingCompletableSubscriberAndCancellable extends ContextPreservingCompletableSubscriber { - ContextPreservingCompletableSubscriberAndCancellable(Subscriber subscriber, CapturedContext current) { - super(subscriber, current); + ContextPreservingCompletableSubscriberAndCancellable(Subscriber subscriber, CapturedContext capturedContext) { + super(subscriber, capturedContext); } @Override void invokeOnSubscribe(Cancellable cancellable) { - subscriber.onSubscribe(ContextPreservingCancellable.wrap(cancellable, saved)); + subscriber.onSubscribe(ContextPreservingCancellable.wrap(cancellable, capturedContext)); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingConsumer.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingConsumer.java index 4ad5703be8..7aa1064943 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingConsumer.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingConsumer.java @@ -15,22 +15,27 @@ */ package io.servicetalk.concurrent.api; +import io.servicetalk.context.api.ContextMap; + import java.util.function.Consumer; import static java.util.Objects.requireNonNull; final class ContextPreservingConsumer implements Consumer { - private final CapturedContext saved; + // TODO: remove after 0.42.55 + private final ContextMap saved; + private final CapturedContext capturedContext; private final Consumer delegate; - ContextPreservingConsumer(Consumer delegate, CapturedContext current) { - this.saved = requireNonNull(current); + ContextPreservingConsumer(Consumer delegate, CapturedContext capturedContext) { + this.capturedContext = requireNonNull(capturedContext); this.delegate = requireNonNull(delegate); + this.saved = capturedContext.captured(); } @Override public void accept(T t) { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { delegate.accept(t); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingFunction.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingFunction.java index 9ed8bdf9e8..21aff097fc 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingFunction.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingFunction.java @@ -15,22 +15,27 @@ */ package io.servicetalk.concurrent.api; +import io.servicetalk.context.api.ContextMap; + import java.util.function.Function; import static java.util.Objects.requireNonNull; final class ContextPreservingFunction implements Function { - private final CapturedContext saved; + // TODO: remove after 0.42.55 + private final ContextMap saved; + private final CapturedContext capturedContext; private final Function delegate; ContextPreservingFunction(Function delegate, CapturedContext capturedContext) { - this.saved = requireNonNull(capturedContext); + this.capturedContext = requireNonNull(capturedContext); this.delegate = requireNonNull(delegate); + this.saved = capturedContext.captured(); } @Override public U apply(T t) { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { return delegate.apply(t); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingRunnable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingRunnable.java index d8ba8b2027..e97dd04941 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingRunnable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingRunnable.java @@ -15,24 +15,29 @@ */ package io.servicetalk.concurrent.api; +import io.servicetalk.context.api.ContextMap; + import static java.util.Objects.requireNonNull; final class ContextPreservingRunnable implements Runnable { - private final CapturedContext saved; + // TODO: remove after 0.42.55 + private final ContextMap saved; + private final CapturedContext capturedContext; private final Runnable delegate; ContextPreservingRunnable(Runnable delegate) { this(delegate, AsyncContext.provider().captureContext()); } - ContextPreservingRunnable(Runnable delegate, CapturedContext current) { - this.saved = requireNonNull(current); + ContextPreservingRunnable(Runnable delegate, CapturedContext capturedContext) { + this.capturedContext = requireNonNull(capturedContext); this.delegate = requireNonNull(delegate); + this.saved = capturedContext.captured(); } @Override public void run() { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { delegate.run(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriber.java index 9ab5b07610..dc4242f77c 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriber.java @@ -18,18 +18,22 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.SingleSource; import io.servicetalk.concurrent.SingleSource.Subscriber; +import io.servicetalk.context.api.ContextMap; import javax.annotation.Nullable; import static java.util.Objects.requireNonNull; class ContextPreservingSingleSubscriber implements Subscriber { - final CapturedContext saved; + // TODO: remove after 0.42.55 + private final ContextMap saved; + final CapturedContext capturedContext; final SingleSource.Subscriber subscriber; - ContextPreservingSingleSubscriber(Subscriber subscriber, CapturedContext current) { + ContextPreservingSingleSubscriber(Subscriber subscriber, CapturedContext capturedContext) { this.subscriber = requireNonNull(subscriber); - this.saved = requireNonNull(current); + this.capturedContext = requireNonNull(capturedContext); + this.saved = capturedContext.captured(); } void invokeOnSubscribe(Cancellable cancellable) { @@ -38,21 +42,21 @@ void invokeOnSubscribe(Cancellable cancellable) { @Override public final void onSubscribe(Cancellable cancellable) { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { invokeOnSubscribe(cancellable); } } @Override public final void onSuccess(@Nullable T result) { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { subscriber.onSuccess(result); } } @Override public final void onError(Throwable t) { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { subscriber.onError(t); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriberAndCancellable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriberAndCancellable.java index c2d3c00e9e..b2c6b00eed 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriberAndCancellable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriberAndCancellable.java @@ -19,12 +19,12 @@ import io.servicetalk.concurrent.SingleSource.Subscriber; final class ContextPreservingSingleSubscriberAndCancellable extends ContextPreservingSingleSubscriber { - ContextPreservingSingleSubscriberAndCancellable(Subscriber subscriber, CapturedContext current) { - super(subscriber, current); + ContextPreservingSingleSubscriberAndCancellable(Subscriber subscriber, CapturedContext capturedContext) { + super(subscriber, capturedContext); } @Override void invokeOnSubscribe(Cancellable cancellable) { - subscriber.onSubscribe(ContextPreservingCancellable.wrap(cancellable, saved)); + subscriber.onSubscribe(ContextPreservingCancellable.wrap(cancellable, capturedContext)); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriber.java index 2f39570b62..b440c8ed25 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriber.java @@ -17,16 +17,20 @@ import io.servicetalk.concurrent.PublisherSource.Subscriber; import io.servicetalk.concurrent.PublisherSource.Subscription; +import io.servicetalk.context.api.ContextMap; import static java.util.Objects.requireNonNull; class ContextPreservingSubscriber implements Subscriber { - final CapturedContext saved; + // TODO: remove after 0.42.55 + private final ContextMap saved; + final CapturedContext capturedContext; final Subscriber subscriber; - ContextPreservingSubscriber(Subscriber subscriber, CapturedContext current) { + ContextPreservingSubscriber(Subscriber subscriber, CapturedContext capturedContext) { this.subscriber = requireNonNull(subscriber); - this.saved = requireNonNull(current); + this.capturedContext = requireNonNull(capturedContext); + this.saved = capturedContext.captured(); } void invokeOnSubscribe(Subscription s) { @@ -35,28 +39,28 @@ void invokeOnSubscribe(Subscription s) { @Override public final void onSubscribe(Subscription s) { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { invokeOnSubscribe(s); } } @Override public final void onNext(T t) { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { subscriber.onNext(t); } } @Override public final void onError(Throwable t) { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { subscriber.onError(t); } } @Override public final void onComplete() { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { subscriber.onComplete(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriberAndSubscription.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriberAndSubscription.java index c86e57901c..7c4a238e4a 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriberAndSubscription.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriberAndSubscription.java @@ -19,12 +19,12 @@ import io.servicetalk.concurrent.PublisherSource.Subscriber; final class ContextPreservingSubscriberAndSubscription extends ContextPreservingSubscriber { - ContextPreservingSubscriberAndSubscription(Subscriber subscriber, CapturedContext current) { - super(subscriber, current); + ContextPreservingSubscriberAndSubscription(Subscriber subscriber, CapturedContext capturedContext) { + super(subscriber, capturedContext); } @Override void invokeOnSubscribe(PublisherSource.Subscription s) { - subscriber.onSubscribe(ContextPreservingSubscription.wrap(s, saved)); + subscriber.onSubscribe(ContextPreservingSubscription.wrap(s, capturedContext)); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscription.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscription.java index e26232cccb..8d3436135d 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscription.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscription.java @@ -16,34 +16,38 @@ package io.servicetalk.concurrent.api; import io.servicetalk.concurrent.PublisherSource.Subscription; +import io.servicetalk.context.api.ContextMap; import static java.util.Objects.requireNonNull; final class ContextPreservingSubscription implements Subscription { - private final CapturedContext saved; + // TODO: remove after 0.42.55 + private final ContextMap saved; + private final CapturedContext capturedContext; private final Subscription subscription; - private ContextPreservingSubscription(Subscription subscription, CapturedContext current) { + private ContextPreservingSubscription(Subscription subscription, CapturedContext capturedContext) { this.subscription = requireNonNull(subscription); - this.saved = requireNonNull(current); + this.capturedContext = requireNonNull(capturedContext); + this.saved = capturedContext.captured(); } static Subscription wrap(Subscription subscription, CapturedContext current) { return subscription instanceof ContextPreservingSubscription && - ((ContextPreservingSubscription) subscription).saved == current ? subscription : + ((ContextPreservingSubscription) subscription).capturedContext == current ? subscription : new ContextPreservingSubscription(subscription, current); } @Override public void request(long l) { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { subscription.request(l); } } @Override public void cancel() { - try (Scope ignored = saved.restoreContext()) { + try (Scope ignored = capturedContext.restoreContext()) { subscription.cancel(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriptionSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriptionSubscriber.java index 099f899561..8b2569eebf 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriptionSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriptionSubscriber.java @@ -17,21 +17,25 @@ import io.servicetalk.concurrent.PublisherSource.Subscriber; import io.servicetalk.concurrent.PublisherSource.Subscription; +import io.servicetalk.context.api.ContextMap; import static java.util.Objects.requireNonNull; final class ContextPreservingSubscriptionSubscriber implements Subscriber { - final CapturedContext saved; + // TODO: remove after 0.42.55 + private final ContextMap saved; + final CapturedContext capturedContext; final Subscriber subscriber; - ContextPreservingSubscriptionSubscriber(Subscriber subscriber, CapturedContext current) { + ContextPreservingSubscriptionSubscriber(Subscriber subscriber, CapturedContext capturedContext) { this.subscriber = requireNonNull(subscriber); - this.saved = requireNonNull(current); + this.capturedContext = requireNonNull(capturedContext); + this.saved = capturedContext.captured(); } @Override public void onSubscribe(final Subscription subscription) { - subscriber.onSubscribe(ContextPreservingSubscription.wrap(subscription, saved)); + subscriber.onSubscribe(ContextPreservingSubscription.wrap(subscription, capturedContext)); } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java index bd2ae89228..ef915bf7ee 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CustomCaptureAsyncContextProvider.java @@ -1,3 +1,18 @@ +/* + * Copyright © 2025 Apple Inc. and the ServiceTalk project authors + * + * Licensed 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 io.servicetalk.concurrent.api; import io.servicetalk.context.api.ContextMap; diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java index 52caf53c4b..dccb223a38 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java @@ -102,12 +102,12 @@ public final CompletableSource.Subscriber wrapCancellable(final CompletableSourc final CapturedContext context) { if (subscriber instanceof ContextPreservingCompletableSubscriber) { final ContextPreservingCompletableSubscriber s = (ContextPreservingCompletableSubscriber) subscriber; - if (s.saved == context) { + if (s.capturedContext == context) { return subscriber instanceof ContextPreservingCompletableSubscriberAndCancellable ? subscriber : new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); } } else if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber && - ((ContextPreservingCancellableCompletableSubscriber) subscriber).saved == context) { + ((ContextPreservingCancellableCompletableSubscriber) subscriber).capturedContext == context) { // no need to check for instanceof ContextPreservingCompletableSubscriberAndCancellable, because // it extends from ContextPreservingSingleSubscriber. return subscriber; @@ -121,12 +121,12 @@ public final CompletableSource.Subscriber wrapCompletableSubscriber(final Comple if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber) { final ContextPreservingCancellableCompletableSubscriber s = (ContextPreservingCancellableCompletableSubscriber) subscriber; - if (s.saved == context) { + if (s.capturedContext == context) { // replace current wrapper with wrapper that includes Subscriber and Cancellable return new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); } } else if (subscriber instanceof ContextPreservingCompletableSubscriber && - ((ContextPreservingCompletableSubscriber) subscriber).saved == context) { + ((ContextPreservingCompletableSubscriber) subscriber).capturedContext == context) { // no need to check for instanceof ContextPreservingCompletableSubscriberAndCancellable, because // it extends from ContextPreservingCompletableSubscriber. return subscriber; @@ -139,14 +139,14 @@ public final CompletableSource.Subscriber wrapCompletableSubscriberAndCancellabl final CompletableSource.Subscriber subscriber, final CapturedContext context) { if (subscriber instanceof ContextPreservingCompletableSubscriber) { final ContextPreservingCompletableSubscriber s = (ContextPreservingCompletableSubscriber) subscriber; - if (s.saved == context) { + if (s.capturedContext == context) { return subscriber instanceof ContextPreservingCompletableSubscriberAndCancellable ? subscriber : new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); } } else if (subscriber instanceof ContextPreservingCancellableCompletableSubscriber) { final ContextPreservingCancellableCompletableSubscriber s = (ContextPreservingCancellableCompletableSubscriber) subscriber; - if (s.saved == context) { + if (s.capturedContext == context) { return new ContextPreservingCompletableSubscriberAndCancellable(s.subscriber, context); } } @@ -158,12 +158,12 @@ public final SingleSource.Subscriber wrapCancellable(final SingleSource.S final CapturedContext context) { if (subscriber instanceof ContextPreservingSingleSubscriber) { final ContextPreservingSingleSubscriber s = (ContextPreservingSingleSubscriber) subscriber; - if (s.saved == context) { + if (s.capturedContext == context) { return subscriber instanceof ContextPreservingSingleSubscriberAndCancellable ? subscriber : new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); } } else if (subscriber instanceof ContextPreservingCancellableSingleSubscriber && - ((ContextPreservingCancellableSingleSubscriber) subscriber).saved == context) { + ((ContextPreservingCancellableSingleSubscriber) subscriber).capturedContext == context) { // no need to check for instanceof ContextPreservingSingleSubscriberAndCancellable, because // it extends from ContextPreservingSingleSubscriber. return subscriber; @@ -177,11 +177,11 @@ public final SingleSource.Subscriber wrapSingleSubscriber(final SingleSou if (subscriber instanceof ContextPreservingCancellableSingleSubscriber) { final ContextPreservingCancellableSingleSubscriber s = (ContextPreservingCancellableSingleSubscriber) subscriber; - if (s.saved == context) { + if (s.capturedContext == context) { return new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); } } else if (subscriber instanceof ContextPreservingSingleSubscriber && - ((ContextPreservingSingleSubscriber) subscriber).saved == context) { + ((ContextPreservingSingleSubscriber) subscriber).capturedContext == context) { // no need to check for instanceof ContextPreservingSingleSubscriberAndCancellable, because // it extends from ContextPreservingSingleSubscriber. return subscriber; @@ -194,14 +194,14 @@ public final SingleSource.Subscriber wrapSingleSubscriberAndCancellable( final SingleSource.Subscriber subscriber, final CapturedContext context) { if (subscriber instanceof ContextPreservingSingleSubscriber) { final ContextPreservingSingleSubscriber s = (ContextPreservingSingleSubscriber) subscriber; - if (s.saved == context) { + if (s.capturedContext == context) { return subscriber instanceof ContextPreservingSingleSubscriberAndCancellable ? subscriber : new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); } } else if (subscriber instanceof ContextPreservingCancellableSingleSubscriber) { final ContextPreservingCancellableSingleSubscriber s = (ContextPreservingCancellableSingleSubscriber) subscriber; - if (s.saved == context) { + if (s.capturedContext == context) { return new ContextPreservingSingleSubscriberAndCancellable<>(s.subscriber, context); } } @@ -212,12 +212,12 @@ public final SingleSource.Subscriber wrapSingleSubscriberAndCancellable( public final PublisherSource.Subscriber wrapSubscription(final PublisherSource.Subscriber subscriber, final CapturedContext context) { if (subscriber instanceof ContextPreservingSubscriber) { final ContextPreservingSubscriber s = (ContextPreservingSubscriber) subscriber; - if (s.saved == context) { + if (s.capturedContext == context) { return subscriber instanceof ContextPreservingSubscriberAndSubscription ? subscriber : new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); } } else if (subscriber instanceof ContextPreservingSubscriptionSubscriber && - ((ContextPreservingSubscriptionSubscriber) subscriber).saved == context) { + ((ContextPreservingSubscriptionSubscriber) subscriber).capturedContext == context) { // no need to check for instanceof ContextPreservingSubscriberAndSubscription, because // it extends from ContextPreservingSubscriptionSubscriber. return subscriber; @@ -230,11 +230,11 @@ public final PublisherSource.Subscriber wrapPublisherSubscriber(final Pub if (subscriber instanceof ContextPreservingSubscriptionSubscriber) { final ContextPreservingSubscriptionSubscriber s = (ContextPreservingSubscriptionSubscriber) subscriber; - if (s.saved == context) { + if (s.capturedContext == context) { return new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); } } else if (subscriber instanceof ContextPreservingSubscriber && - ((ContextPreservingSubscriber) subscriber).saved == context) { + ((ContextPreservingSubscriber) subscriber).capturedContext == context) { // no need to check for instanceof ContextPreservingSubscriberAndSubscription, because // it extends from ContextPreservingSubscriptionSubscriber. return subscriber; @@ -247,14 +247,14 @@ public final PublisherSource.Subscriber wrapPublisherSubscriberAndSubscri final CapturedContext context) { if (subscriber instanceof ContextPreservingSubscriber) { final ContextPreservingSubscriber s = (ContextPreservingSubscriber) subscriber; - if (s.saved == context) { + if (s.capturedContext == context) { return subscriber instanceof ContextPreservingSubscriberAndSubscription ? subscriber : new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); } } else if (subscriber instanceof ContextPreservingSubscriptionSubscriber) { final ContextPreservingSubscriptionSubscriber s = (ContextPreservingSubscriptionSubscriber) subscriber; - if (s.saved == context) { + if (s.capturedContext == context) { return new ContextPreservingSubscriberAndSubscription<>(s.subscriber, context); } } From 8bb46d5dee8a68fdad43edb58f9d49f5ec36cc17 Mon Sep 17 00:00:00 2001 From: Bryce Anderson Date: Thu, 6 Feb 2025 11:48:54 -0700 Subject: [PATCH 08/14] Fix linter warnings --- ...tractCompletableAndSingleConcatenated.java | 3 +- .../concurrent/api/CapturedContext.java | 2 +- .../CompletableConcatWithCompletables.java | 3 +- .../api/CompletableConcatWithSingle.java | 6 ++-- .../api/CompletableMergeWithPublisher.java | 3 +- .../api/CompletableToPublisher.java | 3 +- .../ContextPreservingCompletableFuture.java | 26 +++++++------- .../api/DefaultAsyncContextProvider.java | 36 +++++++++---------- .../concurrent/api/PublisherGroupBy.java | 5 +-- .../concurrent/api/RepeatWhenSingle.java | 3 +- .../concurrent/api/RetrySingle.java | 5 +-- .../api/SingleConcatWithCompletable.java | 3 +- .../api/SingleToCompletableFuture.java | 1 - .../concurrent/api/TimeoutSingle.java | 3 +- 14 files changed, 56 insertions(+), 46 deletions(-) diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractCompletableAndSingleConcatenated.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractCompletableAndSingleConcatenated.java index 990ed7bfa7..f50c5fb096 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractCompletableAndSingleConcatenated.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AbstractCompletableAndSingleConcatenated.java @@ -26,7 +26,8 @@ abstract class AbstractCompletableAndSingleConcatenated extends AbstractNoHan @Override protected void handleSubscribe(final Subscriber subscriber, final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { - final Subscriber wrappedSubscriber = contextProvider.wrapSingleSubscriber(subscriber, capturedContext); + final Subscriber wrappedSubscriber = + contextProvider.wrapSingleSubscriber(subscriber, capturedContext); delegateSubscribeToOriginal(wrappedSubscriber, capturedContext, contextProvider); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java index 40ccc4ffee..9af3dcb851 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java @@ -25,7 +25,7 @@ interface CapturedContext { /** * The {@link ContextMap} that was captured as part of the context. - * @return + * @return {@link ContextMap} that was captured as part of the context. */ ContextMap captured(); diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithCompletables.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithCompletables.java index bcd1a5b821..eb7e9e44c9 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithCompletables.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithCompletables.java @@ -39,7 +39,8 @@ final class CompletableConcatWithCompletables extends AbstractNoHandleSubscribeC protected void handleSubscribe(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { Subscriber wrappedSubscriber = contextProvider.wrapCompletableSubscriber(subscriber, capturedContext); - original.delegateSubscribe(new ConcatWithSubscriber(wrappedSubscriber, nexts), capturedContext, contextProvider); + original.delegateSubscribe( + new ConcatWithSubscriber(wrappedSubscriber, nexts), capturedContext, contextProvider); } private static final class ConcatWithSubscriber implements Subscriber { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithSingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithSingle.java index 54b922c7e9..9ea8c5dcc5 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithSingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableConcatWithSingle.java @@ -35,8 +35,10 @@ final class CompletableConcatWithSingle extends AbstractCompletableAndSingleC @Override void delegateSubscribeToOriginal(final Subscriber offloadSubscriber, - final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { - original.delegateSubscribe(new ConcatWithSubscriber<>(offloadSubscriber, next), capturedContext, contextProvider); + final CapturedContext capturedContext, + final AsyncContextProvider contextProvider) { + original.delegateSubscribe(new ConcatWithSubscriber<>(offloadSubscriber, next), + capturedContext, contextProvider); } private static final class ConcatWithSubscriber extends AbstractConcatWithSubscriber { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableMergeWithPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableMergeWithPublisher.java index 2dc991c6cb..f7726455f4 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableMergeWithPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableMergeWithPublisher.java @@ -184,7 +184,8 @@ private static final class Merger implements Subscriber { private Throwable completableError; private volatile int state; - Merger(Subscriber subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { + Merger(Subscriber subscriber, CapturedContext capturedContext, + AsyncContextProvider contextProvider) { this.wrappedSubscriber = contextProvider.wrapPublisherSubscriber(subscriber, capturedContext); completableSubscriber = new CompletableSubscriber(); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableToPublisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableToPublisher.java index 6f145befc5..9280baab07 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableToPublisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableToPublisher.java @@ -56,7 +56,8 @@ private static final class ConversionSubscriber extends SequentialCancellable private volatile int terminated; private ConversionSubscriber(Subscriber subscriber, - final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, + final AsyncContextProvider contextProvider) { this.subscriber = subscriber; this.capturedContext = capturedContext; this.contextProvider = contextProvider; diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableFuture.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableFuture.java index d8e9decdc7..17e393e8e8 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableFuture.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableFuture.java @@ -107,8 +107,8 @@ public CompletableFuture thenRunAsync(final Runnable action) { @Override public CompletableFuture thenRunAsync(final Runnable action, final java.util.concurrent.Executor executor) { - return newContextPreservingFuture(delegate.thenRunAsync(AsyncContext.provider().wrapRunnable(action, capturedContext), - executor), capturedContext); + return newContextPreservingFuture(delegate.thenRunAsync( + AsyncContext.provider().wrapRunnable(action, capturedContext), executor), capturedContext); } @Override @@ -239,13 +239,14 @@ public CompletableFuture runAfterEitherAsync(final CompletionStage othe @Override public CompletableFuture thenCompose(final Function> fn) { - return newContextPreservingFuture(delegate.thenCompose(AsyncContext.provider().wrapFunction(fn, capturedContext)), capturedContext); + return newContextPreservingFuture(delegate.thenCompose( + AsyncContext.provider().wrapFunction(fn, capturedContext)), capturedContext); } @Override public CompletableFuture thenComposeAsync(final Function> fn) { - return newContextPreservingFuture(delegate.thenComposeAsync(AsyncContext.provider().wrapFunction(fn, capturedContext)), - capturedContext); + return newContextPreservingFuture(delegate.thenComposeAsync( + AsyncContext.provider().wrapFunction(fn, capturedContext)), capturedContext); } @Override @@ -257,14 +258,14 @@ public CompletableFuture thenComposeAsync(final Function exceptionally(final Function fn) { - return newContextPreservingFuture(delegate.exceptionally(AsyncContext.provider().wrapFunction(fn, capturedContext)), - capturedContext); + return newContextPreservingFuture(delegate.exceptionally( + AsyncContext.provider().wrapFunction(fn, capturedContext)), capturedContext); } @Override public CompletableFuture whenComplete(final BiConsumer action) { - return newContextPreservingFuture(delegate.whenComplete(AsyncContext.provider().wrapBiConsumer(action, capturedContext)), - capturedContext); + return newContextPreservingFuture(delegate.whenComplete( + AsyncContext.provider().wrapBiConsumer(action, capturedContext)), capturedContext); } @Override @@ -282,13 +283,14 @@ public CompletableFuture whenCompleteAsync(final BiConsumer CompletableFuture handle(final BiFunction fn) { - return newContextPreservingFuture(delegate.handle(AsyncContext.provider().wrapBiFunction(fn, capturedContext)), capturedContext); + return newContextPreservingFuture(delegate.handle( + AsyncContext.provider().wrapBiFunction(fn, capturedContext)), capturedContext); } @Override public CompletableFuture handleAsync(final BiFunction fn) { - return newContextPreservingFuture(delegate.handleAsync(AsyncContext.provider().wrapBiFunction(fn, capturedContext)), - capturedContext); + return newContextPreservingFuture(delegate.handleAsync( + AsyncContext.provider().wrapBiFunction(fn, capturedContext)), capturedContext); } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java index dccb223a38..defe5cbfaa 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java @@ -20,12 +20,10 @@ import io.servicetalk.concurrent.SingleSource; import io.servicetalk.context.api.ContextMap; import io.servicetalk.context.api.ContextMapHolder; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -50,7 +48,6 @@ class DefaultAsyncContextProvider implements AsyncContextProvider { protected DefaultAsyncContextProvider() { } - @Nonnull @Override public final ContextMap context() { final Thread t = Thread.currentThread(); @@ -68,13 +65,11 @@ public final ContextMap context() { } @Override - public final void context(@Nullable ContextMap contextMap) { + public final void context(ContextMap contextMap) { final Thread currentThread = Thread.currentThread(); if (currentThread instanceof ContextMapHolder) { final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; asyncContextMapHolder.context(contextMap); - } else if (contextMap == null) { - CONTEXT_THREAD_LOCAL.remove(); } else { CONTEXT_THREAD_LOCAL.set(contextMap); } @@ -209,7 +204,8 @@ public final SingleSource.Subscriber wrapSingleSubscriberAndCancellable( } @Override - public final PublisherSource.Subscriber wrapSubscription(final PublisherSource.Subscriber subscriber, final CapturedContext context) { + public final PublisherSource.Subscriber wrapSubscription( + final PublisherSource.Subscriber subscriber, final CapturedContext context) { if (subscriber instanceof ContextPreservingSubscriber) { final ContextPreservingSubscriber s = (ContextPreservingSubscriber) subscriber; if (s.capturedContext == context) { @@ -226,7 +222,8 @@ public final PublisherSource.Subscriber wrapSubscription(final PublisherS } @Override - public final PublisherSource.Subscriber wrapPublisherSubscriber(final PublisherSource.Subscriber subscriber, final CapturedContext context) { + public final PublisherSource.Subscriber wrapPublisherSubscriber( + final PublisherSource.Subscriber subscriber, final CapturedContext context) { if (subscriber instanceof ContextPreservingSubscriptionSubscriber) { final ContextPreservingSubscriptionSubscriber s = (ContextPreservingSubscriptionSubscriber) subscriber; @@ -243,8 +240,8 @@ public final PublisherSource.Subscriber wrapPublisherSubscriber(final Pub } @Override - public final PublisherSource.Subscriber wrapPublisherSubscriberAndSubscription(final PublisherSource.Subscriber subscriber, - final CapturedContext context) { + public final PublisherSource.Subscriber wrapPublisherSubscriberAndSubscription( + final PublisherSource.Subscriber subscriber, final CapturedContext context) { if (subscriber instanceof ContextPreservingSubscriber) { final ContextPreservingSubscriber s = (ContextPreservingSubscriber) subscriber; if (s.capturedContext == context) { @@ -272,7 +269,8 @@ public final ExecutorService wrapJdkExecutorService(final ExecutorService execut } @Override - public final io.servicetalk.concurrent.api.Executor wrapExecutor(final io.servicetalk.concurrent.api.Executor executor) { + public final io.servicetalk.concurrent.api.Executor wrapExecutor( + final io.servicetalk.concurrent.api.Executor executor) { return ContextPreservingStExecutor.of(executor); } @@ -308,12 +306,14 @@ public final Function wrapFunction(final Function func, final } @Override - public final BiConsumer wrapBiConsumer(final BiConsumer consumer, final CapturedContext context) { + public final BiConsumer wrapBiConsumer( + final BiConsumer consumer, final CapturedContext context) { return new ContextPreservingBiConsumer<>(consumer, context); } @Override - public final BiFunction wrapBiFunction(final BiFunction func, final CapturedContext context) { + public final BiFunction wrapBiFunction( + final BiFunction func, final CapturedContext context) { return new ContextPreservingBiFunction<>(func, context); } @@ -356,11 +356,9 @@ private static ContextMap exchangeContext(ContextMap contextMap) { private static void detachContext(ContextMap expectedContext, ContextMap toRestore) { ContextMap current = exchangeContext(toRestore); - if (current != expectedContext) { - if (!NO_DEBUG_LOGGING) { - LOGGER.debug("Current context didn't match the expected context. current: {}, expected: {}", - current, expectedContext, new Throwable("stack trace")); - } + if (current != expectedContext && !NO_DEBUG_LOGGING) { + LOGGER.debug("Current context didn't match the expected context. current: {}, expected: {}", + current, expectedContext, new Throwable("stack trace")); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherGroupBy.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherGroupBy.java index 924567604e..6da2d7589f 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherGroupBy.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherGroupBy.java @@ -42,8 +42,9 @@ final class PublisherGroupBy extends AbstractPublisherGroupBy { @Override void handleSubscribe(Subscriber> subscriber, CapturedContext capturedContext, AsyncContextProvider contextProvider) { - original.delegateSubscribe(new GroupBySubscriber(subscriber, queueLimit, initialCapacityForGroups, capturedContext, - contextProvider), capturedContext, contextProvider); + original.delegateSubscribe(new GroupBySubscriber( + subscriber, queueLimit, initialCapacityForGroups, capturedContext, contextProvider), + capturedContext, contextProvider); } private final class GroupBySubscriber extends AbstractGroupBySubscriber { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RepeatWhenSingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RepeatWhenSingle.java index c9c721bf8c..4c3c0020c8 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RepeatWhenSingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RepeatWhenSingle.java @@ -147,7 +147,8 @@ public void onComplete() { // Either we copy the map up front before subscribe, or we just re-use the same map and // let the async source at the top of the chain reset if necessary. We currently choose // the second option. - outer.original.delegateSubscribe(RepeatSubscriber.this, capturedContext, contextProvider); + outer.original.delegateSubscribe( + RepeatSubscriber.this, capturedContext, contextProvider); } break; } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RetrySingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RetrySingle.java index cd71fe6ec8..9ff6af08ea 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RetrySingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/RetrySingle.java @@ -43,8 +43,9 @@ void handleSubscribe(final Subscriber subscriber, // resubscribe. This allows for async context to be shared across each request retry, and follows the same // shared state model as the request object on the client. If copy-on-each-resubscribe is desired this could // be provided by an independent operator, or manually cleared/overwritten. - original.delegateSubscribe(new RetrySubscriber<>(new SequentialCancellable(), this, subscriber, 0, capturedContext, - contextProvider), capturedContext, contextProvider); + original.delegateSubscribe(new RetrySubscriber<>( + new SequentialCancellable(), this, subscriber, 0, capturedContext, contextProvider), + capturedContext, contextProvider); } abstract static class AbstractRetrySubscriber implements Subscriber { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleConcatWithCompletable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleConcatWithCompletable.java index 134ca01260..081cbf364f 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleConcatWithCompletable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleConcatWithCompletable.java @@ -35,7 +35,8 @@ final class SingleConcatWithCompletable extends AbstractCompletableAndSingleC @Override void delegateSubscribeToOriginal(final Subscriber offloadSubscriber, - final CapturedContext capturedContext, final AsyncContextProvider contextProvider) { + final CapturedContext capturedContext, + final AsyncContextProvider contextProvider) { original.delegateSubscribe(new ConcatWithSubscriber<>(offloadSubscriber, next), capturedContext, contextProvider); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToCompletableFuture.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToCompletableFuture.java index 612da90dad..d09cb8ba99 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToCompletableFuture.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleToCompletableFuture.java @@ -18,7 +18,6 @@ import io.servicetalk.concurrent.Cancellable; import io.servicetalk.concurrent.SingleSource.Subscriber; import io.servicetalk.concurrent.internal.SequentialCancellable; -import io.servicetalk.context.api.ContextMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutSingle.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutSingle.java index efc092e425..f18387e866 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutSingle.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/TimeoutSingle.java @@ -92,7 +92,8 @@ private TimeoutSubscriber(TimeoutSingle parent, Subscriber target, } static TimeoutSubscriber newInstance(TimeoutSingle parent, Subscriber target, - CapturedContext capturedContext, AsyncContextProvider contextProvider) { + CapturedContext capturedContext, + AsyncContextProvider contextProvider) { TimeoutSubscriber s = new TimeoutSubscriber<>(parent, target, contextProvider); Cancellable localTimerCancellable; try { From a91ca4882d361bf62010a4bb2fb9293f1b85378b Mon Sep 17 00:00:00 2001 From: Bryce Anderson Date: Thu, 6 Feb 2025 12:10:49 -0700 Subject: [PATCH 09/14] Actually service load the CaptureContextProviders --- .../concurrent/api/AsyncContext.java | 9 +-- .../api/CapturedContextProvider.java | 3 + .../api/CapturedContextProviders.java | 62 +++++++++++++++++++ 3 files changed, 66 insertions(+), 8 deletions(-) create mode 100644 servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProviders.java diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContext.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContext.java index bad3327ba7..7c3c4bcd15 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContext.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContext.java @@ -20,9 +20,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; import java.util.ConcurrentModificationException; -import java.util.List; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; @@ -70,12 +68,11 @@ public final class AsyncContext { static { CapturedContextProvider capturedContextProvider = null; - for (CapturedContextProvider provider : asyncProviderWrappers()) { + for (CapturedContextProvider provider : CapturedContextProviders.providers()) { if (capturedContextProvider == null) { capturedContextProvider = provider; } else { final CapturedContextProvider finalCapturedContextProvider = capturedContextProvider; - // TODO: at some point this is perhaps better as a list iteration. capturedContextProvider = (context) -> provider.captureContext(finalCapturedContextProvider.captureContext(context)); } @@ -585,8 +582,4 @@ private static void disable0() { EXECUTOR_PLUGINS.remove(EXECUTOR_PLUGIN); LOGGER.info("Disabled. Features that depend on AsyncContext will stop working."); } - - private static List asyncProviderWrappers() { - return Collections.emptyList(); - } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProvider.java index cafa4d5e94..f0c1575109 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProvider.java @@ -56,6 +56,9 @@ interface CapturedContextProvider { * return new CapturedContextImpl(delegate, getMyString()); * } * } + * @param underlying additional context that must be utilized as part of the returned + * {@link CapturedContext}, usually wrapped as described above. + * @return the wrapped {@link CapturedContext}, or the original if there was no additional state captured. */ CapturedContext captureContext(CapturedContext underlying); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProviders.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProviders.java new file mode 100644 index 0000000000..26906350ca --- /dev/null +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProviders.java @@ -0,0 +1,62 @@ +/* + * Copyright © 2025 Apple Inc. and the ServiceTalk project authors + * + * Licensed 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 io.servicetalk.concurrent.api; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.ServiceLoader; + +import static java.util.Collections.emptyList; +import static java.util.Collections.unmodifiableList; + +final class CapturedContextProviders { + + private static final Logger LOGGER = LoggerFactory.getLogger(CapturedContextProviders.class); + private static final List PROVIDERS; + + static { + final ClassLoader classLoader = CapturedContextProviders.class.getClassLoader(); + PROVIDERS = loadProviders(CapturedContextProvider.class, classLoader, LOGGER); + } + + private CapturedContextProviders() { + // no instances + } + + static List providers() { + return PROVIDERS; + } + + // TODO: this was copied from `ServiceLoaderUtils` because the `CapturedContextProvider` interface is package + // private and the call to `ServiceLoader.load(..)` can only load implementations for types that are visible + // to the package from which it is called (because it checks if it's accessible by checking stack trace). + // One we make CapturedContextProvider public we can return to using ServiceLoaderUtils. + private static List loadProviders(final Class clazz, final ClassLoader classLoader, final Logger logger) { + final List list = new ArrayList<>(0); + for (T provider : ServiceLoader.load(clazz, classLoader)) { + list.add(provider); + } + if (list.isEmpty()) { + logger.debug("ServiceLoader {}(s) registered: []", clazz.getSimpleName()); + return emptyList(); + } + logger.info("ServiceLoader {}(s) registered: {}", clazz.getSimpleName(), list); + return unmodifiableList(list); + } +} From 958edf55bf5c6cf8779f4b684f1df5178cb26516 Mon Sep 17 00:00:00 2001 From: Bryce Anderson Date: Thu, 6 Feb 2025 14:42:23 -0700 Subject: [PATCH 10/14] Rename a method to be more clear and add a few tests --- .../concurrent/api/AsyncContextProvider.java | 18 +++++++++++-- .../api/CompletableSetContextOnSubscribe.java | 2 +- .../concurrent/api/CopyOnWriteContextMap.java | 2 +- .../api/DefaultAsyncContextProvider.java | 8 +++--- .../api/NoopAsyncContextProvider.java | 2 +- .../api/PublisherSetContextOnSubscribe.java | 2 +- .../io/servicetalk/concurrent/api/Scope.java | 4 +-- .../api/SingleSetContextOnSubscribe.java | 2 +- .../api/DefaultAsyncContextProviderTest.java | 25 +++++++++++++++++++ 9 files changed, 52 insertions(+), 13 deletions(-) diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java index ea22cfee68..378945ff59 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java @@ -55,11 +55,25 @@ interface AsyncContextProvider { */ void context(ContextMap context); - Scope attachContext(ContextMap contextMap); + /** + * Attach the {@link ContextMap} to the current local scope. + * @param contextMap the {@link ContextMap} to attach. + * @return a {@link Scope} that must be used to restore the previous {@link ContextMap} when the operation is + * complete. + */ + Scope attachContextMap(ContextMap contextMap); + /** + * Capture the current context with the provided {@link ContextMap} as the captured {@link AsyncContext} state. + * @return the captured context to be restored across async boundaries. + */ CapturedContext captureContext(ContextMap contextMap); - // TODO: refactor to remove this + /** + * Capture the current context. This is expected to provide identical results as calling + * {@code captureContext(context());}. + * @return the captured context to be restored across async boundaries. + */ CapturedContext captureContext(); /** diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java index 3cea407b58..b4479b4539 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java @@ -40,7 +40,7 @@ public ContextMap captured() { @Override public Scope restoreContext() { Scope outer = parentContext.restoreContext(); - Scope inner = AsyncContext.provider().attachContext(context); + Scope inner = AsyncContext.provider().attachContextMap(context); return () -> { inner.close(); outer.close(); diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java index eb2ac1e5f6..8b8ace21e2 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java @@ -196,7 +196,7 @@ public ContextMap captured() { @Override public Scope restoreContext() { - return AsyncContext.provider().attachContext(this); + return AsyncContext.provider().attachContextMap(this); } // Scope method diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java index defe5cbfaa..e92b5dc086 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java @@ -76,9 +76,9 @@ public final void context(ContextMap contextMap) { } @Override - public final Scope attachContext(ContextMap contextMap) { + public final Scope attachContextMap(ContextMap contextMap) { ContextMap prev = exchangeContext(contextMap); - return NO_DEBUG_LOGGING && prev instanceof Scope ? (Scope) prev : () -> detachContext(contextMap, prev); + return NO_DEBUG_LOGGING && prev instanceof Scope ? (Scope) prev : () -> detachContextMap(contextMap, prev); } @Override @@ -333,7 +333,7 @@ public ContextMap captured() { @Override public Scope restoreContext() { ContextMap prev = exchangeContext(contextMap); - return NO_DEBUG_LOGGING && prev instanceof Scope ? (Scope) prev : () -> detachContext(contextMap, prev); + return NO_DEBUG_LOGGING && prev instanceof Scope ? (Scope) prev : () -> detachContextMap(contextMap, prev); } } @@ -354,7 +354,7 @@ private static ContextMap exchangeContext(ContextMap contextMap) { return result; } - private static void detachContext(ContextMap expectedContext, ContextMap toRestore) { + private static void detachContextMap(ContextMap expectedContext, ContextMap toRestore) { ContextMap current = exchangeContext(toRestore); if (current != expectedContext && !NO_DEBUG_LOGGING) { LOGGER.debug("Current context didn't match the expected context. current: {}, expected: {}", diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java index 1973bdd545..3b64b28b4d 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java @@ -74,7 +74,7 @@ public void context(@Nullable ContextMap contextMap) { } @Override - public Scope attachContext(ContextMap contextMap) { + public Scope attachContextMap(ContextMap contextMap) { return Scope.NOOP; } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java index 53e9b242d5..79160c62a0 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java @@ -40,7 +40,7 @@ public ContextMap captured() { @Override public Scope restoreContext() { Scope outer = parentContext.restoreContext(); - Scope inner = AsyncContext.provider().attachContext(context); + Scope inner = AsyncContext.provider().attachContextMap(context); return () -> { inner.close(); outer.close(); diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Scope.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Scope.java index 332e7dbca9..45fb75d3d0 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Scope.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Scope.java @@ -21,8 +21,8 @@ * An abstraction for detaching a context from the current thread. * * This abstraction is intended to allow the modifications performed by - * {@link AsyncContextProvider#attachContext(ContextMap)} to be undone. In practice, this may look like restoring - * a {@link ThreadLocal} to the state it had before the call to {@link AsyncContextProvider#attachContext(ContextMap)} + * {@link AsyncContextProvider#attachContextMap(ContextMap)} to be undone. In practice, this may look like restoring + * a {@link ThreadLocal} to the state it had before the call to {@link AsyncContextProvider#attachContextMap(ContextMap)} * call. */ interface Scope extends AutoCloseable { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java index 4d92956d0a..3d7835547c 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java @@ -40,7 +40,7 @@ public ContextMap captured() { @Override public Scope restoreContext() { Scope outer = parentContext.restoreContext(); - Scope inner = AsyncContext.provider().attachContext(context); + Scope inner = AsyncContext.provider().attachContextMap(context); return () -> { inner.close(); outer.close(); diff --git a/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java b/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java index 5f19f9ce10..84ea467beb 100644 --- a/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java +++ b/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java @@ -21,6 +21,7 @@ import io.servicetalk.concurrent.PublisherSource.Subscriber; import io.servicetalk.concurrent.PublisherSource.Subscription; import io.servicetalk.concurrent.SingleSource; +import io.servicetalk.concurrent.internal.DefaultContextMap; import io.servicetalk.concurrent.internal.SubscriberUtils; import io.servicetalk.concurrent.test.internal.TestPublisherSubscriber; import io.servicetalk.context.api.ContextMap; @@ -912,6 +913,30 @@ void eightPutMultiplePermutations() { testPutMultiplePermutations(asList(K1, K2, K3, K4, K5, K6, K7, K8)); } + + @Test + void captureDefaultContextMap() { + testContextMap(new DefaultContextMap()); + } + + @Test + void captureCopyOnWriteContextMap() { + testContextMap(new CopyOnWriteContextMap()); + } + + private static void testContextMap(ContextMap toCapture) { + assertNull(AsyncContext.put(K1, "k1")); + toCapture.put(K1, "k1-override"); + CapturedContext capturedContext = AsyncContext.provider().captureContext(toCapture); + try (Scope ignored = capturedContext.restoreContext()) { + assertEquals("k1-override", AsyncContext.get(K1)); + AsyncContext.put(K2, "k2"); + } + assertEquals("k1", AsyncContext.get(K1)); + assertNull(AsyncContext.get(K2)); + assertEquals("k2", toCapture.get(K2)); + } + private static void testPutMultiplePermutations(List> initialKeys) { final Key[] putKeys = {K1, K2, K3, K4, K5, K6, K7, K8}; for (int i = 0; i < initialKeys.size(); ++i) { From c02ab422c6736c1f3dbb674388af9542490b4284 Mon Sep 17 00:00:00 2001 From: Bryce Anderson Date: Thu, 6 Feb 2025 16:44:44 -0700 Subject: [PATCH 11/14] Fix style checks --- .../src/main/java/io/servicetalk/concurrent/api/Scope.java | 4 ++-- .../concurrent/api/DefaultAsyncContextProviderTest.java | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Scope.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Scope.java index 45fb75d3d0..9cc5024f7f 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Scope.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Scope.java @@ -22,8 +22,8 @@ * * This abstraction is intended to allow the modifications performed by * {@link AsyncContextProvider#attachContextMap(ContextMap)} to be undone. In practice, this may look like restoring - * a {@link ThreadLocal} to the state it had before the call to {@link AsyncContextProvider#attachContextMap(ContextMap)} - * call. + * a {@link ThreadLocal} to the state it had before the call to + * {@link AsyncContextProvider#attachContextMap(ContextMap)}. */ interface Scope extends AutoCloseable { diff --git a/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java b/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java index 84ea467beb..189bfcc425 100644 --- a/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java +++ b/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java @@ -913,7 +913,6 @@ void eightPutMultiplePermutations() { testPutMultiplePermutations(asList(K1, K2, K3, K4, K5, K6, K7, K8)); } - @Test void captureDefaultContextMap() { testContextMap(new DefaultContextMap()); From d5304335760fab7a5670b3f73333747fca60b382 Mon Sep 17 00:00:00 2001 From: Bryce Anderson Date: Thu, 6 Feb 2025 17:59:24 -0700 Subject: [PATCH 12/14] Some feedback --- .../concurrent/api/CapturedContext.java | 20 ++++++---- .../api/CapturedContextProvider.java | 7 ++-- .../concurrent/api/Completable.java | 2 +- .../api/CompletableSetContextOnSubscribe.java | 4 +- .../CompletableShareContextOnSubscribe.java | 2 +- .../api/ContextPreservingBiConsumer.java | 2 +- .../api/ContextPreservingBiFunction.java | 2 +- .../api/ContextPreservingCallable.java | 2 +- .../api/ContextPreservingCancellable.java | 2 +- ...ontextPreservingCompletableSubscriber.java | 6 +-- .../api/ContextPreservingConsumer.java | 2 +- .../api/ContextPreservingFunction.java | 2 +- .../api/ContextPreservingRunnable.java | 2 +- .../ContextPreservingSingleSubscriber.java | 6 +-- .../api/ContextPreservingSubscriber.java | 8 ++-- .../api/ContextPreservingSubscription.java | 4 +- .../concurrent/api/CopyOnWriteContextMap.java | 2 +- .../api/DefaultAsyncContextProvider.java | 40 +++++++++++++------ .../api/NoopAsyncContextProvider.java | 35 ++++++++-------- .../servicetalk/concurrent/api/Publisher.java | 2 +- .../api/PublisherSetContextOnSubscribe.java | 4 +- .../io/servicetalk/concurrent/api/Single.java | 2 +- .../api/SingleSetContextOnSubscribe.java | 4 +- .../api/DefaultAsyncContextProviderTest.java | 2 +- 24 files changed, 93 insertions(+), 71 deletions(-) diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java index 9af3dcb851..203119f49f 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContext.java @@ -18,21 +18,25 @@ import io.servicetalk.context.api.ContextMap; /** - * An interface representing the restoration of the thread-local like context that can be restored later - * during an async operation. + * A representation of a context state that can be attached to the current thread. + * + * Instances represent captured context state which includes things like the {@link AsyncContext} state and potentially + * additional state if instances of {@link CapturedContextProvider} are found. This state can be attached to the current + * thread via the {@link CapturedContext#attachContext()} method which will return a {@link Scope} used to detach this + * state, restoring any context information that existed beforehand. */ interface CapturedContext { /** - * The {@link ContextMap} that was captured as part of the context. - * @return {@link ContextMap} that was captured as part of the context. + * The {@link ContextMap} that was captured. + * @return {@link ContextMap} that was captured. */ ContextMap captured(); /** - * Restore the thread-local like context. - * @return a {@link Scope} that will revert the restoration and return the thread-local like state to the state - * that it had before restoring this context. + * Attach the captured context to the thread state. + * @return a {@link Scope} that will be used to restore the previous context associated with the current thread + * state when the scoped operation completes. */ - Scope restoreContext(); + Scope attachContext(); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProvider.java index f0c1575109..6a48ca5a7b 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CapturedContextProvider.java @@ -18,12 +18,13 @@ /** * Functionality related to capturing thread-local like context for later restoration across async boundaries. */ +@FunctionalInterface interface CapturedContextProvider { /** - * Save existing context in preparation for an asynchronous thread jump. + * Capture existing context in preparation for an asynchronous thread jump. * - * If you want to save any external state you can create a wrapper {@link CapturedContext} to add additional + * If you want to capture any external state you can create a wrapper {@link CapturedContext} to add additional * state capturing to the context pathway. This state can then be restored by wrapping the {@link CapturedContext} * with the additional functionality to restore and finally revert the context state. *

@@ -35,7 +36,7 @@ interface CapturedContextProvider { * * Scope restoreContext() { * String old = getMyString(); - * MY_THREAD_LOCAL.set(state); + * setMyString(state); * Scope outer = delegate.restoreContext(); * return () -> { * outer.close(); diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Completable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Completable.java index 45dd7e5a9e..c8e9e7c333 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Completable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Completable.java @@ -1749,7 +1749,7 @@ protected final void subscribeInternal(Subscriber subscriber) { handleSubscribe(wrapped, capturedContext, contextProvider); } else { // Ensure that AsyncContext used for handleSubscribe() is the contextMap for the subscribe() - try (Scope unused = capturedContext.restoreContext()) { + try (Scope unused = capturedContext.attachContext()) { handleSubscribe(wrapped, capturedContext, contextProvider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java index b4479b4539..d07837a337 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java @@ -38,8 +38,8 @@ public ContextMap captured() { } @Override - public Scope restoreContext() { - Scope outer = parentContext.restoreContext(); + public Scope attachContext() { + Scope outer = parentContext.attachContext(); Scope inner = AsyncContext.provider().attachContextMap(context); return () -> { inner.close(); diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableShareContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableShareContextOnSubscribe.java index bc3b743a18..84dd669fbb 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableShareContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableShareContextOnSubscribe.java @@ -24,7 +24,7 @@ final class CompletableShareContextOnSubscribe extends AbstractNoHandleSubscribe @Override CapturedContext contextForSubscribe(AsyncContextProvider provider) { - return provider.captureContext(provider.context()); + return provider.captureContext(); } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiConsumer.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiConsumer.java index 5ac97ff41c..629bf9cb3d 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiConsumer.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiConsumer.java @@ -36,7 +36,7 @@ final class ContextPreservingBiConsumer implements BiConsumer { @Override public void accept(T t, U u) { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { delegate.accept(t, u); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java index 3de5388f67..a24188203d 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingBiFunction.java @@ -36,7 +36,7 @@ final class ContextPreservingBiFunction implements BiFunction @Override public V apply(T t, U u) { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { return delegate.apply(t, u); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCallable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCallable.java index fe8d31cd5d..58a8e5ca35 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCallable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCallable.java @@ -39,7 +39,7 @@ final class ContextPreservingCallable implements Callable { @Override public V call() throws Exception { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { return delegate.call(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellable.java index 20a5d41cda..b3d4a72c4a 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCancellable.java @@ -42,7 +42,7 @@ static Cancellable wrap(Cancellable delegate, CapturedContext capturedContext) { @Override public void cancel() { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { delegate.cancel(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriber.java index 217ddc2351..60b5384ca0 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingCompletableSubscriber.java @@ -39,21 +39,21 @@ void invokeOnSubscribe(Cancellable cancellable) { @Override public final void onSubscribe(final Cancellable cancellable) { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { invokeOnSubscribe(cancellable); } } @Override public final void onComplete() { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { subscriber.onComplete(); } } @Override public final void onError(Throwable t) { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { subscriber.onError(t); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingConsumer.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingConsumer.java index 7aa1064943..f74746f9f3 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingConsumer.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingConsumer.java @@ -35,7 +35,7 @@ final class ContextPreservingConsumer implements Consumer { @Override public void accept(T t) { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { delegate.accept(t); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingFunction.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingFunction.java index 21aff097fc..8b4b0fb8df 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingFunction.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingFunction.java @@ -35,7 +35,7 @@ final class ContextPreservingFunction implements Function { @Override public U apply(T t) { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { return delegate.apply(t); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingRunnable.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingRunnable.java index e97dd04941..41208e372e 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingRunnable.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingRunnable.java @@ -37,7 +37,7 @@ final class ContextPreservingRunnable implements Runnable { @Override public void run() { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { delegate.run(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriber.java index dc4242f77c..1185392897 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSingleSubscriber.java @@ -42,21 +42,21 @@ void invokeOnSubscribe(Cancellable cancellable) { @Override public final void onSubscribe(Cancellable cancellable) { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { invokeOnSubscribe(cancellable); } } @Override public final void onSuccess(@Nullable T result) { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { subscriber.onSuccess(result); } } @Override public final void onError(Throwable t) { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { subscriber.onError(t); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriber.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriber.java index b440c8ed25..70fddeaf18 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriber.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscriber.java @@ -39,28 +39,28 @@ void invokeOnSubscribe(Subscription s) { @Override public final void onSubscribe(Subscription s) { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { invokeOnSubscribe(s); } } @Override public final void onNext(T t) { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { subscriber.onNext(t); } } @Override public final void onError(Throwable t) { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { subscriber.onError(t); } } @Override public final void onComplete() { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { subscriber.onComplete(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscription.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscription.java index 8d3436135d..099f05e166 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscription.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/ContextPreservingSubscription.java @@ -40,14 +40,14 @@ static Subscription wrap(Subscription subscription, CapturedContext current) { @Override public void request(long l) { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { subscription.request(l); } } @Override public void cancel() { - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { subscription.cancel(); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java index 8b8ace21e2..2fd2270d32 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java @@ -195,7 +195,7 @@ public ContextMap captured() { } @Override - public Scope restoreContext() { + public Scope attachContext() { return AsyncContext.provider().attachContextMap(this); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java index e92b5dc086..d8708a286d 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java @@ -77,8 +77,7 @@ public final void context(ContextMap contextMap) { @Override public final Scope attachContextMap(ContextMap contextMap) { - ContextMap prev = exchangeContext(contextMap); - return NO_DEBUG_LOGGING && prev instanceof Scope ? (Scope) prev : () -> detachContextMap(contextMap, prev); + return doAttachContextMap(contextMap); } @Override @@ -331,12 +330,35 @@ public ContextMap captured() { } @Override - public Scope restoreContext() { - ContextMap prev = exchangeContext(contextMap); - return NO_DEBUG_LOGGING && prev instanceof Scope ? (Scope) prev : () -> detachContextMap(contextMap, prev); + public Scope attachContext() { + return doAttachContextMap(contextMap); } } + private static final class DetachScope implements Scope { + private final ContextMap expectedContext; + private final ContextMap toRestore; + + DetachScope(ContextMap expectedContext, ContextMap toRestore) { + this.expectedContext = expectedContext; + this.toRestore = toRestore; + } + + @Override + public void close() { + ContextMap current = exchangeContext(toRestore); + if (current != expectedContext && !NO_DEBUG_LOGGING) { + LOGGER.debug("Current context didn't match the expected context. current: {}, expected: {}", + current, expectedContext, new Throwable("stack trace")); + } + } + } + + private static Scope doAttachContextMap(ContextMap contextMap) { + ContextMap prev = exchangeContext(contextMap); + return NO_DEBUG_LOGGING && prev instanceof Scope ? (Scope) prev : new DetachScope(contextMap, prev); + } + private static ContextMap exchangeContext(ContextMap contextMap) { final Thread currentThread = Thread.currentThread(); ContextMap result; @@ -354,14 +376,6 @@ private static ContextMap exchangeContext(ContextMap contextMap) { return result; } - private static void detachContextMap(ContextMap expectedContext, ContextMap toRestore) { - ContextMap current = exchangeContext(toRestore); - if (current != expectedContext && !NO_DEBUG_LOGGING) { - LOGGER.debug("Current context didn't match the expected context. current: {}, expected: {}", - current, expectedContext, new Throwable("stack trace")); - } - } - private static ContextMap newContextMap() { return new CopyOnWriteContextMap(); } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java index 3b64b28b4d..36bd4c7f7b 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java @@ -37,18 +37,6 @@ final class NoopAsyncContextProvider implements AsyncContextProvider { static final AsyncContextProvider INSTANCE = new NoopAsyncContextProvider(); - private static final CapturedContext NOOP_CAPTURED_CONTEXT = new CapturedContext() { - @Override - public ContextMap captured() { - return NoopContextMap.INSTANCE; - } - - @Override - public Scope restoreContext() { - return Scope.NOOP; - } - }; - private NoopAsyncContextProvider() { // singleton } @@ -60,12 +48,12 @@ public ContextMap context() { @Override public CapturedContext captureContext() { - return NOOP_CAPTURED_CONTEXT; + return NoopContextMap.INSTANCE; } @Override public CapturedContext captureContext(ContextMap contextMap) { - return NOOP_CAPTURED_CONTEXT; + return NoopContextMap.INSTANCE; } @Override @@ -187,8 +175,8 @@ public BiFunction wrapBiFunction(final BiFunction fu return func; } - static final class NoopContextMap implements ContextMap { - static final ContextMap INSTANCE = new NoopContextMap(); + private static final class NoopContextMap implements ContextMap, CapturedContext, Scope { + static final NoopContextMap INSTANCE = new NoopContextMap(); private NoopContextMap() { // Singleton @@ -298,6 +286,21 @@ public boolean equals(final Object o) { return ((ContextMap) o).isEmpty(); } + @Override + public ContextMap captured() { + return this; + } + + @Override + public Scope attachContext() { + return this; + } + + @Override + public void close() { + // noop + } + @Override public String toString() { return ContextMapUtils.toString(this); diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Publisher.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Publisher.java index 9854d203ca..07e6151ae1 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Publisher.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Publisher.java @@ -4374,7 +4374,7 @@ protected void subscribeInternal(Subscriber subscriber) { handleSubscribe(wrapped, capturedContext, provider); } else { // Ensure that AsyncContext used for handleSubscribe() is the contextMap for the subscribe() - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { handleSubscribe(wrapped, capturedContext, provider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java index 79160c62a0..adcf54ba72 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java @@ -38,8 +38,8 @@ public ContextMap captured() { } @Override - public Scope restoreContext() { - Scope outer = parentContext.restoreContext(); + public Scope attachContext() { + Scope outer = parentContext.attachContext(); Scope inner = AsyncContext.provider().attachContextMap(context); return () -> { inner.close(); diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Single.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Single.java index ba63e7bb32..3aec619324 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Single.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Single.java @@ -2693,7 +2693,7 @@ final CapturedContext subscribeAndReturnContext(Subscriber subscriber handleSubscribe(wrapped, capturedContext, provider); } else { // Ensure that AsyncContext used for handleSubscribe() is the contextMap for the subscribe() - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { handleSubscribe(wrapped, capturedContext, provider); } } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java index 3d7835547c..231f8d44ac 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java @@ -38,8 +38,8 @@ public ContextMap captured() { } @Override - public Scope restoreContext() { - Scope outer = parentContext.restoreContext(); + public Scope attachContext() { + Scope outer = parentContext.attachContext(); Scope inner = AsyncContext.provider().attachContextMap(context); return () -> { inner.close(); diff --git a/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java b/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java index 189bfcc425..21d5148ed2 100644 --- a/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java +++ b/servicetalk-concurrent-api/src/test/java/io/servicetalk/concurrent/api/DefaultAsyncContextProviderTest.java @@ -927,7 +927,7 @@ private static void testContextMap(ContextMap toCapture) { assertNull(AsyncContext.put(K1, "k1")); toCapture.put(K1, "k1-override"); CapturedContext capturedContext = AsyncContext.provider().captureContext(toCapture); - try (Scope ignored = capturedContext.restoreContext()) { + try (Scope ignored = capturedContext.attachContext()) { assertEquals("k1-override", AsyncContext.get(K1)); AsyncContext.put(K2, "k2"); } From 36bcc82bf124208af5f75e5e81b44061567ee32f Mon Sep 17 00:00:00 2001 From: Bryce Anderson Date: Thu, 6 Feb 2025 18:02:19 -0700 Subject: [PATCH 13/14] Fix *SetContextOnSubscribe implementations --- .../api/CompletableSetContextOnSubscribe.java | 18 +----------------- .../api/PublisherSetContextOnSubscribe.java | 18 +----------------- .../api/SingleSetContextOnSubscribe.java | 18 +----------------- 3 files changed, 3 insertions(+), 51 deletions(-) diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java index d07837a337..0e905f2a00 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CompletableSetContextOnSubscribe.java @@ -30,23 +30,7 @@ final class CompletableSetContextOnSubscribe extends AbstractNoHandleSubscribeCo @Override CapturedContext contextForSubscribe(AsyncContextProvider provider) { - CapturedContext parentContext = super.contextForSubscribe(provider); - return new CapturedContext() { - @Override - public ContextMap captured() { - return context; - } - - @Override - public Scope attachContext() { - Scope outer = parentContext.attachContext(); - Scope inner = AsyncContext.provider().attachContextMap(context); - return () -> { - inner.close(); - outer.close(); - }; - } - }; + return provider.captureContext(context); } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java index adcf54ba72..4dbba5750a 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/PublisherSetContextOnSubscribe.java @@ -30,23 +30,7 @@ final class PublisherSetContextOnSubscribe extends AbstractNoHandleSubscribeP @Override CapturedContext contextForSubscribe(AsyncContextProvider provider) { - CapturedContext parentContext = super.contextForSubscribe(provider); - return new CapturedContext() { - @Override - public ContextMap captured() { - return context; - } - - @Override - public Scope attachContext() { - Scope outer = parentContext.attachContext(); - Scope inner = AsyncContext.provider().attachContextMap(context); - return () -> { - inner.close(); - outer.close(); - }; - } - }; + return provider.captureContext(context); } @Override diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java index 231f8d44ac..e8d53e499e 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/SingleSetContextOnSubscribe.java @@ -30,23 +30,7 @@ final class SingleSetContextOnSubscribe extends AbstractNoHandleSubscribeSing @Override CapturedContext contextForSubscribe(AsyncContextProvider provider) { - CapturedContext parentContext = super.contextForSubscribe(provider); - return new CapturedContext() { - @Override - public ContextMap captured() { - return context; - } - - @Override - public Scope attachContext() { - Scope outer = parentContext.attachContext(); - Scope inner = AsyncContext.provider().attachContextMap(context); - return () -> { - inner.close(); - outer.close(); - }; - } - }; + return provider.captureContext(context); } @Override From 3609778f09e940eb04ef62cfaa424a22da2f39bf Mon Sep 17 00:00:00 2001 From: Bryce Anderson Date: Fri, 7 Feb 2025 09:23:40 -0700 Subject: [PATCH 14/14] Rename context(ContextMap) to setContextMap(ContextMap) and fix some doc strings --- .../servicetalk/concurrent/api/AsyncContextProvider.java | 8 +++++--- .../servicetalk/concurrent/api/CopyOnWriteContextMap.java | 7 +++++-- .../concurrent/api/DefaultAsyncContextProvider.java | 2 +- .../concurrent/api/NoopAsyncContextProvider.java | 2 +- .../main/java/io/servicetalk/concurrent/api/Scope.java | 7 ++----- 5 files changed, 14 insertions(+), 12 deletions(-) diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java index 378945ff59..7fcf3b5484 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/AsyncContextProvider.java @@ -48,12 +48,14 @@ interface AsyncContextProvider { ContextMap context(); /** - * Set the {@link ContextMap}. + * Set the async context {@link ContextMap}. + * + * Note that unlike {@link AsyncContextProvider#attachContextMap(ContextMap)}, this method does not provide a + * {@link Scope} to restore the state to what it was previously. * * @param context the new value for {@link ContextMap}. - * @return {@code this}. */ - void context(ContextMap context); + void setContextMap(ContextMap context); /** * Attach the {@link ContextMap} to the current local scope. diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java index 2fd2270d32..d6419fa4b1 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/CopyOnWriteContextMap.java @@ -194,15 +194,18 @@ public ContextMap captured() { return this; } + // CapturedContext method. For the base async context implementation, attaching means setting the correct + // AsyncContextProvider thread-local to _this_ ContextMap instance. @Override public Scope attachContext() { return AsyncContext.provider().attachContextMap(this); } - // Scope method + // Scope method. For the base async context implementation, the `prev` map instance is returned as the Scope if that + // is the context state that needs to be restored and restoring just means setting it to the thread-local. @Override public void close() { - AsyncContext.provider().context(this); + AsyncContext.provider().setContextMap(this); } private interface CopyContextMap { diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java index d8708a286d..af79af7144 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/DefaultAsyncContextProvider.java @@ -65,7 +65,7 @@ public final ContextMap context() { } @Override - public final void context(ContextMap contextMap) { + public final void setContextMap(ContextMap contextMap) { final Thread currentThread = Thread.currentThread(); if (currentThread instanceof ContextMapHolder) { final ContextMapHolder asyncContextMapHolder = (ContextMapHolder) currentThread; diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java index 36bd4c7f7b..8458a71e8b 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/NoopAsyncContextProvider.java @@ -57,7 +57,7 @@ public CapturedContext captureContext(ContextMap contextMap) { } @Override - public void context(@Nullable ContextMap contextMap) { + public void setContextMap(@Nullable ContextMap contextMap) { // noop } diff --git a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Scope.java b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Scope.java index 9cc5024f7f..b78c4ea5e6 100644 --- a/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Scope.java +++ b/servicetalk-concurrent-api/src/main/java/io/servicetalk/concurrent/api/Scope.java @@ -15,15 +15,12 @@ */ package io.servicetalk.concurrent.api; -import io.servicetalk.context.api.ContextMap; - /** * An abstraction for detaching a context from the current thread. * * This abstraction is intended to allow the modifications performed by - * {@link AsyncContextProvider#attachContextMap(ContextMap)} to be undone. In practice, this may look like restoring - * a {@link ThreadLocal} to the state it had before the call to - * {@link AsyncContextProvider#attachContextMap(ContextMap)}. + * {@link CapturedContext#attachContext()} to be undone. In practice, this may look like restoring a {@link ThreadLocal} + * to the state it had before the call to {@link CapturedContext#attachContext()}. */ interface Scope extends AutoCloseable {